1
0
Fork 0
mirror of https://github.com/ACINQ/eclair.git synced 2025-02-22 22:25:26 +01:00

Make UpfrontShutdownScript a TLV record (#1333)

Make DLP data mandatory in ChannelReestablish.
We make them mandatory to allow extending the message with TLVs.

Make upfront_shutdown_script a TLV record that we always include in
open_channel / accept_channel.

See https://github.com/lightningnetwork/lightning-rfc/pull/714.
This commit is contained in:
Bastien Teinturier 2020-02-28 10:32:15 +01:00 committed by GitHub
parent e2b16302ea
commit 35051d0535
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
11 changed files with 140 additions and 135 deletions

View file

@ -32,6 +32,7 @@ import fr.acinq.eclair.payment.relay.{CommandBuffer, Origin, Relayer}
import fr.acinq.eclair.router.Announcements
import fr.acinq.eclair.transactions._
import fr.acinq.eclair.wire._
import scodec.bits.ByteVector
import scala.collection.immutable.Queue
import scala.compat.Platform
@ -168,7 +169,10 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
delayedPaymentBasepoint = keyManager.delayedPaymentPoint(channelKeyPath).publicKey,
htlcBasepoint = keyManager.htlcPoint(channelKeyPath).publicKey,
firstPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, 0),
channelFlags = channelFlags)
channelFlags = channelFlags,
// In order to allow TLV extensions and keep backwards-compatibility, we include an empty upfront_shutdown_script.
// See https://github.com/lightningnetwork/lightning-rfc/pull/714.
tlvStream = TlvStream(ChannelTlv.UpfrontShutdownScript(ByteVector.empty)))
goto(WAIT_FOR_ACCEPT_CHANNEL) using DATA_WAIT_FOR_ACCEPT_CHANNEL(initFunder, open) sending open
case Event(inputFundee@INPUT_INIT_FUNDEE(_, localParams, remote, _), Nothing) if !localParams.isFunder =>
@ -292,7 +296,10 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
paymentBasepoint = keyManager.paymentPoint(channelKeyPath).publicKey,
delayedPaymentBasepoint = keyManager.delayedPaymentPoint(channelKeyPath).publicKey,
htlcBasepoint = keyManager.htlcPoint(channelKeyPath).publicKey,
firstPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, 0))
firstPerCommitmentPoint = keyManager.commitmentPoint(channelKeyPath, 0),
// In order to allow TLV extensions and keep backwards-compatibility, we include an empty upfront_shutdown_script.
// See https://github.com/lightningnetwork/lightning-rfc/pull/714.
tlvStream = TlvStream(ChannelTlv.UpfrontShutdownScript(ByteVector.empty)))
val remoteParams = RemoteParams(
nodeId = remoteNodeId,
dustLimit = open.dustLimitSatoshis,
@ -1387,8 +1394,8 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
channelId = d.channelId,
nextLocalCommitmentNumber = d.commitments.localCommit.index + 1,
nextRemoteRevocationNumber = d.commitments.remoteCommit.index,
yourLastPerCommitmentSecret = Some(PrivateKey(yourLastPerCommitmentSecret)),
myCurrentPerCommitmentPoint = Some(myCurrentPerCommitmentPoint)
yourLastPerCommitmentSecret = PrivateKey(yourLastPerCommitmentSecret),
myCurrentPerCommitmentPoint = myCurrentPerCommitmentPoint
)
// we update local/remote connection-local global/local features, we don't persist it right now
@ -1450,7 +1457,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
var sendQueue = Queue.empty[LightningMessage]
val channelKeyPath = keyManager.channelKeyPath(d.commitments.localParams, d.commitments.channelVersion)
channelReestablish match {
case ChannelReestablish(_, _, nextRemoteRevocationNumber, Some(yourLastPerCommitmentSecret), _) if !Helpers.checkLocalCommit(d, nextRemoteRevocationNumber) =>
case ChannelReestablish(_, _, nextRemoteRevocationNumber, yourLastPerCommitmentSecret, _) if !Helpers.checkLocalCommit(d, nextRemoteRevocationNumber) =>
// if next_remote_revocation_number is greater than our local commitment index, it means that either we are using an outdated commitment, or they are lying
// but first we need to make sure that the last per_commitment_secret that they claim to have received from us is correct for that next_remote_revocation_number minus 1
if (keyManager.commitmentSecret(channelKeyPath, nextRemoteRevocationNumber - 1) == yourLastPerCommitmentSecret) {
@ -2040,7 +2047,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
def handleRemoteSpentFuture(commitTx: Transaction, d: DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT) = {
log.warning(s"they published their future commit (because we asked them to) in txid=${commitTx.txid}")
// if we are in this state, then this field is defined
val remotePerCommitmentPoint = d.remoteChannelReestablish.myCurrentPerCommitmentPoint.get
val remotePerCommitmentPoint = d.remoteChannelReestablish.myCurrentPerCommitmentPoint
val remoteCommitPublished = Helpers.Closing.claimRemoteCommitMainOutput(keyManager, d.commitments, remotePerCommitmentPoint, commitTx, nodeParams.onChainFeeConf.feeEstimator, nodeParams.onChainFeeConf.feeTargets)
val nextData = DATA_CLOSING(d.commitments, fundingTx = None, waitingSince = now, Nil, futureRemoteCommitPublished = Some(remoteCommitPublished))

View file

@ -0,0 +1,56 @@
/*
* Copyright 2019 ACINQ SAS
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package fr.acinq.eclair.wire
import fr.acinq.eclair.UInt64
import fr.acinq.eclair.wire.CommonCodecs._
import fr.acinq.eclair.wire.TlvCodecs.tlvStream
import scodec.Codec
import scodec.bits.ByteVector
import scodec.codecs._
sealed trait OpenChannelTlv extends Tlv
sealed trait AcceptChannelTlv extends Tlv
object ChannelTlv {
/** Commitment to where the funds will go in case of a mutual close, which remote node will enforce in case we're compromised. */
case class UpfrontShutdownScript(script: ByteVector) extends OpenChannelTlv with AcceptChannelTlv {
val isEmpty: Boolean = script.isEmpty
}
}
object OpenChannelTlv {
import ChannelTlv._
val openTlvCodec: Codec[TlvStream[OpenChannelTlv]] = tlvStream(discriminated[OpenChannelTlv].by(varint)
.typecase(UInt64(0), variableSizeBytesLong(varintoverflow, bytes).as[UpfrontShutdownScript])
)
}
object AcceptChannelTlv {
import ChannelTlv._
val acceptTlvCodec: Codec[TlvStream[AcceptChannelTlv]] = tlvStream(discriminated[AcceptChannelTlv].by(varint)
.typecase(UInt64(0), variableSizeBytesLong(varintoverflow, bytes).as[UpfrontShutdownScript])
)
}

View file

@ -20,9 +20,9 @@ import fr.acinq.eclair.wire.CommonCodecs._
import fr.acinq.eclair.{KamonExt, wire}
import kamon.Kamon
import kamon.tag.TagSet
import scodec.bits.{BitVector, ByteVector, HexStringSyntax}
import scodec.bits.{BitVector, ByteVector}
import scodec.codecs._
import scodec.{Attempt, Codec, DecodeResult}
import scodec.{Attempt, Codec}
/**
* Created by PM on 15/11/2016.
@ -56,33 +56,10 @@ object LightningMessageCodecs {
("channelId" | bytes32) ::
("nextLocalCommitmentNumber" | uint64overflow) ::
("nextRemoteRevocationNumber" | uint64overflow) ::
("yourLastPerCommitmentSecret" | optional(bitsRemaining, privateKey)) ::
("myCurrentPerCommitmentPoint" | optional(bitsRemaining, publicKey))).as[ChannelReestablish]
("yourLastPerCommitmentSecret" | privateKey) ::
("myCurrentPerCommitmentPoint" | publicKey)).as[ChannelReestablish]
// Legacy nodes may encode an empty upfront_shutdown_script (0x0000) even if we didn't advertise support for option_upfront_shutdown_script.
// To allow extending all messages with TLV streams, the upfront_shutdown_script field was made mandatory in https://github.com/lightningnetwork/lightning-rfc/pull/714.
// This codec decodes both legacy and new versions, while always encoding with an upfront_shutdown_script (of length 0 if none actually provided).
private val shutdownScriptGuard = Codec[Boolean](
// Similar to bitsRemaining but encodes 0x0000 for an empty upfront_shutdown_script.
(included: Boolean) => if (included) Attempt.Successful(BitVector.empty) else Attempt.Successful(hex"0000".bits),
// Bolt 2 specifies that upfront_shutdown_scripts must be P2PKH/P2SH or segwit-v0 P2WPK/P2WSH.
// The length of such scripts will always start with 0x00.
// On top of that, since TLV records start with a varint, a TLV stream will never start with 0x00 unless the spec
// assigns TLV type 0 to a new record. If that happens, that record should be the upfront_shutdown_script to allow
// easy backwards-compatibility (as proposed here: https://github.com/lightningnetwork/lightning-rfc/pull/714).
// That means we can discriminate on byte 0x00 to know whether we're decoding an upfront_shutdown_script or a TLV
// stream.
(b: BitVector) => Attempt.successful(DecodeResult(b.startsWith(hex"00".bits), b))
)
private def emptyToNone(script: Option[ByteVector]): Option[ByteVector] = script match {
case Some(s) if s.nonEmpty => script
case _ => None
}
private val upfrontShutdownScript = optional(shutdownScriptGuard, variableSizeBytes(uint16, bytes)).xmap(emptyToNone, emptyToNone)
private def openChannelCodec_internal(upfrontShutdownScriptCodec: Codec[Option[ByteVector]]): Codec[OpenChannel] = (
val openChannelCodec: Codec[OpenChannel] = (
("chainHash" | bytes32) ::
("temporaryChannelId" | bytes32) ::
("fundingSatoshis" | satoshi) ::
@ -101,19 +78,7 @@ object LightningMessageCodecs {
("htlcBasepoint" | publicKey) ::
("firstPerCommitmentPoint" | publicKey) ::
("channelFlags" | byte) ::
("upfront_shutdown_script" | upfrontShutdownScriptCodec) ::
("tlvStream_opt" | optional(bitsRemaining, OpenTlv.openTlvCodec))).as[OpenChannel]
val openChannelCodec = Codec[OpenChannel](
(open: OpenChannel) => {
// Phoenix versions <= 1.1.0 don't support the upfront_shutdown_script field (they interpret it as a tlv stream
// with an unknown tlv record). For these channels we use an encoding that omits the upfront_shutdown_script for
// backwards-compatibility (once enough Phoenix users have upgraded, we can remove work-around).
val upfrontShutdownScriptCodec = if (open.tlvStream_opt.isDefined) provide(Option.empty[ByteVector]) else upfrontShutdownScript
openChannelCodec_internal(upfrontShutdownScriptCodec).encode(open)
},
(bits: BitVector) => openChannelCodec_internal(upfrontShutdownScript).decode(bits)
)
("tlvStream" | OpenChannelTlv.openTlvCodec)).as[OpenChannel]
val acceptChannelCodec: Codec[AcceptChannel] = (
("temporaryChannelId" | bytes32) ::
@ -130,7 +95,7 @@ object LightningMessageCodecs {
("delayedPaymentBasepoint" | publicKey) ::
("htlcBasepoint" | publicKey) ::
("firstPerCommitmentPoint" | publicKey) ::
("upfront_shutdown_script" | upfrontShutdownScript)).as[AcceptChannel]
("tlvStream" | AcceptChannelTlv.acceptTlvCodec)).as[AcceptChannel]
val fundingCreatedCodec: Codec[FundingCreated] = (
("temporaryChannelId" | bytes32) ::

View file

@ -64,8 +64,8 @@ case class Pong(data: ByteVector) extends SetupMessage
case class ChannelReestablish(channelId: ByteVector32,
nextLocalCommitmentNumber: Long,
nextRemoteRevocationNumber: Long,
yourLastPerCommitmentSecret: Option[PrivateKey] = None,
myCurrentPerCommitmentPoint: Option[PublicKey] = None) extends ChannelMessage with HasChannelId
yourLastPerCommitmentSecret: PrivateKey,
myCurrentPerCommitmentPoint: PublicKey) extends ChannelMessage with HasChannelId
case class OpenChannel(chainHash: ByteVector32,
temporaryChannelId: ByteVector32,
@ -85,8 +85,7 @@ case class OpenChannel(chainHash: ByteVector32,
htlcBasepoint: PublicKey,
firstPerCommitmentPoint: PublicKey,
channelFlags: Byte,
upfrontShutdownScript: Option[ByteVector] = None,
tlvStream_opt: Option[TlvStream[OpenTlv]] = None) extends ChannelMessage with HasTemporaryChannelId with HasChainHash
tlvStream: TlvStream[OpenChannelTlv] = TlvStream.empty) extends ChannelMessage with HasTemporaryChannelId with HasChainHash
case class AcceptChannel(temporaryChannelId: ByteVector32,
dustLimitSatoshis: Satoshi,
@ -102,7 +101,7 @@ case class AcceptChannel(temporaryChannelId: ByteVector32,
delayedPaymentBasepoint: PublicKey,
htlcBasepoint: PublicKey,
firstPerCommitmentPoint: PublicKey,
upfrontShutdownScript: Option[ByteVector] = None) extends ChannelMessage with HasTemporaryChannelId
tlvStream: TlvStream[AcceptChannelTlv] = TlvStream.empty) extends ChannelMessage with HasTemporaryChannelId
case class FundingCreated(temporaryChannelId: ByteVector32,
fundingTxid: ByteVector32,

View file

@ -1,36 +0,0 @@
/*
* Copyright 2019 ACINQ SAS
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package fr.acinq.eclair.wire
import fr.acinq.eclair.UInt64
import fr.acinq.eclair.wire.CommonCodecs._
import fr.acinq.eclair.wire.TlvCodecs.tlvStream
import scodec.Codec
import scodec.bits.ByteVector
import scodec.codecs._
sealed trait OpenTlv extends Tlv
object OpenTlv {
case class Placeholder(b: ByteVector) extends OpenTlv
val openTlvCodec: Codec[TlvStream[OpenTlv]] = tlvStream(discriminated.by(varint)
.typecase(UInt64(65717), variableSizeBytesLong(varintoverflow, bytes).as[Placeholder])
)
}

View file

@ -98,13 +98,13 @@ class RecoverySpec extends TestkitBaseClass with StateTestsHelperMethods {
val fundingPubKey = Seq(PublicKey(pub1), PublicKey(pub2)).find {
pub =>
val channelKeyPath = KeyManager.channelKeyPath(pub)
val localPubkey = Generators.derivePubKey(keyManager.paymentPoint(channelKeyPath).publicKey, ce.myCurrentPerCommitmentPoint.get)
val localPubkey = Generators.derivePubKey(keyManager.paymentPoint(channelKeyPath).publicKey, ce.myCurrentPerCommitmentPoint)
localPubkey.hash160 == pubKeyHash
} get
// compute our to-remote pubkey
val channelKeyPath = KeyManager.channelKeyPath(fundingPubKey)
val ourToRemotePubKey = Generators.derivePubKey(keyManager.paymentPoint(channelKeyPath).publicKey, ce.myCurrentPerCommitmentPoint.get)
val ourToRemotePubKey = Generators.derivePubKey(keyManager.paymentPoint(channelKeyPath).publicKey, ce.myCurrentPerCommitmentPoint)
// spend our output
val tx = Transaction(version = 2,
@ -115,7 +115,7 @@ class RecoverySpec extends TestkitBaseClass with StateTestsHelperMethods {
val sig = keyManager.sign(
ClaimP2WPKHOutputTx(InputInfo(OutPoint(bobCommitTx, bobCommitTx.txOut.indexOf(ourOutput)), ourOutput, Script.pay2pkh(ourToRemotePubKey)), tx),
keyManager.paymentPoint(channelKeyPath),
ce.myCurrentPerCommitmentPoint.get)
ce.myCurrentPerCommitmentPoint)
val tx1 = tx.updateWitness(0, ScriptWitness(Scripts.der(sig) :: ourToRemotePubKey.value :: Nil))
Transaction.correctlySpends(tx1, bobCommitTx :: Nil, ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)
}

View file

@ -23,7 +23,7 @@ import fr.acinq.eclair.blockchain.{MakeFundingTxResponse, TestWallet}
import fr.acinq.eclair.channel.Channel.TickChannelOpenTimeout
import fr.acinq.eclair.channel.states.StateTestsHelperMethods
import fr.acinq.eclair.channel.{WAIT_FOR_FUNDING_INTERNAL, _}
import fr.acinq.eclair.wire.{AcceptChannel, Error, Init, OpenChannel}
import fr.acinq.eclair.wire.{AcceptChannel, ChannelTlv, Error, Init, OpenChannel, TlvStream}
import fr.acinq.eclair.{CltvExpiryDelta, LongToBtcAmount, TestConstants, TestkitBaseClass}
import org.scalatest.{Outcome, Tag}
import scodec.bits.ByteVector
@ -65,7 +65,9 @@ class WaitForAcceptChannelStateSpec extends TestkitBaseClass with StateTestsHelp
test("recv AcceptChannel") { f =>
import f._
bob2alice.expectMsgType[AcceptChannel]
val accept = bob2alice.expectMsgType[AcceptChannel]
// Since https://github.com/lightningnetwork/lightning-rfc/pull/714 we must include an empty upfront_shutdown_script.
assert(accept.tlvStream === TlvStream(ChannelTlv.UpfrontShutdownScript(ByteVector.empty)))
bob2alice.forward(alice)
awaitCond(alice.stateName == WAIT_FOR_FUNDING_INTERNAL)
}

View file

@ -21,9 +21,10 @@ import fr.acinq.bitcoin.{Block, ByteVector32}
import fr.acinq.eclair.TestConstants.{Alice, Bob}
import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.states.StateTestsHelperMethods
import fr.acinq.eclair.wire.{Error, Init, OpenChannel}
import fr.acinq.eclair.wire.{ChannelTlv, Error, Init, OpenChannel, TlvStream}
import fr.acinq.eclair.{CltvExpiryDelta, LongToBtcAmount, TestConstants, TestkitBaseClass, ToMilliSatoshiConversion}
import org.scalatest.Outcome
import scodec.bits.ByteVector
import scala.concurrent.duration._
@ -52,7 +53,9 @@ class WaitForOpenChannelStateSpec extends TestkitBaseClass with StateTestsHelper
test("recv OpenChannel") { f =>
import f._
alice2bob.expectMsgType[OpenChannel]
val open = alice2bob.expectMsgType[OpenChannel]
// Since https://github.com/lightningnetwork/lightning-rfc/pull/714 we must include an empty upfront_shutdown_script.
assert(open.tlvStream === TlvStream(ChannelTlv.UpfrontShutdownScript(ByteVector.empty)))
alice2bob.forward(bob)
awaitCond(bob.stateName == WAIT_FOR_FUNDING_CREATED)
}

View file

@ -97,9 +97,9 @@ class OfflineStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
// a didn't receive any update or sig
val ab_reestablish = alice2bob.expectMsg(ChannelReestablish(ab_add_0.channelId, 1, 0, Some(PrivateKey(ByteVector32.Zeroes)), Some(aliceCurrentPerCommitmentPoint)))
val ab_reestablish = alice2bob.expectMsg(ChannelReestablish(ab_add_0.channelId, 1, 0, PrivateKey(ByteVector32.Zeroes), aliceCurrentPerCommitmentPoint))
// b didn't receive the sig
val ba_reestablish = bob2alice.expectMsg(ChannelReestablish(ab_add_0.channelId, 1, 0, Some(PrivateKey(ByteVector32.Zeroes)), Some(bobCurrentPerCommitmentPoint)))
val ba_reestablish = bob2alice.expectMsg(ChannelReestablish(ab_add_0.channelId, 1, 0, PrivateKey(ByteVector32.Zeroes), bobCurrentPerCommitmentPoint))
// reestablish ->b
alice2bob.forward(bob, ab_reestablish)
@ -184,9 +184,9 @@ class OfflineStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
aliceCommitments.localCommit.index)
// a didn't receive the sig
val ab_reestablish = alice2bob.expectMsg(ChannelReestablish(ab_add_0.channelId, 1, 0, Some(PrivateKey(ByteVector32.Zeroes)), Some(aliceCurrentPerCommitmentPoint)))
val ab_reestablish = alice2bob.expectMsg(ChannelReestablish(ab_add_0.channelId, 1, 0, PrivateKey(ByteVector32.Zeroes), aliceCurrentPerCommitmentPoint))
// b did receive the sig
val ba_reestablish = bob2alice.expectMsg(ChannelReestablish(ab_add_0.channelId, 2, 0, Some(PrivateKey(ByteVector32.Zeroes)), Some(bobCurrentPerCommitmentPoint)))
val ba_reestablish = bob2alice.expectMsg(ChannelReestablish(ab_add_0.channelId, 2, 0, PrivateKey(ByteVector32.Zeroes), bobCurrentPerCommitmentPoint))
// reestablish ->b
alice2bob.forward(bob, ab_reestablish)
@ -341,7 +341,7 @@ class OfflineStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
// alice then finds out bob is lying
bob2alice.send(alice, ba_reestablish_forged)
val error = alice2bob.expectMsgType[Error]
assert(new String(error.data.toArray) === InvalidRevokedCommitProof(channelId(alice), 0, 42, ba_reestablish_forged.yourLastPerCommitmentSecret.get).getMessage)
assert(new String(error.data.toArray) === InvalidRevokedCommitProof(channelId(alice), 0, 42, ba_reestablish_forged.yourLastPerCommitmentSecret).getMessage)
}
test("change relay fee while offline") { f =>

View file

@ -21,13 +21,14 @@ import java.util.UUID
import akka.actor.Status
import akka.actor.Status.Failure
import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.bitcoin.Crypto.PrivateKey
import fr.acinq.bitcoin.{ByteVector32, OutPoint, ScriptFlags, Transaction, TxIn}
import fr.acinq.eclair.TestConstants.{Alice, Bob}
import fr.acinq.eclair.blockchain._
import fr.acinq.eclair.blockchain.fee.FeeratesPerKw
import fr.acinq.eclair.channel.Helpers.Closing
import fr.acinq.eclair.channel.states.StateTestsHelperMethods
import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.states.StateTestsHelperMethods
import fr.acinq.eclair.payment._
import fr.acinq.eclair.payment.relay.Relayer._
import fr.acinq.eclair.payment.relay.{CommandBuffer, Origin}
@ -806,8 +807,14 @@ class ClosingStateSpec extends TestkitBaseClass with StateTestsHelperMethods {
import f._
mutualClose(alice, bob, alice2bob, bob2alice, alice2blockchain, bob2blockchain)
val initialState = alice.stateData.asInstanceOf[DATA_CLOSING]
val bobCommitments = bob.stateData.asInstanceOf[HasCommitments].commitments
val bobCurrentPerCommitmentPoint = TestConstants.Bob.keyManager.commitmentPoint(
TestConstants.Bob.keyManager.channelKeyPath(bobCommitments.localParams, bobCommitments.channelVersion),
bobCommitments.localCommit.index)
val sender = TestProbe()
sender.send(alice, ChannelReestablish(channelId(bob), 42, 42))
sender.send(alice, ChannelReestablish(channelId(bob), 42, 42, PrivateKey(ByteVector32.Zeroes), bobCurrentPerCommitmentPoint))
val error = alice2bob.expectMsgType[Error]
assert(new String(error.data.toArray) === FundingTxSpent(channelId(alice), initialState.spendingTxes.head).getMessage)
}

View file

@ -89,30 +89,31 @@ class LightningMessageCodecsSpec extends FunSuite {
test("encode/decode open_channel") {
val defaultOpen = OpenChannel(ByteVector32.Zeroes, ByteVector32.Zeroes, 1 sat, 1 msat, 1 sat, UInt64(1), 1 sat, 1 msat, 1, CltvExpiryDelta(1), 1, publicKey(1), point(2), point(3), point(4), point(5), point(6), 0.toByte)
// Default encoding that completely omits the upfront_shutdown_script and trailing tlv stream.
// To allow extending all messages with TLV streams, the upfront_shutdown_script was made mandatory in https://github.com/lightningnetwork/lightning-rfc/pull/714
// Legacy encoding that omits the upfront_shutdown_script and trailing tlv stream.
// To allow extending all messages with TLV streams, the upfront_shutdown_script was moved to a TLV stream extension
// in https://github.com/lightningnetwork/lightning-rfc/pull/714 and made mandatory when including a TLV stream.
// We don't make it mandatory at the codec level: it's the job of the actor creating the message to include it.
val defaultEncoded = hex"000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000001000000000000000100000000000000010000000000000001000000000000000100000000000000010000000100010001031b84c5567b126440995d3ed5aaba0565d71e1834604819ff9c17f5e9d5dd078f024d4b6cd1361032ca9bd2aeb9d900aa4d45d9ead80ac9423374c451a7254d076602531fe6068134503d2723133227c867ac8fa6c83c537e9a44c3c5bdbdcb1fe33703462779ad4aad39514614751a71085f2f10e1c7a593e4e030efb5b8721ce55b0b0362c0a046dacce86ddd0343c6d3c7c79c2208ba0d9c9cf24a6d046d21d21f90f703f006a18d5653c4edf5391ff23a61f03ff83d237e880ee61187fa9f379a028e0a00"
case class TestCase(encoded: ByteVector, decoded: OpenChannel, reEncoded: Option[ByteVector] = None)
val testCases = Seq(
val testCases = Map(
// legacy encoding without upfront_shutdown_script
TestCase(defaultEncoded, defaultOpen, Some(defaultEncoded ++ hex"0000")),
defaultEncoded -> defaultOpen,
// empty upfront_shutdown_script
TestCase(defaultEncoded ++ hex"0000", defaultOpen),
defaultEncoded ++ hex"0000" -> defaultOpen.copy(tlvStream = TlvStream(ChannelTlv.UpfrontShutdownScript(ByteVector.empty))),
// non-empty upfront_shutdown_script
TestCase(defaultEncoded ++ hex"0004 01abcdef", defaultOpen.copy(upfrontShutdownScript = Some(hex"01abcdef"))),
defaultEncoded ++ hex"0004 01abcdef" -> defaultOpen.copy(tlvStream = TlvStream(ChannelTlv.UpfrontShutdownScript(hex"01abcdef"))),
// missing upfront_shutdown_script + unknown odd tlv records
TestCase(defaultEncoded ++ hex"0302002a 050102", defaultOpen.copy(tlvStream_opt = Some(TlvStream(Nil, Seq(GenericTlv(UInt64(3), hex"002a"), GenericTlv(UInt64(5), hex"02")))))),
// empty upfront_shutdown_script + unknown odd tlv records: we don't encode the upfront_shutdown_script when a tlv stream is provided
TestCase(defaultEncoded ++ hex"0000 0302002a 050102", defaultOpen.copy(tlvStream_opt = Some(TlvStream(Nil, Seq(GenericTlv(UInt64(3), hex"002a"), GenericTlv(UInt64(5), hex"02"))))), Some(defaultEncoded ++ hex"0302002a 050102")),
// non-empty upfront_shutdown_script + unknown odd tlv records: we don't encode the upfront_shutdown_script when a tlv stream is provided
TestCase(defaultEncoded ++ hex"0002 1234 0303010203", defaultOpen.copy(upfrontShutdownScript = Some(hex"1234"), tlvStream_opt = Some(TlvStream(Nil, Seq(GenericTlv(UInt64(3), hex"010203"))))), Some(defaultEncoded ++ hex"0303010203"))
defaultEncoded ++ hex"0302002a 050102" -> defaultOpen.copy(tlvStream = TlvStream(Nil, Seq(GenericTlv(UInt64(3), hex"002a"), GenericTlv(UInt64(5), hex"02")))),
// empty upfront_shutdown_script + unknown odd tlv records
defaultEncoded ++ hex"0000 0302002a 050102" -> defaultOpen.copy(tlvStream = TlvStream(Seq(ChannelTlv.UpfrontShutdownScript(ByteVector.empty)), Seq(GenericTlv(UInt64(3), hex"002a"), GenericTlv(UInt64(5), hex"02")))),
// non-empty upfront_shutdown_script + unknown odd tlv records
defaultEncoded ++ hex"0002 1234 0303010203" -> defaultOpen.copy(tlvStream = TlvStream(Seq(ChannelTlv.UpfrontShutdownScript(hex"1234")), Seq(GenericTlv(UInt64(3), hex"010203"))))
)
for (testCase <- testCases) {
val decoded = openChannelCodec.decode(testCase.encoded.bits).require.value
assert(decoded === testCase.decoded)
for ((encoded, expected) <- testCases) {
val decoded = openChannelCodec.decode(encoded.bits).require.value
assert(decoded === expected)
val reEncoded = openChannelCodec.encode(decoded).require.bytes
assert(reEncoded === testCase.reEncoded.getOrElse(testCase.encoded))
assert(reEncoded === encoded)
}
}
@ -121,7 +122,7 @@ class LightningMessageCodecsSpec extends FunSuite {
val testCases = Seq(
defaultEncoded ++ hex"00", // truncated length
defaultEncoded ++ hex"01", // truncated length
defaultEncoded ++ hex"0004 123456", // truncated script
defaultEncoded ++ hex"0004 123456", // truncated upfront_shutdown_script
defaultEncoded ++ hex"0000 02012a", // invalid tlv stream (unknown even record)
defaultEncoded ++ hex"0000 01012a 030201", // invalid tlv stream (truncated)
defaultEncoded ++ hex"02012a", // invalid tlv stream (unknown even record)
@ -135,24 +136,25 @@ class LightningMessageCodecsSpec extends FunSuite {
test("encode/decode accept_channel") {
val defaultAccept = AcceptChannel(ByteVector32.Zeroes, 1 sat, UInt64(1), 1 sat, 1 msat, 1, CltvExpiryDelta(1), 1, publicKey(1), point(2), point(3), point(4), point(5), point(6))
// Default encoding that completely omits the upfront_shutdown_script (nodes were supposed to encode it only if both
// sides advertised support for option_upfront_shutdown_script).
// To allow extending all messages with TLV streams, the upfront_shutdown_script was made mandatory in https://github.com/lightningnetwork/lightning-rfc/pull/714
// Legacy encoding that omits the upfront_shutdown_script and trailing tlv stream.
// To allow extending all messages with TLV streams, the upfront_shutdown_script was moved to a TLV stream extension
// in https://github.com/lightningnetwork/lightning-rfc/pull/714 and made mandatory when including a TLV stream.
// We don't make it mandatory at the codec level: it's the job of the actor creating the message to include it.
val defaultEncoded = hex"000000000000000000000000000000000000000000000000000000000000000000000000000000010000000000000001000000000000000100000000000000010000000100010001031b84c5567b126440995d3ed5aaba0565d71e1834604819ff9c17f5e9d5dd078f024d4b6cd1361032ca9bd2aeb9d900aa4d45d9ead80ac9423374c451a7254d076602531fe6068134503d2723133227c867ac8fa6c83c537e9a44c3c5bdbdcb1fe33703462779ad4aad39514614751a71085f2f10e1c7a593e4e030efb5b8721ce55b0b0362c0a046dacce86ddd0343c6d3c7c79c2208ba0d9c9cf24a6d046d21d21f90f703f006a18d5653c4edf5391ff23a61f03ff83d237e880ee61187fa9f379a028e0a"
case class TestCase(encoded: ByteVector, decoded: AcceptChannel, reEncoded: Option[ByteVector] = None)
val testCases = Seq(
TestCase(defaultEncoded, defaultAccept, Some(defaultEncoded ++ hex"0000")), // legacy encoding without upfront_shutdown_script
TestCase(defaultEncoded ++ hex"0000", defaultAccept), // empty upfront_shutdown_script
TestCase(defaultEncoded ++ hex"0004 01abcdef", defaultAccept.copy(upfrontShutdownScript = Some(hex"01abcdef"))), // non-empty upfront_shutdown_script
TestCase(defaultEncoded ++ hex"0000 010202a 030102", defaultAccept, Some(defaultEncoded ++ hex"0000")), // empty upfront_shutdown_script + unknown odd tlv records
TestCase(defaultEncoded ++ hex"0002 1234 0303010203", defaultAccept.copy(upfrontShutdownScript = Some(hex"1234")), Some(defaultEncoded ++ hex"0002 1234")) // non-empty upfront_shutdown_script + unknown odd tlv records
val testCases = Map(
defaultEncoded -> defaultAccept, // legacy encoding without upfront_shutdown_script
defaultEncoded ++ hex"0000" -> defaultAccept.copy(tlvStream = TlvStream(ChannelTlv.UpfrontShutdownScript(ByteVector.empty))), // empty upfront_shutdown_script
defaultEncoded ++ hex"0004 01abcdef" -> defaultAccept.copy(tlvStream = TlvStream(ChannelTlv.UpfrontShutdownScript(hex"01abcdef"))), // non-empty upfront_shutdown_script
defaultEncoded ++ hex"0000 0102002a 030102" -> defaultAccept.copy(tlvStream = TlvStream(ChannelTlv.UpfrontShutdownScript(ByteVector.empty) :: Nil, GenericTlv(UInt64(1), hex"002a") :: GenericTlv(UInt64(3), hex"02") :: Nil)), // empty upfront_shutdown_script + unknown odd tlv records
defaultEncoded ++ hex"0002 1234 0303010203" -> defaultAccept.copy(tlvStream = TlvStream(ChannelTlv.UpfrontShutdownScript(hex"1234") :: Nil, GenericTlv(UInt64(3), hex"010203") :: Nil)), // non-empty upfront_shutdown_script + unknown odd tlv records
defaultEncoded ++ hex"0303010203 05020123" -> defaultAccept.copy(tlvStream = TlvStream(Nil, GenericTlv(UInt64(3), hex"010203") :: GenericTlv(UInt64(5), hex"0123") :: Nil)) // no upfront_shutdown_script + unknown odd tlv records
)
for (testCase <- testCases) {
val decoded = acceptChannelCodec.decode(testCase.encoded.bits).require.value
assert(decoded === testCase.decoded)
for ((encoded, expected) <- testCases) {
val decoded = acceptChannelCodec.decode(encoded.bits).require.value
assert(decoded === expected)
val reEncoded = acceptChannelCodec.encode(decoded).require.bytes
assert(reEncoded === testCase.reEncoded.getOrElse(testCase.encoded))
assert(reEncoded === encoded)
}
}
@ -190,7 +192,7 @@ class LightningMessageCodecsSpec extends FunSuite {
)
val ping = Ping(100, bin(10, 1))
val pong = Pong(bin(10, 1))
val channel_reestablish = ChannelReestablish(randomBytes32, 242842L, 42L)
val channel_reestablish = ChannelReestablish(randomBytes32, 242842L, 42L, randomKey, randomKey.publicKey)
val msgs: List[LightningMessage] =
open :: accept :: funding_created :: funding_signed :: funding_locked :: update_fee :: shutdown :: closing_signed ::