1
0
mirror of https://github.com/ACINQ/eclair.git synced 2024-11-20 10:39:19 +01:00

added AnnouncementSignatures message, added support for feature bits

This commit is contained in:
pm47 2017-02-13 14:45:26 +01:00
parent e8a74899f3
commit 6bd14bb089
19 changed files with 355 additions and 78 deletions

View File

@ -0,0 +1,12 @@
package fr.acinq.eclair
import fr.acinq.bitcoin.BinaryData
/**
* Created by PM on 13/02/2017.
*/
object Features {
def isChannelPublic(localFeatures: BinaryData): Boolean = localFeatures.size >= 1 && localFeatures.data(0) == 0x01
}

View File

@ -30,6 +30,9 @@ object Globals {
val address = new InetSocketAddress(config.getString("server.host"), config.getInt("server.port"))
}
val global_features = BinaryData("")
val local_features = BinaryData("01") // public channel
val expiry_delta_blocks = config.getInt("expiry-delta-blocks")
val htlc_minimum_msat = config.getInt("htlc-minimum-msat")
val delay_blocks = config.getInt("delay-blocks")

View File

@ -69,12 +69,22 @@ class Channel(val them: ActorRef, val blockchain: ActorRef, router: ActorRef, re
startWith(WAIT_FOR_INIT_INTERNAL, Nothing)
when(WAIT_FOR_INIT_INTERNAL)(handleExceptions {
case Event(INPUT_INIT_FUNDER(fundingSatoshis, pushMsat), Nothing) if localParams.isFunder =>
case Event(i@INPUT_INIT_FUNDER(fundingSatoshis, pushMsat), Nothing) if localParams.isFunder =>
them ! Init(globalFeatures = localParams.globalFeatures, localFeatures = localParams.localFeatures)
goto(WAIT_FOR_INIT) using DATA_WAIT_FOR_INIT(localParams, Left(i), autoSignInterval)
case Event(i@INPUT_INIT_FUNDEE(), Nothing) if !localParams.isFunder =>
them ! Init(globalFeatures = localParams.globalFeatures, localFeatures = localParams.localFeatures)
goto(WAIT_FOR_INIT) using DATA_WAIT_FOR_INIT(localParams, Right(i), autoSignInterval)
})
when(WAIT_FOR_INIT)(handleExceptions {
case Event(Init(remoteGlobalFeatures, remoteLocalFeatures), DATA_WAIT_FOR_INIT(localParams, Left(initFunder), autoSignInterval)) =>
val temporaryChannelId = Platform.currentTime
val firstPerCommitmentPoint = Generators.perCommitPoint(localParams.shaSeed, 0)
them ! OpenChannel(temporaryChannelId = temporaryChannelId,
fundingSatoshis = fundingSatoshis,
pushMsat = pushMsat,
fundingSatoshis = initFunder.fundingSatoshis,
pushMsat = initFunder.pushMsat,
dustLimitSatoshis = localParams.dustLimitSatoshis,
maxHtlcValueInFlightMsat = localParams.maxHtlcValueInFlightMsat,
channelReserveSatoshis = localParams.channelReserveSatoshis,
@ -87,15 +97,14 @@ class Channel(val them: ActorRef, val blockchain: ActorRef, router: ActorRef, re
paymentBasepoint = localParams.paymentKey.toPoint,
delayedPaymentBasepoint = localParams.delayedPaymentKey.toPoint,
firstPerCommitmentPoint = firstPerCommitmentPoint)
goto(WAIT_FOR_ACCEPT_CHANNEL) using DATA_WAIT_FOR_ACCEPT_CHANNEL(temporaryChannelId, localParams, fundingSatoshis = fundingSatoshis, pushMsat = pushMsat, autoSignInterval = autoSignInterval)
goto(WAIT_FOR_ACCEPT_CHANNEL) using DATA_WAIT_FOR_ACCEPT_CHANNEL(temporaryChannelId, localParams, fundingSatoshis = initFunder.fundingSatoshis, pushMsat = initFunder.pushMsat, remoteGlobalFeatures = remoteGlobalFeatures, remoteLocalFeatures = remoteLocalFeatures, autoSignInterval = autoSignInterval)
case Event(INPUT_INIT_FUNDEE(), Nothing) if !localParams.isFunder =>
goto(WAIT_FOR_OPEN_CHANNEL) using DATA_WAIT_FOR_OPEN_CHANNEL(localParams, autoSignInterval = autoSignInterval)
case Event(Init(remoteGlobalFeatures, remoteLocalFeatures), DATA_WAIT_FOR_INIT(localParams, Right(initFundee), autoSignInterval)) =>
goto(WAIT_FOR_OPEN_CHANNEL) using DATA_WAIT_FOR_OPEN_CHANNEL(localParams, remoteGlobalFeatures = remoteGlobalFeatures, remoteLocalFeatures = remoteLocalFeatures, autoSignInterval = autoSignInterval)
})
when(WAIT_FOR_OPEN_CHANNEL)(handleExceptions {
case Event(open: OpenChannel, DATA_WAIT_FOR_OPEN_CHANNEL(localParams, autoSignInterval)) =>
case Event(open: OpenChannel, DATA_WAIT_FOR_OPEN_CHANNEL(localParams, remoteGlobalFeatures, remoteLocalFeatures, autoSignInterval)) =>
Try(Funding.validateParams(open.channelReserveSatoshis, open.fundingSatoshis)) match {
case Failure(t) =>
log.warning(t.getMessage)
@ -129,7 +138,9 @@ class Channel(val them: ActorRef, val blockchain: ActorRef, router: ActorRef, re
fundingPubKey = open.fundingPubkey,
revocationBasepoint = open.revocationBasepoint,
paymentBasepoint = open.paymentBasepoint,
delayedPaymentBasepoint = open.delayedPaymentBasepoint)
delayedPaymentBasepoint = open.delayedPaymentBasepoint,
globalFeatures = remoteGlobalFeatures,
localFeatures = remoteLocalFeatures)
log.debug(s"remote params: $remoteParams")
val params = ChannelParams(
localParams = localParams.copy(feeratePerKw = open.feeratePerKw), // funder gets to choose the first feerate
@ -148,7 +159,7 @@ class Channel(val them: ActorRef, val blockchain: ActorRef, router: ActorRef, re
})
when(WAIT_FOR_ACCEPT_CHANNEL)(handleExceptions {
case Event(accept: AcceptChannel, DATA_WAIT_FOR_ACCEPT_CHANNEL(temporaryChannelId, localParams, fundingSatoshis, pushMsat, autoSignInterval)) =>
case Event(accept: AcceptChannel, DATA_WAIT_FOR_ACCEPT_CHANNEL(temporaryChannelId, localParams, fundingSatoshis, pushMsat, remoteGlobalFeatures, remoteLocalFeatures, autoSignInterval)) =>
Try(Funding.validateParams(accept.channelReserveSatoshis, fundingSatoshis)) match {
case Failure(t) =>
log.warning(t.getMessage)
@ -167,7 +178,9 @@ class Channel(val them: ActorRef, val blockchain: ActorRef, router: ActorRef, re
fundingPubKey = accept.fundingPubkey,
revocationBasepoint = accept.revocationBasepoint,
paymentBasepoint = accept.paymentBasepoint,
delayedPaymentBasepoint = accept.delayedPaymentBasepoint
delayedPaymentBasepoint = accept.delayedPaymentBasepoint,
globalFeatures = remoteGlobalFeatures,
localFeatures = remoteLocalFeatures
)
log.debug(s"remote params: $remoteParams")
val params = ChannelParams(
@ -300,8 +313,7 @@ class Channel(val them: ActorRef, val blockchain: ActorRef, router: ActorRef, re
val channelId = toShortId(blockHeight, txIndex, commitments.commitInput.outPoint.index.toInt)
blockchain ! WatchLost(self, commitments.anchorId, params.minimumDepth, BITCOIN_FUNDING_LOST)
val nextPerCommitmentPoint = Generators.perCommitPoint(localParams.shaSeed, 1)
val (localNodeSig, localBitcoinSig) = Announcements.signChannelAnnouncement(channelId, Globals.Node.privateKey, remoteNodeId, d.params.localParams.fundingPrivKey, d.params.remoteParams.fundingPubKey)
them ! FundingLocked(temporaryChannelId, channelId, Some(localNodeSig), Some(localBitcoinSig), nextPerCommitmentPoint)
them ! FundingLocked(temporaryChannelId, channelId, nextPerCommitmentPoint)
deferred.map(self ! _)
// TODO: htlcIdx should not be 0 when resuming connection
goto(WAIT_FOR_FUNDING_LOCKED) using DATA_NORMAL(params, commitments.copy(channelId = channelId), None)
@ -336,31 +348,23 @@ class Channel(val them: ActorRef, val blockchain: ActorRef, router: ActorRef, re
})
when(WAIT_FOR_FUNDING_LOCKED)(handleExceptions {
case Event(FundingLocked(_, remoteChannelId, _, _, nextPerCommitmentPoint), d: DATA_NORMAL) if remoteChannelId != d.channelId =>
case Event(FundingLocked(_, remoteChannelId, nextPerCommitmentPoint), d: DATA_NORMAL) if remoteChannelId != d.channelId =>
// TODO: channel id mismatch, can happen if minDepth is to low, negotiation not suported yet
handleLocalError(new RuntimeException(s"channel id mismatch local=${d.channelId} remote=$remoteChannelId"), d)
case Event(FundingLocked(_, _, remoteNodeSig_opt, remoteBitcoinSig_opt, nextPerCommitmentPoint), d: DATA_NORMAL) =>
log.info(s"channel ready with channelId=${java.lang.Long.toUnsignedString(d.channelId)}")
case Event(FundingLocked(_, _, nextPerCommitmentPoint), d: DATA_NORMAL) =>
log.info(s"channelId=${java.lang.Long.toUnsignedString(d.channelId)}")
Register.createAlias(remoteNodeId.hash160, d.channelId)
(remoteNodeSig_opt, remoteBitcoinSig_opt) match {
case (Some(remoteNodeSig), Some(remoteBitcoinSig)) =>
log.info(s"announcing channel ${d.channelId} on the network")
val (localNodeSig, localBitcoinSig) = Announcements.signChannelAnnouncement(d.channelId, Globals.Node.privateKey, remoteNodeId, d.params.localParams.fundingPrivKey, d.params.remoteParams.fundingPubKey)
val channelAnn = Announcements.makeChannelAnnouncement(d.channelId, Globals.Node.publicKey, remoteNodeId, d.params.localParams.fundingPrivKey.publicKey, d.params.remoteParams.fundingPubKey, localNodeSig, remoteNodeSig, localBitcoinSig, remoteBitcoinSig)
val nodeAnn = Announcements.makeNodeAnnouncement(Globals.Node.privateKey, Globals.Node.alias, Globals.Node.color, Globals.Node.address :: Nil, Platform.currentTime / 1000)
val channelUpdate = Announcements.makeChannelUpdate(Globals.Node.privateKey, remoteNodeId, d.commitments.channelId, Globals.expiry_delta_blocks, Globals.htlc_minimum_msat, Globals.fee_base_msat, Globals.fee_proportional_millionth, Platform.currentTime / 1000)
router ! channelAnn
router ! nodeAnn
router ! channelUpdate
// let's trigger the broadcast immediately so that we don't wait for 60 seconds to announce our newly created channel
// we give 3 seconds for the router-watcher roundtrip
context.system.scheduler.scheduleOnce(3 seconds, router, 'tick_broadcast)
case _ => log.info(s"channel ${d.channelId} won't be announced")
}
// this clock will be used to detect htlc timeouts
context.system.eventStream.subscribe(self, classOf[CurrentBlockCount])
goto(NORMAL) using d.copy(commitments = d.commitments.copy(remoteNextCommitInfo = Right(nextPerCommitmentPoint)))
if (Features.isChannelPublic(d.params.localParams.localFeatures) && Features.isChannelPublic(d.params.remoteParams.localFeatures)) {
val (localNodeSig, localBitcoinSig) = Announcements.signChannelAnnouncement(d.channelId, Globals.Node.privateKey, remoteNodeId, d.params.localParams.fundingPrivKey, d.params.remoteParams.fundingPubKey)
them ! AnnouncementSignatures(d.channelId, localNodeSig, localBitcoinSig)
goto(WAIT_FOR_ANN_SIGNATURES) using d.copy(commitments = d.commitments.copy(remoteNextCommitInfo = Right(nextPerCommitmentPoint)))
} else {
log.info(s"channel ${d.channelId} won't be announced")
goto(NORMAL) using d.copy(commitments = d.commitments.copy(remoteNextCommitInfo = Right(nextPerCommitmentPoint)))
}
case Event(WatchEventSpent(BITCOIN_FUNDING_SPENT, tx: Transaction), d: DATA_NORMAL) if tx.txid == d.commitments.remoteCommit.txid => handleRemoteSpentCurrent(tx, d)
@ -378,6 +382,39 @@ class Channel(val them: ActorRef, val blockchain: ActorRef, router: ActorRef, re
case Event(e: Error, d: DATA_NORMAL) => handleRemoteError(e, d)
})
when(WAIT_FOR_ANN_SIGNATURES)(handleExceptions {
case Event(AnnouncementSignatures(_, remoteNodeSig, remoteBitcoinSig), d: DATA_NORMAL) =>
log.info(s"announcing channel ${d.channelId} on the network")
val (localNodeSig, localBitcoinSig) = Announcements.signChannelAnnouncement(d.channelId, Globals.Node.privateKey, remoteNodeId, d.params.localParams.fundingPrivKey, d.params.remoteParams.fundingPubKey)
val channelAnn = Announcements.makeChannelAnnouncement(d.channelId, Globals.Node.publicKey, remoteNodeId, d.params.localParams.fundingPrivKey.publicKey, d.params.remoteParams.fundingPubKey, localNodeSig, remoteNodeSig, localBitcoinSig, remoteBitcoinSig)
val nodeAnn = Announcements.makeNodeAnnouncement(Globals.Node.privateKey, Globals.Node.alias, Globals.Node.color, Globals.Node.address :: Nil, Platform.currentTime / 1000)
val channelUpdate = Announcements.makeChannelUpdate(Globals.Node.privateKey, remoteNodeId, d.commitments.channelId, Globals.expiry_delta_blocks, Globals.htlc_minimum_msat, Globals.fee_base_msat, Globals.fee_proportional_millionth, Platform.currentTime / 1000)
router ! channelAnn
router ! nodeAnn
router ! channelUpdate
// let's trigger the broadcast immediately so that we don't wait for 60 seconds to announce our newly created channel
// we give 3 seconds for the router-watcher roundtrip
context.system.scheduler.scheduleOnce(3 seconds, router, 'tick_broadcast)
goto(NORMAL) using d
case Event(WatchEventSpent(BITCOIN_FUNDING_SPENT, tx: Transaction), d: DATA_NORMAL) if tx.txid == d.commitments.remoteCommit.txid => handleRemoteSpentCurrent(tx, d)
case Event(WatchEventSpent(BITCOIN_FUNDING_SPENT, _), d: DATA_NORMAL) => handleInformationLeak(d)
case Event(cmd: CMD_CLOSE, d: DATA_NORMAL) =>
val commitTx = d.commitments.localCommit.publishableTxs.commitTx.tx
blockchain ! PublishAsap(commitTx)
blockchain ! WatchConfirmed(self, commitTx.txid, d.params.minimumDepth, BITCOIN_CLOSE_DONE)
// there can't be htlcs at this stage
// TODO: LocalCommitPublished.claimDelayedOutputTx should be defined
val localCommitPublished = LocalCommitPublished(commitTx, None, Nil, Nil, Nil)
goto(CLOSING) using DATA_CLOSING(d.commitments, localCommitPublished = Some(localCommitPublished))
case Event(e: Error, d: DATA_NORMAL)
=> handleRemoteError(e, d)
})
/*
888b d888 d8888 8888888 888b 888 888 .d88888b. .d88888b. 8888888b.
@ -407,7 +444,6 @@ class Channel(val them: ActorRef, val blockchain: ActorRef, router: ActorRef, re
case Event(add: UpdateAddHtlc, d@DATA_NORMAL(params, commitments, _)) =>
Try(Commitments.receiveAdd(commitments, add)) match {
case Success(commitments1) =>
import scala.concurrent.ExecutionContext.Implicits.global
params.autoSignInterval.map(interval => context.system.scheduler.scheduleOnce(interval, self, CMD_SIGN))
stay using d.copy(commitments = commitments1)
case Failure(cause) => handleLocalError(cause, d)
@ -425,7 +461,6 @@ class Channel(val them: ActorRef, val blockchain: ActorRef, router: ActorRef, re
Try(Commitments.receiveFulfill(d.commitments, fulfill)) match {
case Success((commitments1, htlc)) =>
relayer ! (htlc, fulfill)
import scala.concurrent.ExecutionContext.Implicits.global
params.autoSignInterval.map(interval => context.system.scheduler.scheduleOnce(interval, self, CMD_SIGN))
stay using d.copy(commitments = commitments1)
case Failure(cause) => handleLocalError(cause, d)
@ -443,7 +478,6 @@ class Channel(val them: ActorRef, val blockchain: ActorRef, router: ActorRef, re
Try(Commitments.receiveFail(d.commitments, fail)) match {
case Success((commitments1, htlc)) =>
relayer ! (htlc, fail)
import scala.concurrent.ExecutionContext.Implicits.global
params.autoSignInterval.map(interval => context.system.scheduler.scheduleOnce(interval, self, CMD_SIGN))
stay using d.copy(commitments = commitments1)
case Failure(cause) => handleLocalError(cause, d)
@ -795,7 +829,9 @@ class Channel(val them: ActorRef, val blockchain: ActorRef, router: ActorRef, re
}
def publishMutualClosing(mutualClosing: Transaction) = {
log.info(s"closingTxId=${mutualClosing.txid}")
log.info(s"closingTxId=${
mutualClosing.txid
}")
blockchain ! PublishAsap(mutualClosing)
// TODO: hardcoded mindepth
blockchain ! WatchConfirmed(self, mutualClosing.txid, 3, BITCOIN_CLOSE_DONE)
@ -817,9 +853,15 @@ class Channel(val them: ActorRef, val blockchain: ActorRef, router: ActorRef, re
// we also watch the htlc-timeout outputs in order to extract payment preimages
localCommitPublished.htlcTimeoutTxs.foreach(tx => {
require(tx.txIn.size == 1, s"an htlc-timeout tx must have exactly 1 input (has ${tx.txIn.size})")
require(tx.txIn.size == 1, s"an htlc-timeout tx must have exactly 1 input (has ${
tx.txIn.size
})")
val outpoint = tx.txIn(0).outPoint
log.info(s"watching output ${outpoint.index} of commit tx ${outpoint.txid}")
log.info(s"watching output ${
outpoint.index
} of commit tx ${
outpoint.txid
}")
blockchain ! WatchSpent(relayer, outpoint.txid, outpoint.index.toInt, BITCOIN_HTLC_SPENT)
})
@ -832,7 +874,9 @@ class Channel(val them: ActorRef, val blockchain: ActorRef, router: ActorRef, re
}
def handleRemoteSpentCurrent(commitTx: Transaction, d: HasCommitments) = {
log.warning(s"they published their current commit in txid=${commitTx.txid}")
log.warning(s"they published their current commit in txid=${
commitTx.txid
}")
require(commitTx.txid == d.commitments.remoteCommit.txid, "txid mismatch")
// TODO hardcoded mindepth + shouldn't we watch the claim tx instead?
@ -845,9 +889,15 @@ class Channel(val them: ActorRef, val blockchain: ActorRef, router: ActorRef, re
// we also watch the htlc-sent outputs in order to extract payment preimages
remoteCommitPublished.claimHtlcTimeoutTxs.foreach(tx => {
require(tx.txIn.size == 1, s"a claim-htlc-timeout tx must have exactly 1 input (has ${tx.txIn.size})")
require(tx.txIn.size == 1, s"a claim-htlc-timeout tx must have exactly 1 input (has ${
tx.txIn.size
})")
val outpoint = tx.txIn(0).outPoint
log.info(s"watching output ${outpoint.index} of commit tx ${outpoint.txid}")
log.info(s"watching output ${
outpoint.index
} of commit tx ${
outpoint.txid
}")
blockchain ! WatchSpent(relayer, outpoint.txid, outpoint.index.toInt, BITCOIN_HTLC_SPENT)
})
@ -860,11 +910,15 @@ class Channel(val them: ActorRef, val blockchain: ActorRef, router: ActorRef, re
}
def handleRemoteSpentOther(tx: Transaction, d: HasCommitments) = {
log.warning(s"funding tx spent in txid=${tx.txid}")
log.warning(s"funding tx spent in txid=${
tx.txid
}")
Helpers.Closing.claimRevokedRemoteCommitTxOutputs(d.commitments, tx) match {
case Some(revokedCommitPublished) =>
log.warning(s"txid=${tx.txid} was a revoked commitment, publishing the penalty tx")
log.warning(s"txid=${
tx.txid
} was a revoked commitment, publishing the penalty tx")
them ! Error(0, "Funding tx has been spent".getBytes)
// TODO hardcoded mindepth + shouldn't we watch the claim tx instead?
@ -883,14 +937,18 @@ class Channel(val them: ActorRef, val blockchain: ActorRef, router: ActorRef, re
goto(CLOSING) using nextData
case None =>
// the published tx was neither their current commitment nor a revoked one
log.error(s"couldn't identify txid=${tx.txid}, something very bad is going on!!!")
log.error(s"couldn't identify txid=${
tx.txid
}, something very bad is going on!!!")
goto(ERR_INFORMATION_LEAK)
}
}
def handleInformationLeak(d: HasCommitments) = {
// this is never supposed to happen !!
log.error(s"our funding tx ${d.commitments.anchorId} was spent !!")
log.error(s"our funding tx ${
d.commitments.anchorId
} was spent !!")
// TODO! channel id
them ! Error(0, "Funding tx has been spent".getBytes)
// TODO: not enough

View File

@ -28,6 +28,7 @@ import scala.concurrent.duration.FiniteDuration
*/
sealed trait State
case object WAIT_FOR_INIT_INTERNAL extends State
case object WAIT_FOR_INIT extends State
case object WAIT_FOR_OPEN_CHANNEL extends State
case object WAIT_FOR_ACCEPT_CHANNEL extends State
case object WAIT_FOR_FUNDING_CREATED_INTERNAL extends State
@ -35,6 +36,7 @@ case object WAIT_FOR_FUNDING_CREATED extends State
case object WAIT_FOR_FUNDING_SIGNED extends State
case object WAIT_FOR_FUNDING_LOCKED_INTERNAL extends State
case object WAIT_FOR_FUNDING_LOCKED extends State
case object WAIT_FOR_ANN_SIGNATURES extends State
case object NORMAL extends State
case object SHUTDOWN extends State
case object NEGOTIATING extends State
@ -119,8 +121,9 @@ case class LocalCommitPublished(commitTx: Transaction, claimMainDelayedOutputTx:
case class RemoteCommitPublished(commitTx: Transaction, claimMainOutputTx: Option[Transaction], claimHtlcSuccessTxs: Seq[Transaction], claimHtlcTimeoutTxs: Seq[Transaction])
case class RevokedCommitPublished(commitTx: Transaction, claimMainOutputTx: Option[Transaction], mainPenaltyTx: Option[Transaction], claimHtlcTimeoutTxs: Seq[Transaction], htlcTimeoutTxs: Seq[Transaction], htlcPenaltyTxs: Seq[Transaction])
final case class DATA_WAIT_FOR_OPEN_CHANNEL(localParams: LocalParams, autoSignInterval: Option[FiniteDuration]) extends Data
final case class DATA_WAIT_FOR_ACCEPT_CHANNEL(temporaryChannelId: Long, localParams: LocalParams, fundingSatoshis: Long, pushMsat: Long, autoSignInterval: Option[FiniteDuration]) extends Data
final case class DATA_WAIT_FOR_INIT(localParams: LocalParams, internalInit: Either[INPUT_INIT_FUNDER, INPUT_INIT_FUNDEE], autoSignInterval: Option[FiniteDuration]) extends Data
final case class DATA_WAIT_FOR_OPEN_CHANNEL(localParams: LocalParams, remoteGlobalFeatures: BinaryData, remoteLocalFeatures: BinaryData, autoSignInterval: Option[FiniteDuration]) extends Data
final case class DATA_WAIT_FOR_ACCEPT_CHANNEL(temporaryChannelId: Long, localParams: LocalParams, fundingSatoshis: Long, pushMsat: Long, remoteGlobalFeatures: BinaryData, remoteLocalFeatures: BinaryData, autoSignInterval: Option[FiniteDuration]) extends Data
final case class DATA_WAIT_FOR_FUNDING_INTERNAL(temporaryChannelId: Long, params: ChannelParams, pushMsat: Long, remoteFirstPerCommitmentPoint: Point) extends Data
final case class DATA_WAIT_FOR_FUNDING_CREATED(temporaryChannelId: Long, params: ChannelParams, pushMsat: Long, remoteFirstPerCommitmentPoint: Point) extends Data
final case class DATA_WAIT_FOR_FUNDING_SIGNED(temporaryChannelId: Long, params: ChannelParams, fundingTx: Transaction, localSpec: CommitmentSpec, localCommitTx: CommitTx, remoteCommit: RemoteCommit) extends Data
@ -158,7 +161,9 @@ final case class LocalParams(dustLimitSatoshis: Long,
delayedPaymentKey: Scalar,
defaultFinalScriptPubKey: Seq[ScriptElt],
shaSeed: BinaryData,
isFunder: Boolean)
isFunder: Boolean,
globalFeatures: BinaryData,
localFeatures: BinaryData)
final case class RemoteParams(dustLimitSatoshis: Long,
maxHtlcValueInFlightMsat: Long,
@ -170,6 +175,8 @@ final case class RemoteParams(dustLimitSatoshis: Long,
fundingPubKey: PublicKey,
revocationBasepoint: Point,
paymentBasepoint: Point,
delayedPaymentBasepoint: Point)
delayedPaymentBasepoint: Point,
globalFeatures: BinaryData,
localFeatures: BinaryData)
// @formatter:on

View File

@ -56,7 +56,9 @@ class Register(watcher: ActorRef, router: ActorRef, relayer: ActorRef, defaultFi
delayedPaymentKey = generateKey(3),
defaultFinalScriptPubKey = defaultFinalScriptPubKey,
shaSeed = Globals.Node.seed,
isFunder = funding_opt.isDefined
isFunder = funding_opt.isDefined,
globalFeatures = Globals.global_features,
localFeatures = Globals.local_features
)
def makeChannel(conn: ActorRef, publicKey: PublicKey, ctx: ActorContext): ActorRef = {

View File

@ -141,8 +141,6 @@ object Codecs {
val fundingLockedCodec: Codec[FundingLocked] = (
("temporaryChannelId" | int64) ::
("channelId" | int64) ::
("announcementNodeSignature" | optionalSignature) ::
("announcementBitcoinSignature" | optionalSignature) ::
("nextPerCommitmentPoint" | point)).as[FundingLocked]
val shutdownCodec: Codec[wire.Shutdown] = (
@ -228,6 +226,11 @@ object Codecs {
("signature" | signature) ::
channelUpdateWitnessCodec).as[ChannelUpdate]
val announcementSignaturesCodec: Codec[AnnouncementSignatures] = (
("channelId" | int64) ::
("nodeSignature" | signature) ::
("bitcoinSignature" | signature)).as[AnnouncementSignatures]
val lightningMessageCodec = discriminated[LightningMessage].by(uint16)
.typecase(16, initCodec)
.typecase(17, errorCodec)
@ -247,6 +250,7 @@ object Codecs {
.typecase(256, channelAnnouncementCodec)
.typecase(257, nodeAnnouncementCodec)
.typecase(258, channelUpdateCodec)
.typecase(259, announcementSignaturesCodec)
val perHopPayloadCodec: Codec[PerHopPayload] = (
("realm" | ignore(8 * 1)) ::

View File

@ -64,8 +64,6 @@ case class FundingSigned(temporaryChannelId: Long,
case class FundingLocked(temporaryChannelId: Long,
channelId: Long,
announcementNodeSignature: Option[BinaryData],
announcementBitcoinSignature: Option[BinaryData],
nextPerCommitmentPoint: Point) extends ChannelMessage
case class Shutdown(channelId: Long,
@ -130,5 +128,9 @@ case class ChannelUpdate(signature: BinaryData,
feeBaseMsat: Long,
feeProportionalMillionths: Long) extends RoutingMessage
case class AnnouncementSignatures(channelId: Long,
nodeSignature: BinaryData,
bitcoinSignature: BinaryData) extends RoutingMessage
case class PerHopPayload(amt_to_forward: Long,
outgoing_cltv_value: Int)

View File

@ -0,0 +1,20 @@
package fr.acinq.eclair
import org.junit.runner.RunWith
import org.scalatest.FunSuite
import org.scalatest.junit.JUnitRunner
import Features._
/**
* Created by PM on 27/01/2017.
*/
@RunWith(classOf[JUnitRunner])
class FeaturesSpec extends FunSuite {
test("'channel_public' feature") {
assert(isChannelPublic("") === false)
assert(isChannelPublic("00") === false)
assert(isChannelPublic("01") === true)
}
}

View File

@ -28,7 +28,9 @@ object TestConstants {
delayedPaymentKey = PrivateKey(Array.fill[Byte](32)(4), compressed = true),
defaultFinalScriptPubKey = Script.pay2wpkh(PrivateKey(Array.fill[Byte](32)(5), compressed = true).publicKey),
shaSeed = Crypto.sha256("alice-seed".getBytes()),
isFunder = true
isFunder = true,
globalFeatures = "",
localFeatures = "00" // no announcement
)
}
@ -48,7 +50,9 @@ object TestConstants {
delayedPaymentKey = PrivateKey(Array.fill[Byte](32)(14), compressed = true),
defaultFinalScriptPubKey = Script.pay2wpkh(PrivateKey(Array.fill[Byte](32)(15), compressed = true).publicKey),
shaSeed = Crypto.sha256("alice-seed".getBytes()),
isFunder = false
isFunder = false,
globalFeatures = "",
localFeatures = "00" // no announcement
)
}

View File

@ -24,6 +24,10 @@ trait StateTestsHelperMethods extends TestKitBase {
bob2blockchain: TestProbe): Unit = {
alice ! INPUT_INIT_FUNDER(TestConstants.fundingSatoshis, TestConstants.pushMsat)
bob ! INPUT_INIT_FUNDEE()
alice2bob.expectMsgType[Init]
alice2bob.forward(bob)
bob2alice.expectMsgType[Init]
bob2alice.forward(alice)
alice2bob.expectMsgType[OpenChannel]
alice2bob.forward(bob)
bob2alice.expectMsgType[AcceptChannel]

View File

@ -5,8 +5,8 @@ import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.eclair.TestConstants.{Alice, Bob}
import fr.acinq.eclair.blockchain.PeerWatcher
import fr.acinq.eclair.channel.{WAIT_FOR_FUNDING_CREATED_INTERNAL, _}
import fr.acinq.eclair.wire.{AcceptChannel, Error, OpenChannel}
import fr.acinq.eclair.{TestkitBaseClass, TestBitcoinClient, TestConstants}
import fr.acinq.eclair.wire.{AcceptChannel, Error, Init, OpenChannel}
import fr.acinq.eclair.{TestBitcoinClient, TestConstants, TestkitBaseClass}
import org.junit.runner.RunWith
import org.scalatest.junit.JUnitRunner
@ -33,6 +33,10 @@ class WaitForAcceptChannelStateSpec extends TestkitBaseClass {
alice ! INPUT_INIT_FUNDER(TestConstants.fundingSatoshis, TestConstants.pushMsat)
bob ! INPUT_INIT_FUNDEE()
within(30 seconds) {
alice2bob.expectMsgType[Init]
alice2bob.forward(bob)
bob2alice.expectMsgType[Init]
bob2alice.forward(alice)
alice2bob.expectMsgType[OpenChannel]
alice2bob.forward(bob)
awaitCond(alice.stateName == WAIT_FOR_ACCEPT_CHANNEL)

View File

@ -1,10 +1,10 @@
package fr.acinq.eclair.channel.states.a
import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.eclair.{TestkitBaseClass, TestConstants}
import fr.acinq.eclair.{TestConstants, TestkitBaseClass}
import fr.acinq.eclair.TestConstants.{Alice, Bob}
import fr.acinq.eclair.channel._
import fr.acinq.eclair.wire.{Error, OpenChannel}
import fr.acinq.eclair.wire.{Error, Init, OpenChannel}
import org.junit.runner.RunWith
import org.scalatest.junit.JUnitRunner
@ -30,6 +30,10 @@ class WaitForOpenChannelStateSpec extends TestkitBaseClass {
alice ! INPUT_INIT_FUNDER(TestConstants.fundingSatoshis, TestConstants.pushMsat)
bob ! INPUT_INIT_FUNDEE()
within(30 seconds) {
alice2bob.expectMsgType[Init]
alice2bob.forward(bob)
bob2alice.expectMsgType[Init]
bob2alice.forward(alice)
awaitCond(bob.stateName == WAIT_FOR_OPEN_CHANNEL)
}
test((bob, alice2bob, bob2alice, bob2blockchain))

View File

@ -5,8 +5,8 @@ import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.eclair.TestConstants.{Alice, Bob}
import fr.acinq.eclair.blockchain.{MakeFundingTx, PeerWatcher}
import fr.acinq.eclair.channel._
import fr.acinq.eclair.wire.{AcceptChannel, Error, FundingCreated, OpenChannel}
import fr.acinq.eclair.{TestkitBaseClass, TestBitcoinClient, TestConstants}
import fr.acinq.eclair.wire._
import fr.acinq.eclair.{TestBitcoinClient, TestConstants, TestkitBaseClass}
import org.junit.runner.RunWith
import org.scalatest.junit.JUnitRunner
@ -33,6 +33,10 @@ class WaitForFundingCreatedInternalStateSpec extends TestkitBaseClass {
alice ! INPUT_INIT_FUNDER(TestConstants.fundingSatoshis, TestConstants.pushMsat)
bob ! INPUT_INIT_FUNDEE()
within(30 seconds) {
alice2bob.expectMsgType[Init]
alice2bob.forward(bob)
bob2alice.expectMsgType[Init]
bob2alice.forward(alice)
alice2bob.expectMsgType[OpenChannel]
alice2bob.forward(bob)
bob2alice.expectMsgType[AcceptChannel]

View File

@ -32,6 +32,10 @@ class WaitForFundingCreatedStateSpec extends TestkitBaseClass {
alice ! INPUT_INIT_FUNDER(TestConstants.fundingSatoshis, TestConstants.pushMsat)
bob ! INPUT_INIT_FUNDEE()
within(30 seconds) {
alice2bob.expectMsgType[Init]
alice2bob.forward(bob)
bob2alice.expectMsgType[Init]
bob2alice.forward(alice)
alice2bob.expectMsgType[OpenChannel]
alice2bob.forward(bob)
bob2alice.expectMsgType[AcceptChannel]

View File

@ -6,8 +6,8 @@ import fr.acinq.bitcoin.BinaryData
import fr.acinq.eclair.TestConstants.{Alice, Bob}
import fr.acinq.eclair.blockchain._
import fr.acinq.eclair.channel._
import fr.acinq.eclair.wire.{AcceptChannel, Error, FundingCreated, FundingSigned, OpenChannel}
import fr.acinq.eclair.{TestkitBaseClass, TestBitcoinClient, TestConstants}
import fr.acinq.eclair.wire.{AcceptChannel, Error, FundingCreated, FundingSigned, Init, OpenChannel}
import fr.acinq.eclair.{TestBitcoinClient, TestConstants, TestkitBaseClass}
import org.junit.runner.RunWith
import org.scalatest.junit.JUnitRunner
@ -34,6 +34,10 @@ class WaitForFundingSignedStateSpec extends TestkitBaseClass {
alice ! INPUT_INIT_FUNDER(TestConstants.fundingSatoshis, TestConstants.pushMsat)
bob ! INPUT_INIT_FUNDEE()
within(30 seconds) {
alice2bob.expectMsgType[Init]
alice2bob.forward(bob)
bob2alice.expectMsgType[Init]
bob2alice.forward(alice)
alice2bob.expectMsgType[OpenChannel]
alice2bob.forward(bob)
bob2alice.expectMsgType[AcceptChannel]

View File

@ -0,0 +1,122 @@
package fr.acinq.eclair.channel.states.c
import akka.actor.{ActorRef, Props}
import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.eclair.TestConstants.{Alice, Bob}
import fr.acinq.eclair.blockchain._
import fr.acinq.eclair.channel._
import fr.acinq.eclair.wire._
import fr.acinq.eclair.{TestBitcoinClient, TestConstants, TestkitBaseClass}
import org.junit.runner.RunWith
import org.scalatest.Tag
import org.scalatest.junit.JUnitRunner
import scala.concurrent.duration._
/**
* Created by PM on 05/07/2016.
*/
@RunWith(classOf[JUnitRunner])
class WaitForAnnSignaturesStateSpec extends TestkitBaseClass {
type FixtureParam = Tuple7[TestFSMRef[State, Data, Channel], TestFSMRef[State, Data, Channel], TestProbe, TestProbe, TestProbe, ActorRef, TestProbe]
override def withFixture(test: OneArgTest) = {
val alice2bob = TestProbe()
val bob2alice = TestProbe()
val alice2blockchain = TestProbe()
val blockchainA = system.actorOf(Props(new PeerWatcher(new TestBitcoinClient())))
val bob2blockchain = TestProbe()
val relayer = TestProbe()
val router = TestProbe()
val (aliceParams, bobParams) = (Alice.channelParams.copy(localFeatures = "01"), Bob.channelParams.copy(localFeatures = "01"))
val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(alice2bob.ref, alice2blockchain.ref, router.ref, relayer.ref, aliceParams, Bob.id))
val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bob2alice.ref, bob2blockchain.ref, router.ref, relayer.ref, bobParams, Alice.id))
alice ! INPUT_INIT_FUNDER(TestConstants.fundingSatoshis, TestConstants.pushMsat)
bob ! INPUT_INIT_FUNDEE()
within(30 seconds) {
alice2bob.expectMsgType[Init]
alice2bob.forward(bob)
bob2alice.expectMsgType[Init]
bob2alice.forward(alice)
alice2bob.expectMsgType[OpenChannel]
alice2bob.forward(bob)
bob2alice.expectMsgType[AcceptChannel]
bob2alice.forward(alice)
alice2blockchain.expectMsgType[MakeFundingTx]
alice2blockchain.forward(blockchainA)
alice2bob.expectMsgType[FundingCreated]
alice2bob.forward(bob)
bob2alice.expectMsgType[FundingSigned]
bob2alice.forward(alice)
alice2blockchain.expectMsgType[WatchSpent]
alice2blockchain.expectMsgType[WatchConfirmed]
alice2blockchain.forward(blockchainA)
alice2blockchain.expectMsgType[PublishAsap]
alice2blockchain.forward(blockchainA)
bob2blockchain.expectMsgType[WatchSpent]
bob2blockchain.expectMsgType[WatchConfirmed]
bob ! WatchEventConfirmed(BITCOIN_FUNDING_DEPTHOK, 400000, 42)
alice2blockchain.expectMsgType[WatchLost]
bob2blockchain.expectMsgType[WatchLost]
alice2bob.expectMsgType[FundingLocked]
alice2bob.forward(bob)
bob2alice.expectMsgType[FundingLocked]
bob2alice.forward(alice)
alice2bob.expectMsgType[AnnouncementSignatures]
awaitCond(alice.stateName == WAIT_FOR_ANN_SIGNATURES)
awaitCond(bob.stateName == WAIT_FOR_ANN_SIGNATURES)
}
test((alice, bob, alice2bob, bob2alice, alice2blockchain, blockchainA, router))
}
test("recv AnnouncementSignatures") { case (alice, _, alice2bob, bob2alice, alice2blockchain, _, router) =>
within(30 seconds) {
bob2alice.expectMsgType[AnnouncementSignatures]
bob2alice.forward(alice)
awaitCond(alice.stateName == NORMAL)
}
}
test("recv BITCOIN_FUNDING_SPENT (remote commit)") { case (alice, bob, alice2bob, bob2alice, alice2blockchain, _, router) =>
within(30 seconds) {
// bob publishes his commitment tx
val tx = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.publishableTxs.commitTx.tx
alice ! WatchEventSpent(BITCOIN_FUNDING_SPENT, tx)
alice2blockchain.expectMsgType[WatchConfirmed]
awaitCond(alice.stateName == CLOSING)
}
}
test("recv BITCOIN_FUNDING_SPENT (other commit)") { case (alice, _, alice2bob, bob2alice, alice2blockchain, _, router) =>
within(30 seconds) {
val tx = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.publishableTxs.commitTx.tx
alice ! WatchEventSpent(BITCOIN_FUNDING_SPENT, null)
alice2bob.expectMsgType[Error]
alice2blockchain.expectMsg(PublishAsap(tx))
awaitCond(alice.stateName == ERR_INFORMATION_LEAK)
}
}
test("recv Error") { case (alice, _, alice2bob, bob2alice, alice2blockchain, _, router) =>
within(30 seconds) {
val tx = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.publishableTxs.commitTx.tx
alice ! Error(0, "oops".getBytes)
awaitCond(alice.stateName == CLOSING)
alice2blockchain.expectMsg(PublishAsap(tx))
assert(alice2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_LOCALCOMMIT_DONE)
}
}
test("recv CMD_CLOSE") { case (alice, _, alice2bob, bob2alice, alice2blockchain, _, router) =>
within(30 seconds) {
val tx = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.publishableTxs.commitTx.tx
alice ! CMD_CLOSE(None)
awaitCond(alice.stateName == CLOSING)
alice2blockchain.expectMsg(PublishAsap(tx))
assert(alice2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_CLOSE_DONE)
}
}
}

View File

@ -5,8 +5,8 @@ import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.eclair.TestConstants.{Alice, Bob}
import fr.acinq.eclair.blockchain._
import fr.acinq.eclair.channel._
import fr.acinq.eclair.wire.{AcceptChannel, Error, FundingCreated, FundingLocked, FundingSigned, OpenChannel}
import fr.acinq.eclair.{TestkitBaseClass, TestBitcoinClient, TestConstants}
import fr.acinq.eclair.wire.{AcceptChannel, Error, FundingCreated, FundingLocked, FundingSigned, Init, OpenChannel}
import fr.acinq.eclair.{TestBitcoinClient, TestConstants, TestkitBaseClass}
import org.junit.runner.RunWith
import org.scalatest.junit.JUnitRunner
@ -33,6 +33,10 @@ class WaitForFundingLockedInternalStateSpec extends TestkitBaseClass {
alice ! INPUT_INIT_FUNDER(TestConstants.fundingSatoshis, TestConstants.pushMsat)
bob ! INPUT_INIT_FUNDEE()
within(30 seconds) {
alice2bob.expectMsgType[Init]
alice2bob.forward(bob)
bob2alice.expectMsgType[Init]
bob2alice.forward(alice)
alice2bob.expectMsgType[OpenChannel]
alice2bob.forward(bob)
bob2alice.expectMsgType[AcceptChannel]

View File

@ -2,7 +2,7 @@ package fr.acinq.eclair.channel.states.c
import akka.actor.{ActorRef, Props}
import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.bitcoin.Crypto
import fr.acinq.bitcoin.{BinaryData, Crypto}
import fr.acinq.eclair.TestConstants.{Alice, Bob}
import fr.acinq.eclair.blockchain._
import fr.acinq.eclair.channel._
@ -10,6 +10,7 @@ import fr.acinq.eclair.router.{Announcements, Router}
import fr.acinq.eclair.wire._
import fr.acinq.eclair.{TestBitcoinClient, TestConstants, TestkitBaseClass}
import org.junit.runner.RunWith
import org.scalatest.{ConfigMap, Tag, TestData}
import org.scalatest.junit.JUnitRunner
import scala.concurrent.duration._
@ -30,11 +31,22 @@ class WaitForFundingLockedStateSpec extends TestkitBaseClass {
val bob2blockchain = TestProbe()
val relayer = TestProbe()
val router = TestProbe()
val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(alice2bob.ref, alice2blockchain.ref, router.ref, relayer.ref, Alice.channelParams, Bob.id))
val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bob2alice.ref, bob2blockchain.ref, router.ref, relayer.ref, Bob.channelParams, Alice.id))
val (aliceParams, bobParams) = if (test.tags.contains("public")) {
(Alice.channelParams.copy(localFeatures = "01"), Bob.channelParams.copy(localFeatures = "01"))
} else {
(Alice.channelParams, Bob.channelParams)
}
val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(alice2bob.ref, alice2blockchain.ref, router.ref, relayer.ref, aliceParams, Bob.id))
val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bob2alice.ref, bob2blockchain.ref, router.ref, relayer.ref, bobParams, Alice.id))
alice ! INPUT_INIT_FUNDER(TestConstants.fundingSatoshis, TestConstants.pushMsat)
bob ! INPUT_INIT_FUNDEE()
within(30 seconds) {
alice2bob.expectMsgType[Init]
alice2bob.forward(bob)
bob2alice.expectMsgType[Init]
bob2alice.forward(alice)
alice2bob.expectMsgType[OpenChannel]
alice2bob.forward(bob)
bob2alice.expectMsgType[AcceptChannel]
@ -67,12 +79,15 @@ class WaitForFundingLockedStateSpec extends TestkitBaseClass {
bob2alice.expectMsgType[FundingLocked]
bob2alice.forward(alice)
awaitCond(alice.stateName == NORMAL)
val channelAnnouncement = router.expectMsgType[ChannelAnnouncement]
val nodeAnnouncement = router.expectMsgType[NodeAnnouncement]
val channelUpdate = router.expectMsgType[ChannelUpdate]
/*assert(Announcements.checkSigs(channelAnnouncement))
assert(Announcements.checkSig(nodeAnnouncement))
assert(Announcements.checkSig(channelUpdate, TestConstants.Alice.id))*/
}
}
test("recv FundingLocked (with announcement)", Tag("public")) { case (alice, _, alice2bob, bob2alice, alice2blockchain, _, router) =>
within(30 seconds) {
bob2alice.expectMsgType[FundingLocked]
bob2alice.forward(alice)
awaitCond(alice.stateName == WAIT_FOR_ANN_SIGNATURES)
alice2bob.expectMsgType[AnnouncementSignatures]
}
}
@ -127,5 +142,4 @@ class WaitForFundingLockedStateSpec extends TestkitBaseClass {
assert(alice2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_CLOSE_DONE)
}
}
}

View File

@ -141,7 +141,7 @@ class CodecsSpec extends FunSuite {
val accept = AcceptChannel(2, 3, 4, 5, 6, 7, 8, 9, publicKey(1), point(2), point(3), point(4), point(5))
val funding_created = FundingCreated(2, bin(32, 0), 3, randomSignature)
val funding_signed = FundingSigned(2, randomSignature)
val funding_locked = FundingLocked(1, 2, Some(randomSignature), Some(randomSignature), point(2))
val funding_locked = FundingLocked(1, 2, point(2))
val update_fee = UpdateFee(1, 2)
val shutdown = Shutdown(1, bin(47, 0))
val closing_signed = ClosingSigned(1, 2, randomSignature)
@ -153,11 +153,12 @@ class CodecsSpec extends FunSuite {
val channel_announcement = ChannelAnnouncement(randomSignature, randomSignature, randomSignature, randomSignature, 1, bin(33, 5), bin(33, 6), bin(33, 7), bin(33, 8))
val node_announcement = NodeAnnouncement(randomSignature, 1, bin(33, 2), (100.toByte, 200.toByte, 300.toByte), "node-alias", bin(0, 0), new InetSocketAddress(InetAddress.getByAddress(Array[Byte](192.toByte, 168.toByte, 1.toByte, 42.toByte)), 42000) :: Nil)
val channel_update = ChannelUpdate(randomSignature, 1, 2, bin(2, 2), 3, 4, 5, 6)
val announcement_signatures = AnnouncementSignatures(1, randomSignature, randomSignature)
val msgs: List[LightningMessage] =
open :: accept :: funding_created :: funding_signed :: funding_locked :: update_fee :: shutdown :: closing_signed ::
update_add_htlc :: update_fulfill_htlc :: update_fail_htlc :: commit_sig :: revoke_and_ack ::
channel_announcement :: node_announcement :: channel_update :: Nil
channel_announcement :: node_announcement :: channel_update :: announcement_signatures :: Nil
msgs.foreach {
case msg => {