From 97666e89127b9635d8f0579498c1a2ae3b60185d Mon Sep 17 00:00:00 2001 From: Thomas HUET Date: Mon, 23 Aug 2021 14:34:42 +0200 Subject: [PATCH 1/4] Fix duplicate channel_updates in auditDb when restarting --- .../fr/acinq/eclair/channel/Channel.scala | 17 +++-- .../acinq/eclair/channel/ChannelEvents.scala | 2 +- .../fr/acinq/eclair/db/DbEventHandler.scala | 11 +--- .../acinq/eclair/router/Announcements.scala | 3 + .../acinq/eclair/channel/RecoverySpec.scala | 65 ++++++++++++++++++- .../fr/acinq/eclair/db/AuditDbSpec.scala | 2 +- .../integration/PaymentIntegrationSpec.scala | 2 +- .../payment/relay/ChannelRelayerSpec.scala | 14 ++-- .../eclair/payment/relay/RelayerSpec.scala | 2 +- .../acinq/eclair/router/BaseRouterSpec.scala | 2 +- .../fr/acinq/eclair/router/RouterSpec.scala | 8 +-- 11 files changed, 94 insertions(+), 34 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala index 7109e304dc..20eb8522bb 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala @@ -304,6 +304,10 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId log.info("refreshing channel_update due to configuration changes old={} new={}", normal.channelUpdate, candidateChannelUpdate) candidateChannelUpdate } + + val hasChanged = !Announcements.areSameWithoutFlags(candidateChannelUpdate, normal.channelUpdate) + context.system.eventStream.publish(LocalChannelUpdate(self, normal.commitments.channelId, normal.shortChannelId, normal.commitments.remoteParams.nodeId, normal.channelAnnouncement, normal.channelUpdate, hasChanged, normal.commitments)) + // we need to periodically re-send channel updates, otherwise channel will be considered stale and get pruned by network // we take into account the date of the last update so that we don't send superfluous updates when we restart the app val periodicRefreshInitialDelay = Helpers.nextChannelUpdateRefresh(channelUpdate1.timestamp) @@ -1879,25 +1883,24 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId case _ => () } - val previousChannelUpdate_opt = stateData match { - case data: DATA_NORMAL => Some(data.channelUpdate) - case _ => None - } - (state, nextState, stateData, nextStateData) match { // ORDER MATTERS! case (WAIT_FOR_INIT_INTERNAL, OFFLINE, _, normal: DATA_NORMAL) => + // LocalChannelUpdate is already published when restoring the channel Logs.withMdc(diagLog)(Logs.mdc(category_opt = Some(Logs.LogCategory.CONNECTION))) { log.debug("re-emitting channel_update={} enabled={} ", normal.channelUpdate, Announcements.isEnabled(normal.channelUpdate.channelFlags)) } - context.system.eventStream.publish(LocalChannelUpdate(self, normal.commitments.channelId, normal.shortChannelId, normal.commitments.remoteParams.nodeId, normal.channelAnnouncement, normal.channelUpdate, previousChannelUpdate_opt, normal.commitments)) case (_, _, d1: DATA_NORMAL, d2: DATA_NORMAL) if d1.channelUpdate == d2.channelUpdate && d1.channelAnnouncement == d2.channelAnnouncement => // don't do anything if neither the channel_update nor the channel_announcement didn't change () case (WAIT_FOR_FUNDING_LOCKED | NORMAL | OFFLINE | SYNCING, NORMAL | OFFLINE, _, normal: DATA_NORMAL) => // when we do WAIT_FOR_FUNDING_LOCKED->NORMAL or NORMAL->NORMAL or SYNCING->NORMAL or NORMAL->OFFLINE, we send out the new channel_update (most of the time it will just be to enable/disable the channel) log.info("emitting channel_update={} enabled={} ", normal.channelUpdate, Announcements.isEnabled(normal.channelUpdate.channelFlags)) - context.system.eventStream.publish(LocalChannelUpdate(self, normal.commitments.channelId, normal.shortChannelId, normal.commitments.remoteParams.nodeId, normal.channelAnnouncement, normal.channelUpdate, previousChannelUpdate_opt, normal.commitments)) + val hasChanged = stateData match { + case data: DATA_NORMAL => !Announcements.areSameWithoutFlags(data.channelUpdate, normal.channelUpdate) + case _ => true + } + context.system.eventStream.publish(LocalChannelUpdate(self, normal.commitments.channelId, normal.shortChannelId, normal.commitments.remoteParams.nodeId, normal.channelAnnouncement, normal.channelUpdate, hasChanged, normal.commitments)) case (_, _, _: DATA_NORMAL, _: DATA_NORMAL) => // in any other case (e.g. OFFLINE->SYNCING) we do nothing () diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelEvents.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelEvents.scala index ba9275e1ee..fbd2e1fdc6 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelEvents.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelEvents.scala @@ -38,7 +38,7 @@ case class ChannelIdAssigned(channel: ActorRef, remoteNodeId: PublicKey, tempora case class ShortChannelIdAssigned(channel: ActorRef, channelId: ByteVector32, shortChannelId: ShortChannelId, previousShortChannelId: Option[ShortChannelId]) extends ChannelEvent -case class LocalChannelUpdate(channel: ActorRef, channelId: ByteVector32, shortChannelId: ShortChannelId, remoteNodeId: PublicKey, channelAnnouncement_opt: Option[ChannelAnnouncement], channelUpdate: ChannelUpdate, previousChannelUpdate_opt: Option[ChannelUpdate], commitments: AbstractCommitments) extends ChannelEvent +case class LocalChannelUpdate(channel: ActorRef, channelId: ByteVector32, shortChannelId: ShortChannelId, remoteNodeId: PublicKey, channelAnnouncement_opt: Option[ChannelAnnouncement], channelUpdate: ChannelUpdate, hasChanged: Boolean, commitments: AbstractCommitments) extends ChannelEvent case class LocalChannelDown(channel: ActorRef, channelId: ByteVector32, shortChannelId: ShortChannelId, remoteNodeId: PublicKey) extends ChannelEvent diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/DbEventHandler.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/DbEventHandler.scala index 7b09d84050..2c5ecfe157 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/DbEventHandler.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/DbEventHandler.scala @@ -118,16 +118,9 @@ class DbEventHandler(nodeParams: NodeParams) extends Actor with ActorLogging { channelsDb.updateChannelMeta(e.channelId, event) case u: LocalChannelUpdate => - u.previousChannelUpdate_opt match { - case Some(previous) if - u.channelUpdate.feeBaseMsat == previous.feeBaseMsat && - u.channelUpdate.feeProportionalMillionths == previous.feeProportionalMillionths && - u.channelUpdate.cltvExpiryDelta == previous.cltvExpiryDelta && - u.channelUpdate.htlcMinimumMsat == previous.htlcMinimumMsat && - u.channelUpdate.htlcMaximumMsat == previous.htlcMaximumMsat => () - case _ => auditDb.addChannelUpdate(u) + if (u.hasChanged) { + auditDb.addChannelUpdate(u) } - } override def unhandled(message: Any): Unit = log.warning(s"unhandled msg=$message") diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/router/Announcements.scala b/eclair-core/src/main/scala/fr/acinq/eclair/router/Announcements.scala index d3ee33472c..7f5726e310 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/router/Announcements.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/router/Announcements.scala @@ -126,6 +126,9 @@ object Announcements { def areSame(u1: ChannelUpdate, u2: ChannelUpdate): Boolean = u1.copy(signature = ByteVector64.Zeroes, timestamp = 0) == u2.copy(signature = ByteVector64.Zeroes, timestamp = 0) + def areSameWithoutFlags(u1: ChannelUpdate, u2: ChannelUpdate): Boolean = + u1.copy(signature = ByteVector64.Zeroes, timestamp = 0, messageFlags = 1, channelFlags = 0) == u2.copy(signature = ByteVector64.Zeroes, timestamp = 0, messageFlags = 1, channelFlags = 0) + def makeMessageFlags(hasOptionChannelHtlcMax: Boolean): Byte = BitVector.bits(hasOptionChannelHtlcMax :: Nil).padLeft(8).toByte() def makeChannelFlags(isNode1: Boolean, enable: Boolean): Byte = BitVector.bits(!enable :: !isNode1 :: Nil).padLeft(8).toByte() diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/RecoverySpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/RecoverySpec.scala index abe96276a7..d2e4176af4 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/RecoverySpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/RecoverySpec.scala @@ -1,13 +1,16 @@ package fr.acinq.eclair.channel -import akka.testkit.TestProbe +import akka.actor.typed.scaladsl.adapter.actorRefAdapter +import akka.testkit.{TestFSMRef, TestProbe} import fr.acinq.bitcoin.Crypto.PublicKey import fr.acinq.bitcoin._ -import fr.acinq.eclair.TestConstants.Alice +import fr.acinq.eclair.TestConstants.{Alice, Bob} import fr.acinq.eclair.blockchain.bitcoind.ZmqWatcher.WatchFundingSpentTriggered import fr.acinq.eclair.channel.states.ChannelStateTestsBase +import fr.acinq.eclair.channel.states.ChannelStateTestsHelperMethods.FakeTxPublisherFactory import fr.acinq.eclair.crypto.Generators import fr.acinq.eclair.crypto.keymanager.ChannelKeyManager +import fr.acinq.eclair.payment.relay.Relayer.{RelayFees, RelayParams} import fr.acinq.eclair.transactions.Scripts import fr.acinq.eclair.transactions.Transactions.{ClaimP2WPKHOutputTx, DefaultCommitmentFormat, InputInfo, TxOwner} import fr.acinq.eclair.wire.protocol.{ChannelReestablish, CommitSig, Error, Init, RevokeAndAck} @@ -122,4 +125,62 @@ class RecoverySpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with Cha val tx1 = tx.updateWitness(0, ScriptWitness(Scripts.der(sig) :: ourToRemotePubKey.value :: Nil)) Transaction.correctlySpends(tx1, bobCommitTx :: Nil, ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS) } + + test("restore channel without configuration change") { f => + import f._ + val sender = TestProbe() + + // we start by storing the current state + assert(alice.stateData.isInstanceOf[HasCommitments]) + val oldStateData = alice.stateData.asInstanceOf[HasCommitments] + + // we simulate a disconnection + sender.send(alice, INPUT_DISCONNECTED) + sender.send(bob, INPUT_DISCONNECTED) + awaitCond(alice.stateName == OFFLINE) + awaitCond(bob.stateName == OFFLINE) + + // we restart Alice + val newAlice: TestFSMRef[ChannelState, ChannelData, Channel] = TestFSMRef(new Channel(TestConstants.Alice.nodeParams, wallet, Bob.nodeParams.nodeId, alice2blockchain.ref, relayerA.ref, FakeTxPublisherFactory(alice2blockchain)), alicePeer.ref) + newAlice ! INPUT_RESTORED(oldStateData) + newAlice ! INPUT_RECONNECTED(alice2bob.ref, aliceInit, bobInit) + bob ! INPUT_RECONNECTED(bob2alice.ref, bobInit, aliceInit) + alice2bob.expectMsgType[ChannelReestablish] + bob2alice.expectMsgType[ChannelReestablish] + alice2bob.forward(bob) + bob2alice.forward(newAlice) + awaitCond(newAlice.stateName == NORMAL) + val u = channelUpdateListener.expectMsgType[LocalChannelUpdate] + assert(!u.hasChanged) + } + + test("restore channel with configuration change") { f => + import f._ + val sender = TestProbe() + + // we start by storing the current state + assert(alice.stateData.isInstanceOf[HasCommitments]) + val oldStateData = alice.stateData.asInstanceOf[HasCommitments] + + // we simulate a disconnection + sender.send(alice, INPUT_DISCONNECTED) + sender.send(bob, INPUT_DISCONNECTED) + awaitCond(alice.stateName == OFFLINE) + awaitCond(bob.stateName == OFFLINE) + + // we restart Alice with a different configuration + val newFees = RelayFees(765 msat, 2345) + val newConfig = TestConstants.Alice.nodeParams.copy(relayParams = RelayParams(newFees, newFees, newFees)) + val newAlice: TestFSMRef[ChannelState, ChannelData, Channel] = TestFSMRef(new Channel(newConfig, wallet, Bob.nodeParams.nodeId, alice2blockchain.ref, relayerA.ref, FakeTxPublisherFactory(alice2blockchain)), alicePeer.ref) + newAlice ! INPUT_RESTORED(oldStateData) + newAlice ! INPUT_RECONNECTED(alice2bob.ref, aliceInit, bobInit) + bob ! INPUT_RECONNECTED(bob2alice.ref, bobInit, aliceInit) + alice2bob.expectMsgType[ChannelReestablish] + bob2alice.expectMsgType[ChannelReestablish] + alice2bob.forward(bob) + bob2alice.forward(newAlice) + awaitCond(newAlice.stateName == NORMAL) + val u = channelUpdateListener.expectMsgType[LocalChannelUpdate] + assert(u.hasChanged) + } } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/db/AuditDbSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/db/AuditDbSpec.scala index 10c4ca76a4..844229705a 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/db/AuditDbSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/db/AuditDbSpec.scala @@ -610,7 +610,7 @@ class AuditDbSpec extends AnyFunSuite { val scid = ShortChannelId(123) val remoteNodeId = randomKey().publicKey val u = Announcements.makeChannelUpdate(randomBytes32(), randomKey(), remoteNodeId, scid, CltvExpiryDelta(56), 2000 msat, 1000 msat, 999, 1000000000 msat) - dbs.audit.addChannelUpdate(LocalChannelUpdate(null, channelId, scid, remoteNodeId, None, u, None, null)) + dbs.audit.addChannelUpdate(LocalChannelUpdate(null, channelId, scid, remoteNodeId, None, u, true, null)) } } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala index f85683c68b..42cdc86d99 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala @@ -179,7 +179,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { // we then forge a new channel_update for B-C... val channelUpdateBC = Announcements.makeChannelUpdate(Block.RegtestGenesisBlock.hash, nodes("B").nodeParams.privateKey, nodes("C").nodeParams.nodeId, shortIdBC, nodes("B").nodeParams.expiryDelta + 1, nodes("C").nodeParams.htlcMinimum, nodes("B").nodeParams.relayParams.publicChannelFees.feeBase, nodes("B").nodeParams.relayParams.publicChannelFees.feeProportionalMillionths, 500000000 msat) // ...and notify B's relayer - nodes("B").system.eventStream.publish(LocalChannelUpdate(system.deadLetters, commitmentBC.channelId, shortIdBC, commitmentBC.remoteParams.nodeId, None, channelUpdateBC, None, commitmentBC)) + nodes("B").system.eventStream.publish(LocalChannelUpdate(system.deadLetters, commitmentBC.channelId, shortIdBC, commitmentBC.remoteParams.nodeId, None, channelUpdateBC, true, commitmentBC)) // we retrieve a payment hash from D val amountMsat = 4200000.msat sender.send(nodes("D").paymentHandler, ReceivePayment(Some(amountMsat), "1 coffee")) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/ChannelRelayerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/ChannelRelayerSpec.scala index a478791f78..efbfa5a29a 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/ChannelRelayerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/ChannelRelayerSpec.scala @@ -273,7 +273,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a val channelId = randomBytes32() val update = Announcements.makeChannelUpdate(Block.RegtestGenesisBlock.hash, randomKey(), remoteNodeId, shortChannelId, CltvExpiryDelta(10), 100 msat, 1000 msat, 100, capacity.toMilliSatoshi) val commitments = PaymentPacketSpec.makeCommitments(ByteVector32.Zeroes, availableBalanceForSend, testCapacity = capacity) - LocalChannelUpdate(null, channelId, shortChannelId, remoteNodeId, None, update, None, commitments) + LocalChannelUpdate(null, channelId, shortChannelId, remoteNodeId, None, update, true, commitments) } val (a, b) = (randomKey().publicKey, randomKey().publicKey) @@ -426,8 +426,8 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a channels } - channelRelayer ! WrappedLocalChannelUpdate(LocalChannelUpdate(null, channelId_ab, channelUpdate_ab.shortChannelId, a, None, channelUpdate_ab, None, makeCommitments(channelId_ab, -2000 msat, 300000 msat))) - channelRelayer ! WrappedLocalChannelUpdate(LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, None, makeCommitments(channelId_bc, 400000 msat, -5000 msat))) + channelRelayer ! WrappedLocalChannelUpdate(LocalChannelUpdate(null, channelId_ab, channelUpdate_ab.shortChannelId, a, None, channelUpdate_ab, true, makeCommitments(channelId_ab, -2000 msat, 300000 msat))) + channelRelayer ! WrappedLocalChannelUpdate(LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, true, makeCommitments(channelId_bc, 400000 msat, -5000 msat))) val channels1 = getOutgoingChannels(true) assert(channels1.size === 2) @@ -445,13 +445,13 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a val channels3 = getOutgoingChannels(true) assert(channels3.size === 1 && channels3.head.commitments.availableBalanceForSend === 100000.msat) - channelRelayer ! WrappedLocalChannelUpdate(LocalChannelUpdate(null, channelId_ab, channelUpdate_ab.shortChannelId, a, None, channelUpdate_ab.copy(channelFlags = 2), None, makeCommitments(channelId_ab, 100000 msat, 200000 msat))) + channelRelayer ! WrappedLocalChannelUpdate(LocalChannelUpdate(null, channelId_ab, channelUpdate_ab.shortChannelId, a, None, channelUpdate_ab.copy(channelFlags = 2), true, makeCommitments(channelId_ab, 100000 msat, 200000 msat))) val channels4 = getOutgoingChannels(true) assert(channels4.isEmpty) val channels5 = getOutgoingChannels(false) assert(channels5.size === 1) - channelRelayer ! WrappedLocalChannelUpdate(LocalChannelUpdate(null, channelId_ab, channelUpdate_ab.shortChannelId, a, None, channelUpdate_ab, None, makeCommitments(channelId_ab, 100000 msat, 200000 msat))) + channelRelayer ! WrappedLocalChannelUpdate(LocalChannelUpdate(null, channelId_ab, channelUpdate_ab.shortChannelId, a, None, channelUpdate_ab, true, makeCommitments(channelId_ab, 100000 msat, 200000 msat))) val channels6 = getOutgoingChannels(true) assert(channels6.size === 1) @@ -461,7 +461,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a assert(channels7.isEmpty) // We should receive the updated channel update containing the new shortChannelId: - channelRelayer ! WrappedLocalChannelUpdate(LocalChannelUpdate(null, channelId_ab, ShortChannelId(42), a, None, channelUpdate_ab.copy(shortChannelId = ShortChannelId(42)), None, makeCommitments(channelId_ab, 100000 msat, 200000 msat))) + channelRelayer ! WrappedLocalChannelUpdate(LocalChannelUpdate(null, channelId_ab, ShortChannelId(42), a, None, channelUpdate_ab.copy(shortChannelId = ShortChannelId(42)), true, makeCommitments(channelId_ab, 100000 msat, 200000 msat))) val channels8 = getOutgoingChannels(true) assert(channels8.size === 1) assert(channels8.head.channelUpdate.shortChannelId === ShortChannelId(42)) @@ -494,6 +494,6 @@ object ChannelRelayerSpec { val channelId = channelIds(shortChannelId) val update = ChannelUpdate(ByteVector64(randomBytes(64)), Block.RegtestGenesisBlock.hash, shortChannelId, 0, 1, Announcements.makeChannelFlags(isNode1 = true, enabled), CltvExpiryDelta(100), htlcMinimum, 1000 msat, 100, Some(capacity.toMilliSatoshi)) val commitments = PaymentPacketSpec.makeCommitments(channelId, testAvailableBalanceForSend = balance, testCapacity = capacity) - LocalChannelUpdate(null, channelId, shortChannelId, outgoingNodeId, None, update, None, commitments) + LocalChannelUpdate(null, channelId, shortChannelId, outgoingNodeId, None, update, true, commitments) } } \ No newline at end of file diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/RelayerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/RelayerSpec.scala index 7933bf2001..6f021a801a 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/RelayerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/RelayerSpec.scala @@ -76,7 +76,7 @@ class RelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat assert(sender.expectMessageType[Relayer.OutgoingChannels].channels.isEmpty) // We publish a channel update, that should be picked up by the channel relayer - system.eventStream ! EventStream.Publish(LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, None, makeCommitments(channelId_bc))) + system.eventStream ! EventStream.Publish(LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, true, makeCommitments(channelId_bc))) eventually(PatienceConfiguration.Timeout(30 seconds), PatienceConfiguration.Interval(1 second)) { childActors.channelRelayer ! ChannelRelayer.GetOutgoingChannels(sender.ref.toClassic, GetOutgoingChannels()) val channels = sender.expectMessageType[Relayer.OutgoingChannels].channels diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/router/BaseRouterSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/router/BaseRouterSpec.scala index 76069dc38d..bd7d5c31d4 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/router/BaseRouterSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/router/BaseRouterSpec.scala @@ -150,7 +150,7 @@ abstract class BaseRouterSpec extends TestKitBaseClass with FixtureAnyFunSuiteLi peerConnection.send(router, PeerRoutingMessage(peerConnection.ref, remoteNodeId, update_gh)) peerConnection.send(router, PeerRoutingMessage(peerConnection.ref, remoteNodeId, update_hg)) // then private channels - sender.send(router, LocalChannelUpdate(sender.ref, randomBytes32(), channelId_ag, g, None, update_ag, None, CommitmentsSpec.makeCommitments(30000000 msat, 8000000 msat, a, g, announceChannel = false))) + sender.send(router, LocalChannelUpdate(sender.ref, randomBytes32(), channelId_ag, g, None, update_ag, true, CommitmentsSpec.makeCommitments(30000000 msat, 8000000 msat, a, g, announceChannel = false))) // watcher receives the get tx requests assert(watcher.expectMsgType[ValidateRequest].ann === chan_ab) assert(watcher.expectMsgType[ValidateRequest].ann === chan_bc) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/router/RouterSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/router/RouterSpec.scala index d4b877c924..3dc2a4ab9a 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/router/RouterSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/router/RouterSpec.scala @@ -388,7 +388,7 @@ class RouterSpec extends BaseRouterSpec { assert(res.routes.head.hops.last.nextNodeId === h) val channelUpdate_ag1 = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_a, g, channelId_ag, CltvExpiryDelta(7), 0 msat, 10 msat, 10, htlcMaximum, enable = false) - sender.send(router, LocalChannelUpdate(sender.ref, null, channelId_ag, g, None, channelUpdate_ag1, None, CommitmentsSpec.makeCommitments(10000 msat, 15000 msat, a, g, announceChannel = false))) + sender.send(router, LocalChannelUpdate(sender.ref, null, channelId_ag, g, None, channelUpdate_ag1, true, CommitmentsSpec.makeCommitments(10000 msat, 15000 msat, a, g, announceChannel = false))) sender.send(router, RouteRequest(a, h, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE)) sender.expectMsg(Failure(RouteNotFound)) } @@ -409,7 +409,7 @@ class RouterSpec extends BaseRouterSpec { sender.send(router, RouteRequest(a, b, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE)) sender.expectMsgType[RouteResponse] val commitments1 = CommitmentsSpec.makeCommitments(10000000 msat, 20000000 msat, a, b, announceChannel = true) - sender.send(router, LocalChannelUpdate(sender.ref, null, channelId_ab, b, Some(chan_ab), update_ab, None, commitments1)) + sender.send(router, LocalChannelUpdate(sender.ref, null, channelId_ab, b, Some(chan_ab), update_ab, true, commitments1)) sender.send(router, RouteRequest(a, b, 12000000 msat, Long.MaxValue.msat)) sender.expectMsg(Failure(BalanceTooLow)) sender.send(router, RouteRequest(a, b, 12000000 msat, Long.MaxValue.msat, allowMultiPart = true)) @@ -588,7 +588,7 @@ class RouterSpec extends BaseRouterSpec { // When the local channel comes back online, it will send a LocalChannelUpdate to the router. val balances = Set[Option[MilliSatoshi]](Some(10000 msat), Some(15000 msat)) val commitments = CommitmentsSpec.makeCommitments(10000 msat, 15000 msat, a, b, announceChannel = true) - sender.send(router, LocalChannelUpdate(sender.ref, null, channelId_ab, b, Some(chan_ab), update_ab, None, commitments)) + sender.send(router, LocalChannelUpdate(sender.ref, null, channelId_ab, b, Some(chan_ab), update_ab, true, commitments)) sender.send(router, GetRoutingState) val channel_ab = sender.expectMsgType[RoutingState].channels.find(_.ann == chan_ab).get assert(Set(channel_ab.meta_opt.map(_.balance1), channel_ab.meta_opt.map(_.balance2)) === balances) @@ -611,7 +611,7 @@ class RouterSpec extends BaseRouterSpec { // Then we update the balance without changing the contents of the channel update; the graph should still be updated. val balances = Set[Option[MilliSatoshi]](Some(11000 msat), Some(14000 msat)) val commitments = CommitmentsSpec.makeCommitments(11000 msat, 14000 msat, a, b, announceChannel = true) - sender.send(router, LocalChannelUpdate(sender.ref, null, channelId_ab, b, Some(chan_ab), update_ab, None, commitments)) + sender.send(router, LocalChannelUpdate(sender.ref, null, channelId_ab, b, Some(chan_ab), update_ab, true, commitments)) sender.send(router, GetRoutingState) val channel_ab = sender.expectMsgType[RoutingState].channels.find(_.ann == chan_ab).get assert(Set(channel_ab.meta_opt.map(_.balance1), channel_ab.meta_opt.map(_.balance2)) === balances) From 1ead78f829d7a3d32f2401394f64f285564dc059 Mon Sep 17 00:00:00 2001 From: Thomas HUET Date: Mon, 23 Aug 2021 15:37:46 +0200 Subject: [PATCH 2/4] Fix bad copy paste --- .../src/main/scala/fr/acinq/eclair/channel/Channel.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala index 20eb8522bb..ce7f48a540 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala @@ -306,7 +306,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId } val hasChanged = !Announcements.areSameWithoutFlags(candidateChannelUpdate, normal.channelUpdate) - context.system.eventStream.publish(LocalChannelUpdate(self, normal.commitments.channelId, normal.shortChannelId, normal.commitments.remoteParams.nodeId, normal.channelAnnouncement, normal.channelUpdate, hasChanged, normal.commitments)) + context.system.eventStream.publish(LocalChannelUpdate(self, normal.commitments.channelId, normal.shortChannelId, normal.commitments.remoteParams.nodeId, normal.channelAnnouncement, channelUpdate1, hasChanged, normal.commitments)) // we need to periodically re-send channel updates, otherwise channel will be considered stale and get pruned by network // we take into account the date of the last update so that we don't send superfluous updates when we restart the app From 8bded11704d5f66f6570045289b9a315855693ae Mon Sep 17 00:00:00 2001 From: Thomas HUET Date: Mon, 23 Aug 2021 16:14:09 +0200 Subject: [PATCH 3/4] Use previousChannelUpdate_opt --- .../scala/fr/acinq/eclair/channel/Channel.scala | 11 +++++------ .../fr/acinq/eclair/channel/ChannelEvents.scala | 2 +- .../fr/acinq/eclair/db/DbEventHandler.scala | 6 ++++-- .../fr/acinq/eclair/channel/RecoverySpec.scala | 16 ++++++++++------ .../scala/fr/acinq/eclair/db/AuditDbSpec.scala | 2 +- .../integration/PaymentIntegrationSpec.scala | 2 +- .../payment/relay/ChannelRelayerSpec.scala | 14 +++++++------- .../acinq/eclair/payment/relay/RelayerSpec.scala | 2 +- .../fr/acinq/eclair/router/BaseRouterSpec.scala | 2 +- .../fr/acinq/eclair/router/RouterSpec.scala | 8 ++++---- 10 files changed, 35 insertions(+), 30 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala index ce7f48a540..2dfd3e28f9 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala @@ -305,8 +305,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId candidateChannelUpdate } - val hasChanged = !Announcements.areSameWithoutFlags(candidateChannelUpdate, normal.channelUpdate) - context.system.eventStream.publish(LocalChannelUpdate(self, normal.commitments.channelId, normal.shortChannelId, normal.commitments.remoteParams.nodeId, normal.channelAnnouncement, channelUpdate1, hasChanged, normal.commitments)) + context.system.eventStream.publish(LocalChannelUpdate(self, normal.commitments.channelId, normal.shortChannelId, normal.commitments.remoteParams.nodeId, normal.channelAnnouncement, channelUpdate1, Some(normal.channelUpdate), normal.commitments)) // we need to periodically re-send channel updates, otherwise channel will be considered stale and get pruned by network // we take into account the date of the last update so that we don't send superfluous updates when we restart the app @@ -1896,11 +1895,11 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId case (WAIT_FOR_FUNDING_LOCKED | NORMAL | OFFLINE | SYNCING, NORMAL | OFFLINE, _, normal: DATA_NORMAL) => // when we do WAIT_FOR_FUNDING_LOCKED->NORMAL or NORMAL->NORMAL or SYNCING->NORMAL or NORMAL->OFFLINE, we send out the new channel_update (most of the time it will just be to enable/disable the channel) log.info("emitting channel_update={} enabled={} ", normal.channelUpdate, Announcements.isEnabled(normal.channelUpdate.channelFlags)) - val hasChanged = stateData match { - case data: DATA_NORMAL => !Announcements.areSameWithoutFlags(data.channelUpdate, normal.channelUpdate) - case _ => true + val previousChannelUpdate_opt = stateData match { + case data: DATA_NORMAL => Some(data.channelUpdate) + case _ => None } - context.system.eventStream.publish(LocalChannelUpdate(self, normal.commitments.channelId, normal.shortChannelId, normal.commitments.remoteParams.nodeId, normal.channelAnnouncement, normal.channelUpdate, hasChanged, normal.commitments)) + context.system.eventStream.publish(LocalChannelUpdate(self, normal.commitments.channelId, normal.shortChannelId, normal.commitments.remoteParams.nodeId, normal.channelAnnouncement, normal.channelUpdate, previousChannelUpdate_opt, normal.commitments)) case (_, _, _: DATA_NORMAL, _: DATA_NORMAL) => // in any other case (e.g. OFFLINE->SYNCING) we do nothing () diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelEvents.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelEvents.scala index fbd2e1fdc6..ba9275e1ee 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelEvents.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelEvents.scala @@ -38,7 +38,7 @@ case class ChannelIdAssigned(channel: ActorRef, remoteNodeId: PublicKey, tempora case class ShortChannelIdAssigned(channel: ActorRef, channelId: ByteVector32, shortChannelId: ShortChannelId, previousShortChannelId: Option[ShortChannelId]) extends ChannelEvent -case class LocalChannelUpdate(channel: ActorRef, channelId: ByteVector32, shortChannelId: ShortChannelId, remoteNodeId: PublicKey, channelAnnouncement_opt: Option[ChannelAnnouncement], channelUpdate: ChannelUpdate, hasChanged: Boolean, commitments: AbstractCommitments) extends ChannelEvent +case class LocalChannelUpdate(channel: ActorRef, channelId: ByteVector32, shortChannelId: ShortChannelId, remoteNodeId: PublicKey, channelAnnouncement_opt: Option[ChannelAnnouncement], channelUpdate: ChannelUpdate, previousChannelUpdate_opt: Option[ChannelUpdate], commitments: AbstractCommitments) extends ChannelEvent case class LocalChannelDown(channel: ActorRef, channelId: ByteVector32, shortChannelId: ShortChannelId, remoteNodeId: PublicKey) extends ChannelEvent diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/DbEventHandler.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/DbEventHandler.scala index 2c5ecfe157..87f4a7168d 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/DbEventHandler.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/DbEventHandler.scala @@ -26,6 +26,7 @@ import fr.acinq.eclair.channel._ import fr.acinq.eclair.db.DbEventHandler.ChannelEvent import fr.acinq.eclair.payment.Monitoring.{Metrics => PaymentMetrics, Tags => PaymentTags} import fr.acinq.eclair.payment._ +import fr.acinq.eclair.router.Announcements /** * This actor sits at the interface between our event stream and the database. @@ -118,8 +119,9 @@ class DbEventHandler(nodeParams: NodeParams) extends Actor with ActorLogging { channelsDb.updateChannelMeta(e.channelId, event) case u: LocalChannelUpdate => - if (u.hasChanged) { - auditDb.addChannelUpdate(u) + u.previousChannelUpdate_opt match { + case Some(previous) if Announcements.areSameWithoutFlags(previous, u.channelUpdate) => () // channel update hasn't changed + case _ => auditDb.addChannelUpdate(u) } } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/RecoverySpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/RecoverySpec.scala index d2e4176af4..42b35b6e80 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/RecoverySpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/RecoverySpec.scala @@ -11,6 +11,7 @@ import fr.acinq.eclair.channel.states.ChannelStateTestsHelperMethods.FakeTxPubli import fr.acinq.eclair.crypto.Generators import fr.acinq.eclair.crypto.keymanager.ChannelKeyManager import fr.acinq.eclair.payment.relay.Relayer.{RelayFees, RelayParams} +import fr.acinq.eclair.router.Announcements import fr.acinq.eclair.transactions.Scripts import fr.acinq.eclair.transactions.Transactions.{ClaimP2WPKHOutputTx, DefaultCommitmentFormat, InputInfo, TxOwner} import fr.acinq.eclair.wire.protocol.{ChannelReestablish, CommitSig, Error, Init, RevokeAndAck} @@ -131,8 +132,8 @@ class RecoverySpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with Cha val sender = TestProbe() // we start by storing the current state - assert(alice.stateData.isInstanceOf[HasCommitments]) - val oldStateData = alice.stateData.asInstanceOf[HasCommitments] + assert(alice.stateData.isInstanceOf[DATA_NORMAL]) + val oldStateData = alice.stateData.asInstanceOf[DATA_NORMAL] // we simulate a disconnection sender.send(alice, INPUT_DISCONNECTED) @@ -151,7 +152,8 @@ class RecoverySpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with Cha bob2alice.forward(newAlice) awaitCond(newAlice.stateName == NORMAL) val u = channelUpdateListener.expectMsgType[LocalChannelUpdate] - assert(!u.hasChanged) + assert(u.previousChannelUpdate_opt.nonEmpty) + assert(Announcements.areSameWithoutFlags(u.previousChannelUpdate_opt.get, u.channelUpdate)) } test("restore channel with configuration change") { f => @@ -159,8 +161,8 @@ class RecoverySpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with Cha val sender = TestProbe() // we start by storing the current state - assert(alice.stateData.isInstanceOf[HasCommitments]) - val oldStateData = alice.stateData.asInstanceOf[HasCommitments] + assert(alice.stateData.isInstanceOf[DATA_NORMAL]) + val oldStateData = alice.stateData.asInstanceOf[DATA_NORMAL] // we simulate a disconnection sender.send(alice, INPUT_DISCONNECTED) @@ -181,6 +183,8 @@ class RecoverySpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with Cha bob2alice.forward(newAlice) awaitCond(newAlice.stateName == NORMAL) val u = channelUpdateListener.expectMsgType[LocalChannelUpdate] - assert(u.hasChanged) + assert(u.previousChannelUpdate_opt.nonEmpty) + assert(!Announcements.areSameWithoutFlags(u.previousChannelUpdate_opt.get, u.channelUpdate)) + assert(Announcements.areSameWithoutFlags(u.previousChannelUpdate_opt.get, oldStateData.channelUpdate)) } } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/db/AuditDbSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/db/AuditDbSpec.scala index 844229705a..10c4ca76a4 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/db/AuditDbSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/db/AuditDbSpec.scala @@ -610,7 +610,7 @@ class AuditDbSpec extends AnyFunSuite { val scid = ShortChannelId(123) val remoteNodeId = randomKey().publicKey val u = Announcements.makeChannelUpdate(randomBytes32(), randomKey(), remoteNodeId, scid, CltvExpiryDelta(56), 2000 msat, 1000 msat, 999, 1000000000 msat) - dbs.audit.addChannelUpdate(LocalChannelUpdate(null, channelId, scid, remoteNodeId, None, u, true, null)) + dbs.audit.addChannelUpdate(LocalChannelUpdate(null, channelId, scid, remoteNodeId, None, u, None, null)) } } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala index 42cdc86d99..f85683c68b 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/PaymentIntegrationSpec.scala @@ -179,7 +179,7 @@ class PaymentIntegrationSpec extends IntegrationSpec { // we then forge a new channel_update for B-C... val channelUpdateBC = Announcements.makeChannelUpdate(Block.RegtestGenesisBlock.hash, nodes("B").nodeParams.privateKey, nodes("C").nodeParams.nodeId, shortIdBC, nodes("B").nodeParams.expiryDelta + 1, nodes("C").nodeParams.htlcMinimum, nodes("B").nodeParams.relayParams.publicChannelFees.feeBase, nodes("B").nodeParams.relayParams.publicChannelFees.feeProportionalMillionths, 500000000 msat) // ...and notify B's relayer - nodes("B").system.eventStream.publish(LocalChannelUpdate(system.deadLetters, commitmentBC.channelId, shortIdBC, commitmentBC.remoteParams.nodeId, None, channelUpdateBC, true, commitmentBC)) + nodes("B").system.eventStream.publish(LocalChannelUpdate(system.deadLetters, commitmentBC.channelId, shortIdBC, commitmentBC.remoteParams.nodeId, None, channelUpdateBC, None, commitmentBC)) // we retrieve a payment hash from D val amountMsat = 4200000.msat sender.send(nodes("D").paymentHandler, ReceivePayment(Some(amountMsat), "1 coffee")) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/ChannelRelayerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/ChannelRelayerSpec.scala index efbfa5a29a..a478791f78 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/ChannelRelayerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/ChannelRelayerSpec.scala @@ -273,7 +273,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a val channelId = randomBytes32() val update = Announcements.makeChannelUpdate(Block.RegtestGenesisBlock.hash, randomKey(), remoteNodeId, shortChannelId, CltvExpiryDelta(10), 100 msat, 1000 msat, 100, capacity.toMilliSatoshi) val commitments = PaymentPacketSpec.makeCommitments(ByteVector32.Zeroes, availableBalanceForSend, testCapacity = capacity) - LocalChannelUpdate(null, channelId, shortChannelId, remoteNodeId, None, update, true, commitments) + LocalChannelUpdate(null, channelId, shortChannelId, remoteNodeId, None, update, None, commitments) } val (a, b) = (randomKey().publicKey, randomKey().publicKey) @@ -426,8 +426,8 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a channels } - channelRelayer ! WrappedLocalChannelUpdate(LocalChannelUpdate(null, channelId_ab, channelUpdate_ab.shortChannelId, a, None, channelUpdate_ab, true, makeCommitments(channelId_ab, -2000 msat, 300000 msat))) - channelRelayer ! WrappedLocalChannelUpdate(LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, true, makeCommitments(channelId_bc, 400000 msat, -5000 msat))) + channelRelayer ! WrappedLocalChannelUpdate(LocalChannelUpdate(null, channelId_ab, channelUpdate_ab.shortChannelId, a, None, channelUpdate_ab, None, makeCommitments(channelId_ab, -2000 msat, 300000 msat))) + channelRelayer ! WrappedLocalChannelUpdate(LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, None, makeCommitments(channelId_bc, 400000 msat, -5000 msat))) val channels1 = getOutgoingChannels(true) assert(channels1.size === 2) @@ -445,13 +445,13 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a val channels3 = getOutgoingChannels(true) assert(channels3.size === 1 && channels3.head.commitments.availableBalanceForSend === 100000.msat) - channelRelayer ! WrappedLocalChannelUpdate(LocalChannelUpdate(null, channelId_ab, channelUpdate_ab.shortChannelId, a, None, channelUpdate_ab.copy(channelFlags = 2), true, makeCommitments(channelId_ab, 100000 msat, 200000 msat))) + channelRelayer ! WrappedLocalChannelUpdate(LocalChannelUpdate(null, channelId_ab, channelUpdate_ab.shortChannelId, a, None, channelUpdate_ab.copy(channelFlags = 2), None, makeCommitments(channelId_ab, 100000 msat, 200000 msat))) val channels4 = getOutgoingChannels(true) assert(channels4.isEmpty) val channels5 = getOutgoingChannels(false) assert(channels5.size === 1) - channelRelayer ! WrappedLocalChannelUpdate(LocalChannelUpdate(null, channelId_ab, channelUpdate_ab.shortChannelId, a, None, channelUpdate_ab, true, makeCommitments(channelId_ab, 100000 msat, 200000 msat))) + channelRelayer ! WrappedLocalChannelUpdate(LocalChannelUpdate(null, channelId_ab, channelUpdate_ab.shortChannelId, a, None, channelUpdate_ab, None, makeCommitments(channelId_ab, 100000 msat, 200000 msat))) val channels6 = getOutgoingChannels(true) assert(channels6.size === 1) @@ -461,7 +461,7 @@ class ChannelRelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("a assert(channels7.isEmpty) // We should receive the updated channel update containing the new shortChannelId: - channelRelayer ! WrappedLocalChannelUpdate(LocalChannelUpdate(null, channelId_ab, ShortChannelId(42), a, None, channelUpdate_ab.copy(shortChannelId = ShortChannelId(42)), true, makeCommitments(channelId_ab, 100000 msat, 200000 msat))) + channelRelayer ! WrappedLocalChannelUpdate(LocalChannelUpdate(null, channelId_ab, ShortChannelId(42), a, None, channelUpdate_ab.copy(shortChannelId = ShortChannelId(42)), None, makeCommitments(channelId_ab, 100000 msat, 200000 msat))) val channels8 = getOutgoingChannels(true) assert(channels8.size === 1) assert(channels8.head.channelUpdate.shortChannelId === ShortChannelId(42)) @@ -494,6 +494,6 @@ object ChannelRelayerSpec { val channelId = channelIds(shortChannelId) val update = ChannelUpdate(ByteVector64(randomBytes(64)), Block.RegtestGenesisBlock.hash, shortChannelId, 0, 1, Announcements.makeChannelFlags(isNode1 = true, enabled), CltvExpiryDelta(100), htlcMinimum, 1000 msat, 100, Some(capacity.toMilliSatoshi)) val commitments = PaymentPacketSpec.makeCommitments(channelId, testAvailableBalanceForSend = balance, testCapacity = capacity) - LocalChannelUpdate(null, channelId, shortChannelId, outgoingNodeId, None, update, true, commitments) + LocalChannelUpdate(null, channelId, shortChannelId, outgoingNodeId, None, update, None, commitments) } } \ No newline at end of file diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/RelayerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/RelayerSpec.scala index 6f021a801a..7933bf2001 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/RelayerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/relay/RelayerSpec.scala @@ -76,7 +76,7 @@ class RelayerSpec extends ScalaTestWithActorTestKit(ConfigFactory.load("applicat assert(sender.expectMessageType[Relayer.OutgoingChannels].channels.isEmpty) // We publish a channel update, that should be picked up by the channel relayer - system.eventStream ! EventStream.Publish(LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, true, makeCommitments(channelId_bc))) + system.eventStream ! EventStream.Publish(LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, None, makeCommitments(channelId_bc))) eventually(PatienceConfiguration.Timeout(30 seconds), PatienceConfiguration.Interval(1 second)) { childActors.channelRelayer ! ChannelRelayer.GetOutgoingChannels(sender.ref.toClassic, GetOutgoingChannels()) val channels = sender.expectMessageType[Relayer.OutgoingChannels].channels diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/router/BaseRouterSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/router/BaseRouterSpec.scala index bd7d5c31d4..76069dc38d 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/router/BaseRouterSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/router/BaseRouterSpec.scala @@ -150,7 +150,7 @@ abstract class BaseRouterSpec extends TestKitBaseClass with FixtureAnyFunSuiteLi peerConnection.send(router, PeerRoutingMessage(peerConnection.ref, remoteNodeId, update_gh)) peerConnection.send(router, PeerRoutingMessage(peerConnection.ref, remoteNodeId, update_hg)) // then private channels - sender.send(router, LocalChannelUpdate(sender.ref, randomBytes32(), channelId_ag, g, None, update_ag, true, CommitmentsSpec.makeCommitments(30000000 msat, 8000000 msat, a, g, announceChannel = false))) + sender.send(router, LocalChannelUpdate(sender.ref, randomBytes32(), channelId_ag, g, None, update_ag, None, CommitmentsSpec.makeCommitments(30000000 msat, 8000000 msat, a, g, announceChannel = false))) // watcher receives the get tx requests assert(watcher.expectMsgType[ValidateRequest].ann === chan_ab) assert(watcher.expectMsgType[ValidateRequest].ann === chan_bc) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/router/RouterSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/router/RouterSpec.scala index 3dc2a4ab9a..d4b877c924 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/router/RouterSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/router/RouterSpec.scala @@ -388,7 +388,7 @@ class RouterSpec extends BaseRouterSpec { assert(res.routes.head.hops.last.nextNodeId === h) val channelUpdate_ag1 = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_a, g, channelId_ag, CltvExpiryDelta(7), 0 msat, 10 msat, 10, htlcMaximum, enable = false) - sender.send(router, LocalChannelUpdate(sender.ref, null, channelId_ag, g, None, channelUpdate_ag1, true, CommitmentsSpec.makeCommitments(10000 msat, 15000 msat, a, g, announceChannel = false))) + sender.send(router, LocalChannelUpdate(sender.ref, null, channelId_ag, g, None, channelUpdate_ag1, None, CommitmentsSpec.makeCommitments(10000 msat, 15000 msat, a, g, announceChannel = false))) sender.send(router, RouteRequest(a, h, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE)) sender.expectMsg(Failure(RouteNotFound)) } @@ -409,7 +409,7 @@ class RouterSpec extends BaseRouterSpec { sender.send(router, RouteRequest(a, b, DEFAULT_AMOUNT_MSAT, DEFAULT_MAX_FEE)) sender.expectMsgType[RouteResponse] val commitments1 = CommitmentsSpec.makeCommitments(10000000 msat, 20000000 msat, a, b, announceChannel = true) - sender.send(router, LocalChannelUpdate(sender.ref, null, channelId_ab, b, Some(chan_ab), update_ab, true, commitments1)) + sender.send(router, LocalChannelUpdate(sender.ref, null, channelId_ab, b, Some(chan_ab), update_ab, None, commitments1)) sender.send(router, RouteRequest(a, b, 12000000 msat, Long.MaxValue.msat)) sender.expectMsg(Failure(BalanceTooLow)) sender.send(router, RouteRequest(a, b, 12000000 msat, Long.MaxValue.msat, allowMultiPart = true)) @@ -588,7 +588,7 @@ class RouterSpec extends BaseRouterSpec { // When the local channel comes back online, it will send a LocalChannelUpdate to the router. val balances = Set[Option[MilliSatoshi]](Some(10000 msat), Some(15000 msat)) val commitments = CommitmentsSpec.makeCommitments(10000 msat, 15000 msat, a, b, announceChannel = true) - sender.send(router, LocalChannelUpdate(sender.ref, null, channelId_ab, b, Some(chan_ab), update_ab, true, commitments)) + sender.send(router, LocalChannelUpdate(sender.ref, null, channelId_ab, b, Some(chan_ab), update_ab, None, commitments)) sender.send(router, GetRoutingState) val channel_ab = sender.expectMsgType[RoutingState].channels.find(_.ann == chan_ab).get assert(Set(channel_ab.meta_opt.map(_.balance1), channel_ab.meta_opt.map(_.balance2)) === balances) @@ -611,7 +611,7 @@ class RouterSpec extends BaseRouterSpec { // Then we update the balance without changing the contents of the channel update; the graph should still be updated. val balances = Set[Option[MilliSatoshi]](Some(11000 msat), Some(14000 msat)) val commitments = CommitmentsSpec.makeCommitments(11000 msat, 14000 msat, a, b, announceChannel = true) - sender.send(router, LocalChannelUpdate(sender.ref, null, channelId_ab, b, Some(chan_ab), update_ab, true, commitments)) + sender.send(router, LocalChannelUpdate(sender.ref, null, channelId_ab, b, Some(chan_ab), update_ab, None, commitments)) sender.send(router, GetRoutingState) val channel_ab = sender.expectMsgType[RoutingState].channels.find(_.ann == chan_ab).get assert(Set(channel_ab.meta_opt.map(_.balance1), channel_ab.meta_opt.map(_.balance2)) === balances) From ef31de10467055e4b8a09a92578e8c2b50041804 Mon Sep 17 00:00:00 2001 From: Thomas HUET Date: Mon, 23 Aug 2021 16:45:21 +0200 Subject: [PATCH 4/4] Put previousChannelUpdate_opt in DATA_NORMAL --- .../src/main/scala/fr/acinq/eclair/channel/Channel.scala | 9 +++------ .../main/scala/fr/acinq/eclair/channel/ChannelData.scala | 1 + .../wire/internal/channel/version0/ChannelCodecs0.scala | 4 +++- .../wire/internal/channel/version1/ChannelCodecs1.scala | 3 ++- .../wire/internal/channel/version2/ChannelCodecs2.scala | 3 ++- .../wire/internal/channel/version3/ChannelCodecs3.scala | 3 ++- .../scala/fr/acinq/eclair/channel/RecoverySpec.scala | 1 + .../eclair/wire/internal/channel/ChannelCodecsSpec.scala | 2 +- 8 files changed, 15 insertions(+), 11 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala index 2dfd3e28f9..593206dad1 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala @@ -304,15 +304,12 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId log.info("refreshing channel_update due to configuration changes old={} new={}", normal.channelUpdate, candidateChannelUpdate) candidateChannelUpdate } - - context.system.eventStream.publish(LocalChannelUpdate(self, normal.commitments.channelId, normal.shortChannelId, normal.commitments.remoteParams.nodeId, normal.channelAnnouncement, channelUpdate1, Some(normal.channelUpdate), normal.commitments)) - // we need to periodically re-send channel updates, otherwise channel will be considered stale and get pruned by network // we take into account the date of the last update so that we don't send superfluous updates when we restart the app val periodicRefreshInitialDelay = Helpers.nextChannelUpdateRefresh(channelUpdate1.timestamp) context.system.scheduler.scheduleWithFixedDelay(initialDelay = periodicRefreshInitialDelay, delay = REFRESH_CHANNEL_UPDATE_INTERVAL, receiver = self, message = BroadcastChannelUpdate(PeriodicRefresh)) - goto(OFFLINE) using normal.copy(channelUpdate = channelUpdate1) + goto(OFFLINE) using normal.copy(channelUpdate = channelUpdate1, channelUpdateBeforeRestore_opt = Some(normal.channelUpdate)) case funding: DATA_WAIT_FOR_FUNDING_CONFIRMED => watchFundingTx(funding.commitments) @@ -669,7 +666,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId val initialChannelUpdate = Announcements.makeChannelUpdate(nodeParams.chainHash, nodeParams.privateKey, remoteNodeId, shortChannelId, nodeParams.expiryDelta, d.commitments.remoteParams.htlcMinimum, fees.feeBase, fees.feeProportionalMillionths, commitments.capacity.toMilliSatoshi, enable = Helpers.aboveReserve(d.commitments)) // we need to periodically re-send channel updates, otherwise channel will be considered stale and get pruned by network context.system.scheduler.scheduleWithFixedDelay(initialDelay = REFRESH_CHANNEL_UPDATE_INTERVAL, delay = REFRESH_CHANNEL_UPDATE_INTERVAL, receiver = self, message = BroadcastChannelUpdate(PeriodicRefresh)) - goto(NORMAL) using DATA_NORMAL(commitments.copy(remoteNextCommitInfo = Right(nextPerCommitmentPoint)), shortChannelId, buried = false, None, initialChannelUpdate, None, None) storing() + goto(NORMAL) using DATA_NORMAL(commitments.copy(remoteNextCommitInfo = Right(nextPerCommitmentPoint)), shortChannelId, buried = false, None, initialChannelUpdate, None, None, None) storing() case Event(remoteAnnSigs: AnnouncementSignatures, d: DATA_WAIT_FOR_FUNDING_LOCKED) if d.commitments.announceChannel => log.debug("received remote announcement signatures, delaying") @@ -1885,10 +1882,10 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId (state, nextState, stateData, nextStateData) match { // ORDER MATTERS! case (WAIT_FOR_INIT_INTERNAL, OFFLINE, _, normal: DATA_NORMAL) => - // LocalChannelUpdate is already published when restoring the channel Logs.withMdc(diagLog)(Logs.mdc(category_opt = Some(Logs.LogCategory.CONNECTION))) { log.debug("re-emitting channel_update={} enabled={} ", normal.channelUpdate, Announcements.isEnabled(normal.channelUpdate.channelFlags)) } + context.system.eventStream.publish(LocalChannelUpdate(self, normal.commitments.channelId, normal.shortChannelId, normal.commitments.remoteParams.nodeId, normal.channelAnnouncement, normal.channelUpdate, normal.channelUpdateBeforeRestore_opt, normal.commitments)) case (_, _, d1: DATA_NORMAL, d2: DATA_NORMAL) if d1.channelUpdate == d2.channelUpdate && d1.channelAnnouncement == d2.channelAnnouncement => // don't do anything if neither the channel_update nor the channel_announcement didn't change () diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala index 05c9716792..5236f74613 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelData.scala @@ -421,6 +421,7 @@ final case class DATA_NORMAL(commitments: Commitments, buried: Boolean, channelAnnouncement: Option[ChannelAnnouncement], channelUpdate: ChannelUpdate, + channelUpdateBeforeRestore_opt: Option[ChannelUpdate], localShutdown: Option[Shutdown], remoteShutdown: Option[Shutdown]) extends ChannelData with HasCommitments final case class DATA_SHUTDOWN(commitments: Commitments, diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version0/ChannelCodecs0.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version0/ChannelCodecs0.scala index e623df31ac..fd373837d8 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version0/ChannelCodecs0.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version0/ChannelCodecs0.scala @@ -26,7 +26,7 @@ import fr.acinq.eclair.transactions._ import fr.acinq.eclair.wire.internal.channel.version0.ChannelTypes0.{HtlcTxAndSigs, PublishableTxs} import fr.acinq.eclair.wire.protocol.CommonCodecs._ import fr.acinq.eclair.wire.protocol.LightningMessageCodecs._ -import fr.acinq.eclair.wire.protocol.UpdateMessage +import fr.acinq.eclair.wire.protocol.{ChannelUpdate, UpdateMessage} import scodec.bits.{BitVector, ByteVector} import scodec.codecs._ import scodec.{Attempt, Codec} @@ -309,6 +309,7 @@ private[channel] object ChannelCodecs0 { ("buried" | bool) :: ("channelAnnouncement" | optional(bool, variableSizeBytes(noUnknownFieldsChannelAnnouncementSizeCodec, channelAnnouncementCodec))) :: ("channelUpdate" | variableSizeBytes(noUnknownFieldsChannelUpdateSizeCodec, channelUpdateCodec)) :: + ("channelUpdateBeforeRestore_opt" | provide[Option[ChannelUpdate]](None)) :: ("localShutdown" | optional(bool, shutdownCodec)) :: ("remoteShutdown" | optional(bool, shutdownCodec))).as[DATA_NORMAL].decodeOnly @@ -318,6 +319,7 @@ private[channel] object ChannelCodecs0 { ("buried" | bool) :: ("channelAnnouncement" | optional(bool, variableSizeBytes(uint16, channelAnnouncementCodec))) :: ("channelUpdate" | variableSizeBytes(uint16, channelUpdateCodec)) :: + ("channelUpdateBeforeRestore_opt" | provide[Option[ChannelUpdate]](None)) :: ("localShutdown" | optional(bool, shutdownCodec)) :: ("remoteShutdown" | optional(bool, shutdownCodec))).as[DATA_NORMAL].decodeOnly diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version1/ChannelCodecs1.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version1/ChannelCodecs1.scala index d8e4293c92..2091b7dfb2 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version1/ChannelCodecs1.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version1/ChannelCodecs1.scala @@ -27,7 +27,7 @@ import fr.acinq.eclair.wire.internal.channel.version0.ChannelTypes0 import fr.acinq.eclair.wire.internal.channel.version0.ChannelTypes0.{HtlcTxAndSigs, PublishableTxs} import fr.acinq.eclair.wire.protocol.CommonCodecs._ import fr.acinq.eclair.wire.protocol.LightningMessageCodecs._ -import fr.acinq.eclair.wire.protocol.UpdateMessage +import fr.acinq.eclair.wire.protocol.{ChannelUpdate, UpdateMessage} import scodec.bits.ByteVector import scodec.codecs._ import scodec.{Attempt, Codec} @@ -247,6 +247,7 @@ private[channel] object ChannelCodecs1 { ("buried" | bool8) :: ("channelAnnouncement" | optional(bool8, lengthDelimited(channelAnnouncementCodec))) :: ("channelUpdate" | lengthDelimited(channelUpdateCodec)) :: + ("channelUpdateBeforeRestore_opt" | provide[Option[ChannelUpdate]](None)) :: ("localShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) :: ("remoteShutdown" | optional(bool8, lengthDelimited(shutdownCodec)))).as[DATA_NORMAL] diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version2/ChannelCodecs2.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version2/ChannelCodecs2.scala index 0fabb04cca..d63b4c44e8 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version2/ChannelCodecs2.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version2/ChannelCodecs2.scala @@ -27,7 +27,7 @@ import fr.acinq.eclair.wire.internal.channel.version0.ChannelTypes0 import fr.acinq.eclair.wire.internal.channel.version0.ChannelTypes0.{HtlcTxAndSigs, PublishableTxs} import fr.acinq.eclair.wire.protocol.CommonCodecs._ import fr.acinq.eclair.wire.protocol.LightningMessageCodecs._ -import fr.acinq.eclair.wire.protocol.UpdateMessage +import fr.acinq.eclair.wire.protocol.{ChannelUpdate, UpdateMessage} import scodec.bits.ByteVector import scodec.codecs._ import scodec.{Attempt, Codec} @@ -282,6 +282,7 @@ private[channel] object ChannelCodecs2 { ("buried" | bool8) :: ("channelAnnouncement" | optional(bool8, lengthDelimited(channelAnnouncementCodec))) :: ("channelUpdate" | lengthDelimited(channelUpdateCodec)) :: + ("channelUpdateBeforeRestore_opt" | provide[Option[ChannelUpdate]](None)) :: ("localShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) :: ("remoteShutdown" | optional(bool8, lengthDelimited(shutdownCodec)))).as[DATA_NORMAL] diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version3/ChannelCodecs3.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version3/ChannelCodecs3.scala index dc213c2cc2..5132ff13e2 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version3/ChannelCodecs3.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/internal/channel/version3/ChannelCodecs3.scala @@ -24,7 +24,7 @@ import fr.acinq.eclair.transactions.Transactions._ import fr.acinq.eclair.transactions.{CommitmentSpec, DirectedHtlc, IncomingHtlc, OutgoingHtlc} import fr.acinq.eclair.wire.protocol.CommonCodecs._ import fr.acinq.eclair.wire.protocol.LightningMessageCodecs._ -import fr.acinq.eclair.wire.protocol.UpdateMessage +import fr.acinq.eclair.wire.protocol.{ChannelUpdate, UpdateMessage} import fr.acinq.eclair.{FeatureSupport, Features, MilliSatoshi} import scodec.bits.{BitVector, ByteVector} import scodec.codecs._ @@ -302,6 +302,7 @@ private[channel] object ChannelCodecs3 { ("buried" | bool8) :: ("channelAnnouncement" | optional(bool8, lengthDelimited(channelAnnouncementCodec))) :: ("channelUpdate" | lengthDelimited(channelUpdateCodec)) :: + ("channelUpdateBeforeRestore_opt" | provide[Option[ChannelUpdate]](None)) :: ("localShutdown" | optional(bool8, lengthDelimited(shutdownCodec))) :: ("remoteShutdown" | optional(bool8, lengthDelimited(shutdownCodec)))).as[DATA_NORMAL] diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/RecoverySpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/RecoverySpec.scala index 42b35b6e80..2ed421d1e5 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/RecoverySpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/RecoverySpec.scala @@ -154,6 +154,7 @@ class RecoverySpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with Cha val u = channelUpdateListener.expectMsgType[LocalChannelUpdate] assert(u.previousChannelUpdate_opt.nonEmpty) assert(Announcements.areSameWithoutFlags(u.previousChannelUpdate_opt.get, u.channelUpdate)) + assert(Announcements.areSameWithoutFlags(u.previousChannelUpdate_opt.get, oldStateData.channelUpdate)) } test("restore channel with configuration change") { f => diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/wire/internal/channel/ChannelCodecsSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/wire/internal/channel/ChannelCodecsSpec.scala index eb76615543..b9ca920cb7 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/wire/internal/channel/ChannelCodecsSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/wire/internal/channel/ChannelCodecsSpec.scala @@ -325,7 +325,7 @@ object ChannelCodecsSpec { commitInput = commitmentInput, remotePerCommitmentSecrets = ShaChain.init) - DATA_NORMAL(commitments, ShortChannelId(42), buried = true, None, channelUpdate, None, None) + DATA_NORMAL(commitments, ShortChannelId(42), buried = true, None, channelUpdate, None, None, None) } }