From b33c9ecaac27108a0469271ad2e454f808b2b009 Mon Sep 17 00:00:00 2001 From: Pierre-Marie Padiou Date: Mon, 3 Jun 2019 19:10:14 +0200 Subject: [PATCH 01/21] Smarter relay logic (#1011) When relaying a payment, we look in the onion to find the next `shortChannelId`. But we may choose a different channel (to the same node), because the requested channel may not have enough balance, of for some other reasons, as permitted by the spec. Currently we limit ourselves to only two attempts: one with a "preferred" channel, and one with the originally requested channel if is different from the preferred one. This has drawbacks, because if we have multiple channels to the same node, we may not be able to relay a payment if the "preferred" channel is currently unavailable (e.g. because of an htlc in-flight value that is too high). We now retry as many times as there are available channels, in our order of preference, and if all fail, then we return a failure message for the originally requested channel. --- .../eclair/channel/ChannelExceptions.scala | 2 +- .../acinq/eclair/channel/ChannelTypes.scala | 2 +- .../fr/acinq/eclair/payment/Relayer.scala | 286 ++++++++++-------- .../eclair/payment/ChannelSelectionSpec.scala | 40 +-- .../fr/acinq/eclair/payment/RelayerSpec.scala | 52 +++- 5 files changed, 233 insertions(+), 149 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelExceptions.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelExceptions.scala index 55456ba48..ab5efa9a1 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelExceptions.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelExceptions.scala @@ -18,7 +18,7 @@ package fr.acinq.eclair.channel import fr.acinq.bitcoin.Crypto.Scalar import fr.acinq.bitcoin.{ByteVector32, Transaction} -import fr.acinq.eclair.UInt64 +import fr.acinq.eclair.{ShortChannelId, UInt64} import fr.acinq.eclair.payment.Origin import fr.acinq.eclair.wire.{ChannelUpdate, UpdateAddHtlc} diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelTypes.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelTypes.scala index 5cab113be..0f451a91b 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelTypes.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelTypes.scala @@ -108,7 +108,7 @@ case class BITCOIN_PARENT_TX_CONFIRMED(childTx: Transaction) extends BitcoinEven */ sealed trait Command -final case class CMD_ADD_HTLC(amountMsat: Long, paymentHash: ByteVector32, cltvExpiry: Long, onion: ByteVector = Sphinx.LAST_PACKET.serialize, upstream: Either[UUID, UpdateAddHtlc], commit: Boolean = false, redirected: Boolean = false) extends Command +final case class CMD_ADD_HTLC(amountMsat: Long, paymentHash: ByteVector32, cltvExpiry: Long, onion: ByteVector = Sphinx.LAST_PACKET.serialize, upstream: Either[UUID, UpdateAddHtlc], commit: Boolean = false, previousFailures: Seq[AddHtlcFailed] = Seq.empty) extends Command final case class CMD_FULFILL_HTLC(id: Long, r: ByteVector32, commit: Boolean = false) extends Command final case class CMD_FAIL_HTLC(id: Long, reason: Either[ByteVector, FailureMessage], commit: Boolean = false) extends Command final case class CMD_FAIL_MALFORMED_HTLC(id: Long, onionHash: ByteVector32, failureCode: Int, commit: Boolean = false) extends Command diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/Relayer.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/Relayer.scala index 050d933a3..0d0b55fe4 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/Relayer.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/Relayer.scala @@ -42,7 +42,7 @@ case class Local(id: UUID, sender: Option[ActorRef]) extends Origin // we don't case class Relayed(originChannelId: ByteVector32, originHtlcId: Long, amountMsatIn: Long, amountMsatOut: Long) extends Origin sealed trait ForwardMessage -case class ForwardAdd(add: UpdateAddHtlc, canRedirect: Boolean = true) extends ForwardMessage +case class ForwardAdd(add: UpdateAddHtlc, previousFailures: Seq[AddHtlcFailed] = Seq.empty) extends ForwardMessage case class ForwardFulfill(fulfill: UpdateFulfillHtlc, to: Origin, htlc: UpdateAddHtlc) extends ForwardMessage case class ForwardFail(fail: UpdateFailHtlc, to: Origin, htlc: UpdateAddHtlc) extends ForwardMessage case class ForwardFailMalformed(fail: UpdateFailMalformedHtlc, to: Origin, htlc: UpdateAddHtlc) extends ForwardMessage @@ -58,13 +58,13 @@ class Relayer(nodeParams: NodeParams, register: ActorRef, paymentHandler: ActorR import Relayer._ // we pass these to helpers classes so that they have the logging context - implicit def implicitLog = log + implicit def implicitLog: LoggingAdapter = log context.system.eventStream.subscribe(self, classOf[LocalChannelUpdate]) context.system.eventStream.subscribe(self, classOf[LocalChannelDown]) context.system.eventStream.subscribe(self, classOf[AvailableBalanceChanged]) - val commandBuffer = context.actorOf(Props(new CommandBuffer(nodeParams, register))) + private val commandBuffer = context.actorOf(Props(new CommandBuffer(nodeParams, register))) override def receive: Receive = main(Map.empty, new mutable.HashMap[PublicKey, mutable.Set[ShortChannelId]] with mutable.MultiMap[PublicKey, ShortChannelId]) @@ -85,7 +85,7 @@ class Relayer(nodeParams: NodeParams, register: ActorRef, paymentHandler: ActorR } context become main(channelUpdates1, node2channels) - case ForwardAdd(add, canRedirect) => + case ForwardAdd(add, previousFailures) => log.debug(s"received forwarding request for htlc #${add.id} paymentHash=${add.paymentHash} from channelId=${add.channelId}") tryParsePacket(add, nodeParams.privateKey) match { case Success(p: FinalPayload) => @@ -98,12 +98,11 @@ class Relayer(nodeParams: NodeParams, register: ActorRef, paymentHandler: ActorR paymentHandler forward addHtlc } case Success(r: RelayPayload) => - val selectedShortChannelId = if (canRedirect) selectPreferredChannel(r, channelUpdates, node2channels) else r.payload.shortChannelId - handleRelay(r, channelUpdates.get(selectedShortChannelId).map(_.channelUpdate)) match { + handleRelay(r, channelUpdates, node2channels, previousFailures) match { case Left(cmdFail) => log.info(s"rejecting htlc #${add.id} paymentHash=${add.paymentHash} from channelId=${add.channelId} to shortChannelId=${r.payload.shortChannelId} reason=${cmdFail.reason}") commandBuffer ! CommandBuffer.CommandSend(add.channelId, add.id, cmdFail) - case Right(cmdAdd) => + case Right((selectedShortChannelId, cmdAdd)) => log.info(s"forwarding htlc #${add.id} paymentHash=${add.paymentHash} from channelId=${add.channelId} to shortChannelId=$selectedShortChannelId") register ! Register.ForwardShortId(selectedShortChannelId, cmdAdd) } @@ -119,79 +118,74 @@ class Relayer(nodeParams: NodeParams, register: ActorRef, paymentHandler: ActorR val cmdFail = CMD_FAIL_HTLC(add.id, Right(UnknownNextPeer), commit = true) commandBuffer ! CommandBuffer.CommandSend(add.channelId, add.id, cmdFail) - case Status.Failure(AddHtlcFailed(_, paymentHash, _, Local(id, None), _, _)) => - // we sent the payment, but we probably restarted and the reference to the original sender was lost, - // we publish the failure on the event stream and update the status in paymentDb - nodeParams.db.payments.updateOutgoingPayment(id, OutgoingPaymentStatus.FAILED) - context.system.eventStream.publish(PaymentFailed(id, paymentHash, Nil)) - - case Status.Failure(AddHtlcFailed(_, _, error, Local(_, Some(sender)), _, _)) => - sender ! Status.Failure(error) - - case Status.Failure(AddHtlcFailed(_, paymentHash, error, Relayed(originChannelId, originHtlcId, _, _), channelUpdate_opt, originalCommand_opt)) => - originalCommand_opt match { - case Some(cmd) if cmd.redirected && cmd.upstream.isRight => // cmd.upstream_opt.isDefined always true since origin = relayed - // if it was redirected, we give it one more try with the original requested channel (meaning that the error returned will always be for the requested channel) - log.info(s"retrying htlc #$originHtlcId paymentHash=$paymentHash from channelId=$originChannelId") - self ! ForwardAdd(cmd.upstream.right.get, canRedirect = false) - case _ => - // otherwise we just return a failure - val failure = (error, channelUpdate_opt) match { - case (_: ExpiryTooSmall, Some(channelUpdate)) => ExpiryTooSoon(channelUpdate) - case (_: ExpiryTooBig, _) => ExpiryTooFar - case (_: InsufficientFunds, Some(channelUpdate)) => TemporaryChannelFailure(channelUpdate) - case (_: TooManyAcceptedHtlcs, Some(channelUpdate)) => TemporaryChannelFailure(channelUpdate) - case (_: ChannelUnavailable, Some(channelUpdate)) if !Announcements.isEnabled(channelUpdate.channelFlags) => ChannelDisabled(channelUpdate.messageFlags, channelUpdate.channelFlags, channelUpdate) - case (_: ChannelUnavailable, None) => PermanentChannelFailure - case (_: HtlcTimedout, _) => PermanentChannelFailure - case _ => TemporaryNodeFailure + case Status.Failure(addFailed: AddHtlcFailed) => + import addFailed.paymentHash + addFailed.origin match { + case Local(id, None) => + // we sent the payment, but we probably restarted and the reference to the original sender was lost, + // we publish the failure on the event stream and update the status in paymentDb + nodeParams.db.payments.updateOutgoingPayment(id, OutgoingPaymentStatus.FAILED) + context.system.eventStream.publish(PaymentFailed(id, paymentHash, Nil)) + case Local(_, Some(sender)) => + sender ! Status.Failure(addFailed) + case Relayed(originChannelId, originHtlcId, _, _) => + addFailed.originalCommand match { + case Some(cmd) => + log.info(s"retrying htlc #$originHtlcId paymentHash=$paymentHash from channelId=$originChannelId") + // NB: cmd.upstream.right is defined since this is a relayed payment + self ! ForwardAdd(cmd.upstream.right.get, cmd.previousFailures :+ addFailed) + case None => + val failure = translateError(addFailed) + val cmdFail = CMD_FAIL_HTLC(originHtlcId, Right(failure), commit = true) + log.info(s"rejecting htlc #$originHtlcId paymentHash=$paymentHash from channelId=$originChannelId reason=${cmdFail.reason}") + commandBuffer ! CommandBuffer.CommandSend(originChannelId, originHtlcId, cmdFail) } - val cmdFail = CMD_FAIL_HTLC(originHtlcId, Right(failure), commit = true) - log.info(s"rejecting htlc #$originHtlcId paymentHash=$paymentHash from channelId=$originChannelId reason=${cmdFail.reason}") - commandBuffer ! CommandBuffer.CommandSend(originChannelId, originHtlcId, cmdFail) } - case ForwardFulfill(fulfill, Local(id, None), add) => - val feesPaid = MilliSatoshi(0) - context.system.eventStream.publish(PaymentSent(id, MilliSatoshi(add.amountMsat), feesPaid, add.paymentHash, fulfill.paymentPreimage, fulfill.channelId)) - // we sent the payment, but we probably restarted and the reference to the original sender was lost, - // we publish the failure on the event stream and update the status in paymentDb - nodeParams.db.payments.updateOutgoingPayment(id, OutgoingPaymentStatus.SUCCEEDED, Some(fulfill.paymentPreimage)) - context.system.eventStream.publish(PaymentSucceeded(id, add.amountMsat, add.paymentHash, fulfill.paymentPreimage, Nil)) // + case ForwardFulfill(fulfill, to, add) => + to match { + case Local(id, None) => + val feesPaid = MilliSatoshi(0) + context.system.eventStream.publish(PaymentSent(id, MilliSatoshi(add.amountMsat), feesPaid, add.paymentHash, fulfill.paymentPreimage, fulfill.channelId)) + // we sent the payment, but we probably restarted and the reference to the original sender was lost, + // we publish the failure on the event stream and update the status in paymentDb + nodeParams.db.payments.updateOutgoingPayment(id, OutgoingPaymentStatus.SUCCEEDED, Some(fulfill.paymentPreimage)) + context.system.eventStream.publish(PaymentSucceeded(id, add.amountMsat, add.paymentHash, fulfill.paymentPreimage, Nil)) // + case Local(_, Some(sender)) => + sender ! fulfill + case Relayed(originChannelId, originHtlcId, amountMsatIn, amountMsatOut) => + val cmd = CMD_FULFILL_HTLC(originHtlcId, fulfill.paymentPreimage, commit = true) + commandBuffer ! CommandBuffer.CommandSend(originChannelId, originHtlcId, cmd) + context.system.eventStream.publish(PaymentRelayed(MilliSatoshi(amountMsatIn), MilliSatoshi(amountMsatOut), add.paymentHash, fromChannelId = originChannelId, toChannelId = fulfill.channelId)) + } - case ForwardFulfill(fulfill, Local(_, Some(sender)), _) => - sender ! fulfill + case ForwardFail(fail, to, add) => + to match { + case Local(id, None) => + // we sent the payment, but we probably restarted and the reference to the original sender was lost + // we publish the failure on the event stream and update the status in paymentDb + nodeParams.db.payments.updateOutgoingPayment(id, OutgoingPaymentStatus.FAILED) + context.system.eventStream.publish(PaymentFailed(id, add.paymentHash, Nil)) + case Local(_, Some(sender)) => + sender ! fail + case Relayed(originChannelId, originHtlcId, _, _) => + val cmd = CMD_FAIL_HTLC(originHtlcId, Left(fail.reason), commit = true) + commandBuffer ! CommandBuffer.CommandSend(originChannelId, originHtlcId, cmd) + } - case ForwardFulfill(fulfill, Relayed(originChannelId, originHtlcId, amountMsatIn, amountMsatOut), add) => - val cmd = CMD_FULFILL_HTLC(originHtlcId, fulfill.paymentPreimage, commit = true) - commandBuffer ! CommandBuffer.CommandSend(originChannelId, originHtlcId, cmd) - context.system.eventStream.publish(PaymentRelayed(MilliSatoshi(amountMsatIn), MilliSatoshi(amountMsatOut), add.paymentHash, fromChannelId = originChannelId, toChannelId = fulfill.channelId)) - - case ForwardFail(_, Local(id, None), add) => - // we sent the payment, but we probably restarted and the reference to the original sender was lost - // we publish the failure on the event stream and update the status in paymentDb - nodeParams.db.payments.updateOutgoingPayment(id, OutgoingPaymentStatus.FAILED) - context.system.eventStream.publish(PaymentFailed(id, add.paymentHash, Nil)) - - case ForwardFail(fail, Local(_, Some(sender)), _) => - sender ! fail - - case ForwardFail(fail, Relayed(originChannelId, originHtlcId, _, _), _) => - val cmd = CMD_FAIL_HTLC(originHtlcId, Left(fail.reason), commit = true) - commandBuffer ! CommandBuffer.CommandSend(originChannelId, originHtlcId, cmd) - - case ForwardFailMalformed(_, Local(id, None), add) => - // we sent the payment, but we probably restarted and the reference to the original sender was lost - // we publish the failure on the event stream and update the status in paymentDb - nodeParams.db.payments.updateOutgoingPayment(id, OutgoingPaymentStatus.FAILED) - context.system.eventStream.publish(PaymentFailed(id, add.paymentHash, Nil)) - - case ForwardFailMalformed(fail, Local(_, Some(sender)), _) => - sender ! fail - - case ForwardFailMalformed(fail, Relayed(originChannelId, originHtlcId, _, _), _) => - val cmd = CMD_FAIL_MALFORMED_HTLC(originHtlcId, fail.onionHash, fail.failureCode, commit = true) - commandBuffer ! CommandBuffer.CommandSend(originChannelId, originHtlcId, cmd) + case ForwardFailMalformed(fail, to, add) => + to match { + case Local(id, None) => + // we sent the payment, but we probably restarted and the reference to the original sender was lost + // we publish the failure on the event stream and update the status in paymentDb + nodeParams.db.payments.updateOutgoingPayment(id, OutgoingPaymentStatus.FAILED) + context.system.eventStream.publish(PaymentFailed(id, add.paymentHash, Nil)) + case Local(_, Some(sender)) => + sender ! fail + case Relayed(originChannelId, originHtlcId, _, _) => + val cmd = CMD_FAIL_MALFORMED_HTLC(originHtlcId, fail.onionHash, fail.failureCode, commit = true) + commandBuffer ! CommandBuffer.CommandSend(originChannelId, originHtlcId, cmd) + } case ack: CommandBuffer.CommandAck => commandBuffer forward ack @@ -209,8 +203,8 @@ object Relayer { sealed trait NextPayload case class FinalPayload(add: UpdateAddHtlc, payload: PerHopPayload) extends NextPayload case class RelayPayload(add: UpdateAddHtlc, payload: PerHopPayload, nextPacket: Sphinx.Packet) extends NextPayload { - val relayFeeMsat = add.amountMsat - payload.amtToForward - val expiryDelta = add.cltvExpiry - payload.outgoingCltvValue + val relayFeeMsat: Long = add.amountMsat - payload.amtToForward + val expiryDelta: Long = add.cltvExpiry - payload.outgoingCltvValue } // @formatter:on @@ -265,7 +259,80 @@ object Relayer { * - a CMD_FAIL_HTLC to be sent back upstream * - a CMD_ADD_HTLC to propagate downstream */ - def handleRelay(relayPayload: RelayPayload, channelUpdate_opt: Option[ChannelUpdate])(implicit log: LoggingAdapter): Either[CMD_FAIL_HTLC, CMD_ADD_HTLC] = { + def handleRelay(relayPayload: RelayPayload, channelUpdates: Map[ShortChannelId, OutgoingChannel], node2channels: mutable.Map[PublicKey, mutable.Set[ShortChannelId]] with mutable.MultiMap[PublicKey, ShortChannelId], previousFailures: Seq[AddHtlcFailed])(implicit log: LoggingAdapter): Either[CMD_FAIL_HTLC, (ShortChannelId, CMD_ADD_HTLC)] = { + import relayPayload._ + log.info(s"relaying htlc #${add.id} paymentHash={} from channelId={} to requestedShortChannelId={} previousAttempts={}", add.paymentHash, add.channelId, relayPayload.payload.shortChannelId, previousFailures.size) + val alreadyTried = previousFailures.flatMap(_.channelUpdate).map(_.shortChannelId) + selectPreferredChannel(relayPayload, channelUpdates, node2channels, alreadyTried) + .flatMap(selectedShortChannelId => channelUpdates.get(selectedShortChannelId).map(_.channelUpdate)) match { + case None if previousFailures.nonEmpty => + // no more channels to try + val error = previousFailures + // we return the error for the initially requested channel if it exists + .find(_.channelUpdate.map(_.shortChannelId).contains(relayPayload.payload.shortChannelId)) + // otherwise we return the error for the first channel tried + .getOrElse(previousFailures.head) + Left(CMD_FAIL_HTLC(add.id, Right(translateError(error)), commit = true)) + case channelUpdate_opt => + relayOrFail(relayPayload, channelUpdate_opt, previousFailures) + } + } + + /** + * Select a channel to the same node to relay the payment to, that has the lowest balance and is compatible in + * terms of fees, expiry_delta, etc. + * + * If no suitable channel is found we default to the originally requested channel. + */ + def selectPreferredChannel(relayPayload: RelayPayload, channelUpdates: Map[ShortChannelId, OutgoingChannel], node2channels: mutable.Map[PublicKey, mutable.Set[ShortChannelId]] with mutable.MultiMap[PublicKey, ShortChannelId], alreadyTried: Seq[ShortChannelId])(implicit log: LoggingAdapter): Option[ShortChannelId] = { + import relayPayload.add + val requestedShortChannelId = relayPayload.payload.shortChannelId + log.debug(s"selecting next channel for htlc #${add.id} paymentHash={} from channelId={} to requestedShortChannelId={} previousAttempts={}", add.paymentHash, add.channelId, requestedShortChannelId, alreadyTried.size) + // first we find out what is the next node + channelUpdates.get(requestedShortChannelId) match { + case Some(OutgoingChannel(nextNodeId, _, _)) => + log.debug(s"next hop for htlc #{} paymentHash={} is nodeId={}", add.id, add.paymentHash, nextNodeId) + // then we retrieve all known channels to this node + val allChannels = node2channels.getOrElse(nextNodeId, Set.empty[ShortChannelId]) + // we then filter out channels that we have already tried + val candidateChannels = allChannels -- alreadyTried + // and we filter keep the ones that are compatible with this payment (mainly fees, expiry delta) + candidateChannels + .map { shortChannelId => + val channelInfo_opt = channelUpdates.get(shortChannelId) + val channelUpdate_opt = channelInfo_opt.map(_.channelUpdate) + val relayResult = relayOrFail(relayPayload, channelUpdate_opt) + log.debug(s"candidate channel for htlc #${add.id} paymentHash=${add.paymentHash}: shortChannelId={} balanceMsat={} channelUpdate={} relayResult={}", shortChannelId, channelInfo_opt.map(_.availableBalanceMsat).getOrElse(""), channelUpdate_opt.getOrElse(""), relayResult) + (shortChannelId, channelInfo_opt, relayResult) + } + .collect { case (shortChannelId, Some(channelInfo), Right(_)) => (shortChannelId, channelInfo.availableBalanceMsat) } + .filter(_._2 > relayPayload.payload.amtToForward) // we only keep channels that have enough balance to handle this payment + .toList // needed for ordering + .sortBy(_._2) // we want to use the channel with the lowest available balance that can process the payment + .headOption match { + case Some((preferredShortChannelId, availableBalanceMsat)) if preferredShortChannelId != requestedShortChannelId => + log.info("replacing requestedShortChannelId={} by preferredShortChannelId={} with availableBalanceMsat={}", requestedShortChannelId, preferredShortChannelId, availableBalanceMsat) + Some(preferredShortChannelId) + case Some(_) => + // the requested short_channel_id is already our preferred channel + Some(requestedShortChannelId) + case None if !alreadyTried.contains(requestedShortChannelId) => + // no channel seem to work for this payment, we keep the requested channel id + Some(requestedShortChannelId) + case None => + // no channel seem to work for this payment and we have already tried the requested channel id: we give up + None + } + case _ => Some(requestedShortChannelId) // we don't have a channel_update for this short_channel_id + } + } + + /** + * This helper method will tell us if it is not even worth attempting to relay the payment to our local outgoing + * channel, because some parameters don't match with our settings for that channel. In that case we directly fail the + * htlc. + */ + def relayOrFail(relayPayload: RelayPayload, channelUpdate_opt: Option[ChannelUpdate], previousFailures: Seq[AddHtlcFailed] = Seq.empty)(implicit log: LoggingAdapter): Either[CMD_FAIL_HTLC, (ShortChannelId, CMD_ADD_HTLC)] = { import relayPayload._ channelUpdate_opt match { case None => @@ -279,58 +346,27 @@ object Relayer { case Some(channelUpdate) if relayPayload.relayFeeMsat < nodeFee(channelUpdate.feeBaseMsat, channelUpdate.feeProportionalMillionths, payload.amtToForward) => Left(CMD_FAIL_HTLC(add.id, Right(FeeInsufficient(add.amountMsat, channelUpdate)), commit = true)) case Some(channelUpdate) => - val isRedirected = (channelUpdate.shortChannelId != payload.shortChannelId) // we may decide to use another channel (to the same node) from the one requested - Right(CMD_ADD_HTLC(payload.amtToForward, add.paymentHash, payload.outgoingCltvValue, nextPacket.serialize, upstream = Right(add), commit = true, redirected = isRedirected)) + Right((channelUpdate.shortChannelId, CMD_ADD_HTLC(payload.amtToForward, add.paymentHash, payload.outgoingCltvValue, nextPacket.serialize, upstream = Right(add), commit = true, previousFailures = previousFailures))) } } /** - * Select a channel to the same node to the relay the payment to, that has the lowest balance and is compatible in - * terms of fees, expiry_delta, etc. - * - * If no suitable channel is found we default to the originally requested channel. - * - * @param relayPayload - * @param channelUpdates - * @param node2channels - * @param log - * @return + * This helper method translates relaying errors (returned by the downstream outgoing channel) to BOLT 4 standard + * errors that we should return upstream. */ - def selectPreferredChannel(relayPayload: RelayPayload, channelUpdates: Map[ShortChannelId, OutgoingChannel], node2channels: mutable.Map[PublicKey, mutable.Set[ShortChannelId]] with mutable.MultiMap[PublicKey, ShortChannelId])(implicit log: LoggingAdapter): ShortChannelId = { - import relayPayload.add - val requestedShortChannelId = relayPayload.payload.shortChannelId - log.debug(s"selecting next channel for htlc #{} paymentHash={} from channelId={} to requestedShortChannelId={}", add.id, add.paymentHash, add.channelId, requestedShortChannelId) - // first we find out what is the next node - channelUpdates.get(requestedShortChannelId) match { - case Some(OutgoingChannel(nextNodeId, _, _)) => - log.debug(s"next hop for htlc #{} paymentHash={} is nodeId={}", add.id, add.paymentHash, nextNodeId) - // then we retrieve all known channels to this node - val candidateChannels = node2channels.get(nextNodeId).getOrElse(Set.empty[ShortChannelId]) - // and we filter keep the ones that are compatible with this payment (mainly fees, expiry delta) - candidateChannels - .map { shortChannelId => - val channelInfo_opt = channelUpdates.get(shortChannelId) - val channelUpdate_opt = channelInfo_opt.map(_.channelUpdate) - val relayResult = handleRelay(relayPayload, channelUpdate_opt) - log.debug(s"candidate channel for htlc #${add.id} paymentHash=${add.paymentHash}: shortChannelId={} balanceMsat={} channelUpdate={} relayResult={}", shortChannelId, channelInfo_opt.map(_.availableBalanceMsat).getOrElse(""), channelUpdate_opt.getOrElse(""), relayResult) - (shortChannelId, channelInfo_opt, relayResult) - } - .collect { case (shortChannelId, Some(channelInfo), Right(_)) => (shortChannelId, channelInfo.availableBalanceMsat) } - .filter(_._2 > relayPayload.payload.amtToForward) // we only keep channels that have enough balance to handle this payment - .toList // needed for ordering - .sortBy(_._2) // we want to use the channel with the lowest available balance that can process the payment - .headOption match { - case Some((preferredShortChannelId, availableBalanceMsat)) if preferredShortChannelId != requestedShortChannelId => - log.info("replacing requestedShortChannelId={} by preferredShortChannelId={} with availableBalanceMsat={}", requestedShortChannelId, preferredShortChannelId, availableBalanceMsat) - preferredShortChannelId - case Some(_) => - // the requested short_channel_id is already our preferred channel - requestedShortChannelId - case None => - // no channel seem to work for this payment, we keep the requested channel id - requestedShortChannelId - } - case _ => requestedShortChannelId // we don't have a channel_update for this short_channel_id + private def translateError(failure: AddHtlcFailed): FailureMessage = { + val error = failure.t + val channelUpdate_opt = failure.channelUpdate + (error, channelUpdate_opt) match { + case (_: ExpiryTooSmall, Some(channelUpdate)) => ExpiryTooSoon(channelUpdate) + case (_: ExpiryTooBig, _) => ExpiryTooFar + case (_: InsufficientFunds, Some(channelUpdate)) => TemporaryChannelFailure(channelUpdate) + case (_: TooManyAcceptedHtlcs, Some(channelUpdate)) => TemporaryChannelFailure(channelUpdate) + case (_: ChannelUnavailable, Some(channelUpdate)) if !Announcements.isEnabled(channelUpdate.channelFlags) => ChannelDisabled(channelUpdate.messageFlags, channelUpdate.channelFlags, channelUpdate) + case (_: ChannelUnavailable, None) => PermanentChannelFailure + case (_: HtlcTimedout, _) => PermanentChannelFailure + case _ => TemporaryNodeFailure } } + } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/ChannelSelectionSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/ChannelSelectionSpec.scala index f03fd617b..7a7508ef0 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/ChannelSelectionSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/ChannelSelectionSpec.scala @@ -18,7 +18,7 @@ package fr.acinq.eclair.payment import fr.acinq.bitcoin.Block import fr.acinq.bitcoin.Crypto.PublicKey -import fr.acinq.eclair.channel.{CMD_ADD_HTLC, CMD_FAIL_HTLC} +import fr.acinq.eclair.channel.{AddHtlcFailed, CMD_ADD_HTLC, CMD_FAIL_HTLC} import fr.acinq.eclair.crypto.Sphinx import fr.acinq.eclair.payment.Relayer.{OutgoingChannel, RelayPayload} import fr.acinq.eclair.router.Announcements @@ -37,7 +37,7 @@ class ChannelSelectionSpec extends FunSuite { def dummyUpdate(shortChannelId: ShortChannelId, cltvExpiryDelta: Int, htlcMinimumMsat: Long, feeBaseMsat: Long, feeProportionalMillionths: Long, htlcMaximumMsat: Long, enable: Boolean = true) = Announcements.makeChannelUpdate(Block.RegtestGenesisBlock.hash, randomKey, randomKey.publicKey, shortChannelId, cltvExpiryDelta, htlcMinimumMsat, feeBaseMsat, feeProportionalMillionths, htlcMaximumMsat, enable) - test("handle relay") { + test("convert to CMD_FAIL_HTLC/CMD_ADD_HTLC") { val relayPayload = RelayPayload( add = UpdateAddHtlc(randomBytes32, 42, 1000000, randomBytes32, 70, ByteVector.empty), payload = PerHopPayload(ShortChannelId(12345), amtToForward = 998900, outgoingCltvValue = 60), @@ -49,29 +49,27 @@ class ChannelSelectionSpec extends FunSuite { implicit val log = akka.event.NoLogging // nominal case - assert(Relayer.handleRelay(relayPayload, Some(channelUpdate)) === Right(CMD_ADD_HTLC(relayPayload.payload.amtToForward, relayPayload.add.paymentHash, relayPayload.payload.outgoingCltvValue, relayPayload.nextPacket.serialize, upstream = Right(relayPayload.add), commit = true, redirected = false))) - // redirected to preferred channel - assert(Relayer.handleRelay(relayPayload, Some(channelUpdate.copy(shortChannelId = ShortChannelId(1111)))) === Right(CMD_ADD_HTLC(relayPayload.payload.amtToForward, relayPayload.add.paymentHash, relayPayload.payload.outgoingCltvValue, relayPayload.nextPacket.serialize, upstream = Right(relayPayload.add), commit = true, redirected = true))) + assert(Relayer.relayOrFail(relayPayload, Some(channelUpdate)) === Right((ShortChannelId(12345), CMD_ADD_HTLC(relayPayload.payload.amtToForward, relayPayload.add.paymentHash, relayPayload.payload.outgoingCltvValue, relayPayload.nextPacket.serialize, upstream = Right(relayPayload.add), commit = true)))) // no channel_update - assert(Relayer.handleRelay(relayPayload, channelUpdate_opt = None) === Left(CMD_FAIL_HTLC(relayPayload.add.id, Right(UnknownNextPeer), commit = true))) + assert(Relayer.relayOrFail(relayPayload, channelUpdate_opt = None) === Left(CMD_FAIL_HTLC(relayPayload.add.id, Right(UnknownNextPeer), commit = true))) // channel disabled val channelUpdate_disabled = channelUpdate.copy(channelFlags = Announcements.makeChannelFlags(true, enable = false)) - assert(Relayer.handleRelay(relayPayload, Some(channelUpdate_disabled)) === Left(CMD_FAIL_HTLC(relayPayload.add.id, Right(ChannelDisabled(channelUpdate_disabled.messageFlags, channelUpdate_disabled.channelFlags, channelUpdate_disabled)), commit = true))) + assert(Relayer.relayOrFail(relayPayload, Some(channelUpdate_disabled)) === Left(CMD_FAIL_HTLC(relayPayload.add.id, Right(ChannelDisabled(channelUpdate_disabled.messageFlags, channelUpdate_disabled.channelFlags, channelUpdate_disabled)), commit = true))) // amount too low val relayPayload_toolow = relayPayload.copy(payload = relayPayload.payload.copy(amtToForward = 99)) - assert(Relayer.handleRelay(relayPayload_toolow, Some(channelUpdate)) === Left(CMD_FAIL_HTLC(relayPayload.add.id, Right(AmountBelowMinimum(relayPayload_toolow.payload.amtToForward, channelUpdate)), commit = true))) + assert(Relayer.relayOrFail(relayPayload_toolow, Some(channelUpdate)) === Left(CMD_FAIL_HTLC(relayPayload.add.id, Right(AmountBelowMinimum(relayPayload_toolow.payload.amtToForward, channelUpdate)), commit = true))) // incorrect cltv expiry val relayPayload_incorrectcltv = relayPayload.copy(payload = relayPayload.payload.copy(outgoingCltvValue = 42)) - assert(Relayer.handleRelay(relayPayload_incorrectcltv, Some(channelUpdate)) === Left(CMD_FAIL_HTLC(relayPayload.add.id, Right(IncorrectCltvExpiry(relayPayload_incorrectcltv.payload.outgoingCltvValue, channelUpdate)), commit = true))) + assert(Relayer.relayOrFail(relayPayload_incorrectcltv, Some(channelUpdate)) === Left(CMD_FAIL_HTLC(relayPayload.add.id, Right(IncorrectCltvExpiry(relayPayload_incorrectcltv.payload.outgoingCltvValue, channelUpdate)), commit = true))) // insufficient fee val relayPayload_insufficientfee = relayPayload.copy(payload = relayPayload.payload.copy(amtToForward = 998910)) - assert(Relayer.handleRelay(relayPayload_insufficientfee, Some(channelUpdate)) === Left(CMD_FAIL_HTLC(relayPayload.add.id, Right(FeeInsufficient(relayPayload_insufficientfee.add.amountMsat, channelUpdate)), commit = true))) + assert(Relayer.relayOrFail(relayPayload_insufficientfee, Some(channelUpdate)) === Left(CMD_FAIL_HTLC(relayPayload.add.id, Right(FeeInsufficient(relayPayload_insufficientfee.add.amountMsat, channelUpdate)), commit = true))) // note that a generous fee is ok! val relayPayload_highfee = relayPayload.copy(payload = relayPayload.payload.copy(amtToForward = 900000)) - assert(Relayer.handleRelay(relayPayload_highfee, Some(channelUpdate)) === Right(CMD_ADD_HTLC(relayPayload_highfee.payload.amtToForward, relayPayload_highfee.add.paymentHash, relayPayload_highfee.payload.outgoingCltvValue, relayPayload_highfee.nextPacket.serialize, upstream = Right(relayPayload.add), commit = true, redirected = false))) + assert(Relayer.relayOrFail(relayPayload_highfee, Some(channelUpdate)) === Right((ShortChannelId(12345), CMD_ADD_HTLC(relayPayload_highfee.payload.amtToForward, relayPayload_highfee.add.paymentHash, relayPayload_highfee.payload.outgoingCltvValue, relayPayload_highfee.nextPacket.serialize, upstream = Right(relayPayload.add), commit = true)))) } - test("relay channel selection") { + test("channel selection") { val relayPayload = RelayPayload( add = UpdateAddHtlc(randomBytes32, 42, 1000000, randomBytes32, 70, ByteVector.empty), @@ -99,15 +97,21 @@ class ChannelSelectionSpec extends FunSuite { import com.softwaremill.quicklens._ // select the channel to the same node, with the lowest balance but still high enough to handle the payment - assert(Relayer.selectPreferredChannel(relayPayload, channelUpdates, node2channels) === ShortChannelId(22222)) + assert(Relayer.selectPreferredChannel(relayPayload, channelUpdates, node2channels, Seq.empty) === Some(ShortChannelId(22222))) + // select 2nd-to-best channel + assert(Relayer.selectPreferredChannel(relayPayload, channelUpdates, node2channels, Seq(ShortChannelId(22222))) === Some(ShortChannelId(12345))) + // select 3rd-to-best channel + assert(Relayer.selectPreferredChannel(relayPayload, channelUpdates, node2channels, Seq(ShortChannelId(22222), ShortChannelId(12345))) === Some(ShortChannelId(11111))) + // all the suitable channels have been tried + assert(Relayer.selectPreferredChannel(relayPayload, channelUpdates, node2channels, Seq(ShortChannelId(22222), ShortChannelId(12345), ShortChannelId(11111))) === None) // higher amount payment (have to increased incoming htlc amount for fees to be sufficient) - assert(Relayer.selectPreferredChannel(relayPayload.modify(_.add.amountMsat).setTo(60000000).modify(_.payload.amtToForward).setTo(50000000), channelUpdates, node2channels) === ShortChannelId(11111)) + assert(Relayer.selectPreferredChannel(relayPayload.modify(_.add.amountMsat).setTo(60000000).modify(_.payload.amtToForward).setTo(50000000), channelUpdates, node2channels, Seq.empty) === Some(ShortChannelId(11111))) // lower amount payment - assert(Relayer.selectPreferredChannel(relayPayload.modify(_.payload.amtToForward).setTo(1000), channelUpdates, node2channels) === ShortChannelId(33333)) - // payment too high, no suitable channel, we keep the requested one - assert(Relayer.selectPreferredChannel(relayPayload.modify(_.payload.amtToForward).setTo(1000000000), channelUpdates, node2channels) === ShortChannelId(12345)) + assert(Relayer.selectPreferredChannel(relayPayload.modify(_.payload.amtToForward).setTo(1000), channelUpdates, node2channels, Seq.empty) === Some(ShortChannelId(33333))) + // payment too high, no suitable channel found + assert(Relayer.selectPreferredChannel(relayPayload.modify(_.payload.amtToForward).setTo(1000000000), channelUpdates, node2channels, Seq.empty) === Some(ShortChannelId(12345))) // invalid cltv expiry, no suitable channel, we keep the requested one - assert(Relayer.selectPreferredChannel(relayPayload.modify(_.payload.outgoingCltvValue).setTo(40), channelUpdates, node2channels) === ShortChannelId(12345)) + assert(Relayer.selectPreferredChannel(relayPayload.modify(_.payload.outgoingCltvValue).setTo(40), channelUpdates, node2channels, Seq.empty) === Some(ShortChannelId(12345))) } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/RelayerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/RelayerSpec.scala index 77c0fe550..a71efb14e 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/RelayerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/RelayerSpec.scala @@ -27,7 +27,7 @@ import fr.acinq.eclair.payment.PaymentLifecycle.buildCommand import fr.acinq.eclair.router.Announcements import fr.acinq.eclair.transactions.CommitmentSpec import fr.acinq.eclair.wire._ -import fr.acinq.eclair.{TestConstants, TestkitBaseClass, randomBytes32, randomKey} +import fr.acinq.eclair.{ShortChannelId, TestConstants, TestkitBaseClass, UInt64, randomBytes32, randomKey} import org.scalatest.Outcome import scodec.bits.ByteVector @@ -58,10 +58,9 @@ class RelayerSpec extends TestkitBaseClass { val channelId_ab = randomBytes32 val channelId_bc = randomBytes32 - def makeCommitments(channelId: ByteVector32) = new Commitments(null, null, 0.toByte, null, - RemoteCommit(42, CommitmentSpec(Set.empty, 20000, 5000000, 100000000), ByteVector32.Zeroes, randomKey.toPoint), + def makeCommitments(channelId: ByteVector32, availableBalanceMsat: Long = 50000000L) = new Commitments(null, null, 0.toByte, null, null, null, null, 0, 0, Map.empty, null, null, null, channelId) { - override def availableBalanceForSendMsat: Long = remoteCommit.spec.toRemoteMsat // approximation + override def availableBalanceForSendMsat: Long = availableBalanceMsat } test("relay an htlc-add") { f => @@ -84,6 +83,51 @@ class RelayerSpec extends TestkitBaseClass { paymentHandler.expectNoMsg(100 millis) } + test("relay an htlc-add with retries") { f => + import f._ + val sender = TestProbe() + + // we use this to build a valid onion + val (cmd, _) = buildCommand(UUID.randomUUID(), finalAmountMsat, finalExpiry, paymentHash, hops) + // and then manually build an htlc + val add_ab = UpdateAddHtlc(channelId = channelId_ab, id = 123456, cmd.amountMsat, cmd.paymentHash, cmd.cltvExpiry, cmd.onion) + + // we tell the relayer about channel B-C + relayer ! LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, makeCommitments(channelId_bc)) + + // this is another channel B-C, with less balance (it will be preferred) + val (channelId_bc_1, channelUpdate_bc_1) = (randomBytes32, channelUpdate_bc.copy(shortChannelId = ShortChannelId("500000x1x1"))) + relayer ! LocalChannelUpdate(null, channelId_bc_1, channelUpdate_bc_1.shortChannelId, c, None, channelUpdate_bc_1, makeCommitments(channelId_bc_1, availableBalanceMsat = 49000000L)) + + sender.send(relayer, ForwardAdd(add_ab)) + + // first try + val fwd1 = register.expectMsgType[Register.ForwardShortId[CMD_ADD_HTLC]] + assert(fwd1.shortChannelId === channelUpdate_bc_1.shortChannelId) + assert(fwd1.message.upstream === Right(add_ab)) + + // channel returns an error + val origin = Relayed(channelId_ab, originHtlcId = 42, amountMsatIn = 1100000, amountMsatOut = 1000000) + sender.send(relayer, Status.Failure(AddHtlcFailed(channelId_bc_1, paymentHash, HtlcValueTooHighInFlight(channelId_bc_1, UInt64(1000000000L), UInt64(1516977616L)), origin, Some(channelUpdate_bc_1), originalCommand = Some(fwd1.message)))) + + // second try + val fwd2 = register.expectMsgType[Register.ForwardShortId[CMD_ADD_HTLC]] + assert(fwd2.shortChannelId === channelUpdate_bc.shortChannelId) + assert(fwd2.message.upstream === Right(add_ab)) + + // failure again + sender.send(relayer, Status.Failure(AddHtlcFailed(channelId_bc, paymentHash, HtlcValueTooHighInFlight(channelId_bc, UInt64(1000000000L), UInt64(1516977616L)), origin, Some(channelUpdate_bc), originalCommand = Some(fwd2.message)))) + + // the relayer should give up + val fwdFail = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]] + assert(fwdFail.channelId === add_ab.channelId) + assert(fwdFail.message.id === add_ab.id) + assert(fwdFail.message.reason === Right(TemporaryNodeFailure)) + + sender.expectNoMsg(100 millis) + paymentHandler.expectNoMsg(100 millis) + } + test("fail to relay an htlc-add when we have no channel_update for the next channel") { f => import f._ val sender = TestProbe() From f180e54b031307c1d4b6206395a6d4586e3eca07 Mon Sep 17 00:00:00 2001 From: Pierre-Marie Padiou Date: Tue, 11 Jun 2019 14:31:23 +0200 Subject: [PATCH 02/21] Close channel if funding tx times out (#1016) This only happens when we are fundee. We *could* have some funds at stake if there was a non-zero `push_msat`, but we already allows 5 days for the funding tx to confirm so the best option is probably to forget about the channel. --- .../src/main/scala/fr/acinq/eclair/channel/Channel.scala | 4 +--- .../src/main/scala/fr/acinq/eclair/channel/ChannelTypes.scala | 1 - .../channel/states/c/WaitForFundingConfirmedStateSpec.scala | 2 +- 3 files changed, 2 insertions(+), 5 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 7374b9ac2..5beac4d1c 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 @@ -1561,8 +1561,6 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId when(ERR_INFORMATION_LEAK)(errorStateHandler) - when(ERR_FUNDING_TIMEOUT)(errorStateHandler) - when(ERR_FUNDING_LOST)(errorStateHandler) whenUnhandled { @@ -1728,7 +1726,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId val exc = FundingTxTimedout(d.channelId) val error = Error(d.channelId, exc.getMessage) context.system.eventStream.publish(ChannelErrorOccured(self, Helpers.getChannelId(stateData), remoteNodeId, stateData, LocalError(exc), isFatal = true)) - goto(ERR_FUNDING_TIMEOUT) sending error + goto(CLOSED) sending error } def handleRevocationTimeout(revocationTimeout: RevocationTimeout, d: HasCommitments) = { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelTypes.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelTypes.scala index 0f451a91b..e538eceea 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelTypes.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelTypes.scala @@ -63,7 +63,6 @@ case object OFFLINE extends State case object SYNCING extends State case object WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT extends State case object ERR_FUNDING_LOST extends State -case object ERR_FUNDING_TIMEOUT extends State case object ERR_INFORMATION_LEAK extends State /* diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForFundingConfirmedStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForFundingConfirmedStateSpec.scala index 694d9dc7e..48d3a4f24 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForFundingConfirmedStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForFundingConfirmedStateSpec.scala @@ -89,7 +89,7 @@ class WaitForFundingConfirmedStateSpec extends TestkitBaseClass with StateTestsH import f._ alice ! BITCOIN_FUNDING_TIMEOUT alice2bob.expectMsgType[Error] - awaitCond(alice.stateName == ERR_FUNDING_TIMEOUT) + awaitCond(alice.stateName == CLOSED) } test("recv BITCOIN_FUNDING_SPENT (remote commit)") { f => From 9087ceeb5fd79c14cce3d769169ac7e9121f0707 Mon Sep 17 00:00:00 2001 From: araspitzu Date: Tue, 11 Jun 2019 15:34:47 +0200 Subject: [PATCH 03/21] Add feeThresholdSat and maxFeePct parameters to payments API (#1023) --- .../main/scala/fr/acinq/eclair/Eclair.scala | 51 +++++++++++-------- .../scala/fr/acinq/eclair/api/Service.scala | 14 ++--- .../scala/fr/acinq/eclair/router/Router.scala | 32 ++++++------ .../fr/acinq/eclair/EclairImplSpec.scala | 19 +++++-- .../fr/acinq/eclair/api/ApiServiceSpec.scala | 8 +-- 5 files changed, 72 insertions(+), 52 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala b/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala index 39d290c8a..6f9c9e838 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala @@ -17,6 +17,7 @@ package fr.acinq.eclair import java.util.UUID + import akka.actor.ActorRef import akka.pattern._ import akka.util.Timeout @@ -28,11 +29,10 @@ import fr.acinq.eclair.db.{IncomingPayment, NetworkFee, OutgoingPayment, Stats} import fr.acinq.eclair.io.Peer.{GetPeerInfo, PeerInfo} import fr.acinq.eclair.io.{NodeURI, Peer} import fr.acinq.eclair.payment.PaymentLifecycle._ -import fr.acinq.eclair.router.{ChannelDesc, RouteRequest, RouteResponse} +import fr.acinq.eclair.router.{ChannelDesc, RouteRequest, RouteResponse, Router} import scodec.bits.ByteVector import scala.concurrent.Future import scala.concurrent.duration._ -import scala.util.{Failure, Success, Try} import fr.acinq.eclair.payment.{PaymentReceived, PaymentRelayed, PaymentRequest, PaymentSent} import fr.acinq.eclair.wire.{ChannelAnnouncement, ChannelUpdate, NodeAddress, NodeAnnouncement} import TimestampQueryFilters._ @@ -57,25 +57,25 @@ trait Eclair { def connect(uri: String)(implicit timeout: Timeout): Future[String] - def open(nodeId: PublicKey, fundingSatoshis: Long, pushMsat: Option[Long], fundingFeerateSatByte: Option[Long], flags: Option[Int], openTimeout_opt: Option[Timeout])(implicit timeout: Timeout): Future[String] + def open(nodeId: PublicKey, fundingSatoshis: Long, pushMsat_opt: Option[Long], fundingFeerateSatByte_opt: Option[Long], flags_opt: Option[Int], openTimeout_opt: Option[Timeout])(implicit timeout: Timeout): Future[String] - def close(channelIdentifier: Either[ByteVector32, ShortChannelId], scriptPubKey: Option[ByteVector])(implicit timeout: Timeout): Future[String] + def close(channelIdentifier: Either[ByteVector32, ShortChannelId], scriptPubKey_opt: Option[ByteVector])(implicit timeout: Timeout): Future[String] def forceClose(channelIdentifier: Either[ByteVector32, ShortChannelId])(implicit timeout: Timeout): Future[String] def updateRelayFee(channelIdentifier: Either[ByteVector32, ShortChannelId], feeBaseMsat: Long, feeProportionalMillionths: Long)(implicit timeout: Timeout): Future[String] - def channelsInfo(toRemoteNode: Option[PublicKey])(implicit timeout: Timeout): Future[Iterable[RES_GETINFO]] + def channelsInfo(toRemoteNode_opt: Option[PublicKey])(implicit timeout: Timeout): Future[Iterable[RES_GETINFO]] def channelInfo(channelIdentifier: Either[ByteVector32, ShortChannelId])(implicit timeout: Timeout): Future[RES_GETINFO] def peersInfo()(implicit timeout: Timeout): Future[Iterable[PeerInfo]] - def receive(description: String, amountMsat: Option[Long], expire: Option[Long], fallbackAddress: Option[String], paymentPreimage: Option[ByteVector32])(implicit timeout: Timeout): Future[PaymentRequest] + def receive(description: String, amountMsat_opt: Option[Long], expire_opt: Option[Long], fallbackAddress_opt: Option[String], paymentPreimage_opt: Option[ByteVector32])(implicit timeout: Timeout): Future[PaymentRequest] def receivedInfo(paymentHash: ByteVector32)(implicit timeout: Timeout): Future[Option[IncomingPayment]] - def send(recipientNodeId: PublicKey, amountMsat: Long, paymentHash: ByteVector32, assistedRoutes: Seq[Seq[PaymentRequest.ExtraHop]] = Seq.empty, minFinalCltvExpiry: Option[Long] = None, maxAttempts: Option[Int] = None)(implicit timeout: Timeout): Future[UUID] + def send(recipientNodeId: PublicKey, amountMsat: Long, paymentHash: ByteVector32, assistedRoutes: Seq[Seq[PaymentRequest.ExtraHop]] = Seq.empty, minFinalCltvExpiry_opt: Option[Long] = None, maxAttempts_opt: Option[Int] = None, feeThresholdSat_opt: Option[Long] = None, maxFeePct_opt: Option[Double] = None)(implicit timeout: Timeout): Future[UUID] def sentInfo(id: Either[UUID, ByteVector32])(implicit timeout: Timeout): Future[Seq[OutgoingPayment]] @@ -99,7 +99,7 @@ trait Eclair { def allChannels()(implicit timeout: Timeout): Future[Iterable[ChannelDesc]] - def allUpdates(nodeId: Option[PublicKey])(implicit timeout: Timeout): Future[Iterable[ChannelUpdate]] + def allUpdates(nodeId_opt: Option[PublicKey])(implicit timeout: Timeout): Future[Iterable[ChannelUpdate]] def getInfoResponse()(implicit timeout: Timeout): Future[GetInfoResponse] @@ -113,20 +113,20 @@ class EclairImpl(appKit: Kit) extends Eclair { (appKit.switchboard ? Peer.Connect(NodeURI.parse(uri))).mapTo[String] } - override def open(nodeId: PublicKey, fundingSatoshis: Long, pushMsat: Option[Long], fundingFeerateSatByte: Option[Long], flags: Option[Int], openTimeout_opt: Option[Timeout])(implicit timeout: Timeout): Future[String] = { + override def open(nodeId: PublicKey, fundingSatoshis: Long, pushMsat_opt: Option[Long], fundingFeerateSatByte_opt: Option[Long], flags_opt: Option[Int], openTimeout_opt: Option[Timeout])(implicit timeout: Timeout): Future[String] = { // we want the open timeout to expire *before* the default ask timeout, otherwise user won't get a generic response val openTimeout = openTimeout_opt.getOrElse(Timeout(10 seconds)) (appKit.switchboard ? Peer.OpenChannel( remoteNodeId = nodeId, fundingSatoshis = Satoshi(fundingSatoshis), - pushMsat = pushMsat.map(MilliSatoshi).getOrElse(MilliSatoshi(0)), - fundingTxFeeratePerKw_opt = fundingFeerateSatByte.map(feerateByte2Kw), - channelFlags = flags.map(_.toByte), + pushMsat = pushMsat_opt.map(MilliSatoshi).getOrElse(MilliSatoshi(0)), + fundingTxFeeratePerKw_opt = fundingFeerateSatByte_opt.map(feerateByte2Kw), + channelFlags = flags_opt.map(_.toByte), timeout_opt = Some(openTimeout))).mapTo[String] } - override def close(channelIdentifier: Either[ByteVector32, ShortChannelId], scriptPubKey: Option[ByteVector])(implicit timeout: Timeout): Future[String] = { - sendToChannel(channelIdentifier, CMD_CLOSE(scriptPubKey)).mapTo[String] + override def close(channelIdentifier: Either[ByteVector32, ShortChannelId], scriptPubKey_opt: Option[ByteVector])(implicit timeout: Timeout): Future[String] = { + sendToChannel(channelIdentifier, CMD_CLOSE(scriptPubKey_opt)).mapTo[String] } override def forceClose(channelIdentifier: Either[ByteVector32, ShortChannelId])(implicit timeout: Timeout): Future[String] = { @@ -142,7 +142,7 @@ class EclairImpl(appKit: Kit) extends Eclair { peerinfos <- Future.sequence(peers.map(peer => (peer ? GetPeerInfo).mapTo[PeerInfo])) } yield peerinfos - override def channelsInfo(toRemoteNode: Option[PublicKey])(implicit timeout: Timeout): Future[Iterable[RES_GETINFO]] = toRemoteNode match { + override def channelsInfo(toRemoteNode_opt: Option[PublicKey])(implicit timeout: Timeout): Future[Iterable[RES_GETINFO]] = toRemoteNode_opt match { case Some(pk) => for { channelIds <- (appKit.register ? 'channelsTo).mapTo[Map[ByteVector32, PublicKey]].map(_.filter(_._2 == pk).keys) channels <- Future.sequence(channelIds.map(channelId => sendToChannel(Left(channelId), CMD_GETINFO).mapTo[RES_GETINFO])) @@ -163,14 +163,14 @@ class EclairImpl(appKit: Kit) extends Eclair { (appKit.router ? 'channels).mapTo[Iterable[ChannelAnnouncement]].map(_.map(c => ChannelDesc(c.shortChannelId, c.nodeId1, c.nodeId2))) } - override def allUpdates(nodeId: Option[PublicKey])(implicit timeout: Timeout): Future[Iterable[ChannelUpdate]] = nodeId match { + override def allUpdates(nodeId_opt: Option[PublicKey])(implicit timeout: Timeout): Future[Iterable[ChannelUpdate]] = nodeId_opt match { case None => (appKit.router ? 'updates).mapTo[Iterable[ChannelUpdate]] case Some(pk) => (appKit.router ? 'updatesMap).mapTo[Map[ChannelDesc, ChannelUpdate]].map(_.filter(e => e._1.a == pk || e._1.b == pk).values) } - override def receive(description: String, amountMsat: Option[Long], expire: Option[Long], fallbackAddress: Option[String], paymentPreimage: Option[ByteVector32])(implicit timeout: Timeout): Future[PaymentRequest] = { - fallbackAddress.map { fa => fr.acinq.eclair.addressToPublicKeyScript(fa, appKit.nodeParams.chainHash) } // if it's not a bitcoin address throws an exception - (appKit.paymentHandler ? ReceivePayment(description = description, amountMsat_opt = amountMsat.map(MilliSatoshi), expirySeconds_opt = expire, fallbackAddress = fallbackAddress, paymentPreimage = paymentPreimage)).mapTo[PaymentRequest] + override def receive(description: String, amountMsat_opt: Option[Long], expire_opt: Option[Long], fallbackAddress_opt: Option[String], paymentPreimage_opt: Option[ByteVector32])(implicit timeout: Timeout): Future[PaymentRequest] = { + fallbackAddress_opt.map { fa => fr.acinq.eclair.addressToPublicKeyScript(fa, appKit.nodeParams.chainHash) } // if it's not a bitcoin address throws an exception + (appKit.paymentHandler ? ReceivePayment(description = description, amountMsat_opt = amountMsat_opt.map(MilliSatoshi), expirySeconds_opt = expire_opt, fallbackAddress = fallbackAddress_opt, paymentPreimage = paymentPreimage_opt)).mapTo[PaymentRequest] } override def findRoute(targetNodeId: PublicKey, amountMsat: Long, assistedRoutes: Seq[Seq[PaymentRequest.ExtraHop]] = Seq.empty)(implicit timeout: Timeout): Future[RouteResponse] = { @@ -181,11 +181,18 @@ class EclairImpl(appKit: Kit) extends Eclair { (appKit.paymentInitiator ? SendPaymentToRoute(amountMsat, paymentHash, route, finalCltvExpiry)).mapTo[UUID] } - override def send(recipientNodeId: PublicKey, amountMsat: Long, paymentHash: ByteVector32, assistedRoutes: Seq[Seq[PaymentRequest.ExtraHop]] = Seq.empty, minFinalCltvExpiry_opt: Option[Long] = None, maxAttempts_opt: Option[Int] = None)(implicit timeout: Timeout): Future[UUID] = { + override def send(recipientNodeId: PublicKey, amountMsat: Long, paymentHash: ByteVector32, assistedRoutes: Seq[Seq[PaymentRequest.ExtraHop]] = Seq.empty, minFinalCltvExpiry_opt: Option[Long], maxAttempts_opt: Option[Int], feeThresholdSat_opt: Option[Long], maxFeePct_opt: Option[Double])(implicit timeout: Timeout): Future[UUID] = { val maxAttempts = maxAttempts_opt.getOrElse(appKit.nodeParams.maxPaymentAttempts) + + val defaultRouteParams = Router.getDefaultRouteParams(appKit.nodeParams.routerConf) + val routeParams = defaultRouteParams.copy( + maxFeePct = maxFeePct_opt.getOrElse(defaultRouteParams.maxFeePct), + maxFeeBaseMsat = feeThresholdSat_opt.map(_ * 1000).getOrElse(defaultRouteParams.maxFeeBaseMsat) + ) + val sendPayment = minFinalCltvExpiry_opt match { - case Some(minCltv) => SendPayment(amountMsat, paymentHash, recipientNodeId, assistedRoutes, finalCltvExpiry = minCltv, maxAttempts = maxAttempts) - case None => SendPayment(amountMsat, paymentHash, recipientNodeId, assistedRoutes, maxAttempts = maxAttempts) + case Some(minCltv) => SendPayment(amountMsat, paymentHash, recipientNodeId, assistedRoutes, finalCltvExpiry = minCltv, maxAttempts = maxAttempts, routeParams = Some(routeParams)) + case None => SendPayment(amountMsat, paymentHash, recipientNodeId, assistedRoutes, maxAttempts = maxAttempts, routeParams = Some(routeParams)) } (appKit.paymentInitiator ? sendPayment).mapTo[UUID] } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/api/Service.scala b/eclair-core/src/main/scala/fr/acinq/eclair/api/Service.scala index 8c50e4e7c..b8d633ada 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/api/Service.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/api/Service.scala @@ -208,17 +208,17 @@ trait Service extends ExtraDirectives with Logging { } } ~ path("payinvoice") { - formFields(invoiceFormParam, amountMsatFormParam.?, "maxAttempts".as[Int].?) { - case (invoice@PaymentRequest(_, Some(amount), _, nodeId, _, _), None, maxAttempts) => - complete(eclairApi.send(nodeId, amount.toLong, invoice.paymentHash, invoice.routingInfo, invoice.minFinalCltvExpiry, maxAttempts)) - case (invoice, Some(overrideAmount), maxAttempts) => - complete(eclairApi.send(invoice.nodeId, overrideAmount, invoice.paymentHash, invoice.routingInfo, invoice.minFinalCltvExpiry, maxAttempts)) + formFields(invoiceFormParam, amountMsatFormParam.?, "maxAttempts".as[Int].?, "feeThresholdSat".as[Long].?, "maxFeePct".as[Double].?) { + case (invoice@PaymentRequest(_, Some(amount), _, nodeId, _, _), None, maxAttempts, feeThresholdSat_opt, maxFeePct_opt) => + complete(eclairApi.send(nodeId, amount.toLong, invoice.paymentHash, invoice.routingInfo, invoice.minFinalCltvExpiry, maxAttempts, feeThresholdSat_opt, maxFeePct_opt)) + case (invoice, Some(overrideAmount), maxAttempts, feeThresholdSat_opt, maxFeePct_opt) => + complete(eclairApi.send(invoice.nodeId, overrideAmount, invoice.paymentHash, invoice.routingInfo, invoice.minFinalCltvExpiry, maxAttempts, feeThresholdSat_opt, maxFeePct_opt)) case _ => reject(MalformedFormFieldRejection("invoice", "The invoice must have an amount or you need to specify one using the field 'amountMsat'")) } } ~ path("sendtonode") { - formFields(amountMsatFormParam, paymentHashFormParam, nodeIdFormParam, "maxAttempts".as[Int].?) { (amountMsat, paymentHash, nodeId, maxAttempts) => - complete(eclairApi.send(nodeId, amountMsat, paymentHash, maxAttempts = maxAttempts)) + formFields(amountMsatFormParam, paymentHashFormParam, nodeIdFormParam, "maxAttempts".as[Int].?, "feeThresholdSat".as[Long].?, "maxFeePct".as[Double].?) { (amountMsat, paymentHash, nodeId, maxAttempts_opt, feeThresholdSat_opt, maxFeePct_opt) => + complete(eclairApi.send(nodeId, amountMsat, paymentHash, maxAttempts_opt = maxAttempts_opt, feeThresholdSat_opt = feeThresholdSat_opt, maxFeePct_opt = maxFeePct_opt)) } } ~ path("sendtoroute") { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/router/Router.scala b/eclair-core/src/main/scala/fr/acinq/eclair/router/Router.scala index 98137c4b8..779919502 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/router/Router.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/router/Router.scala @@ -121,21 +121,7 @@ class Router(nodeParams: NodeParams, watcher: ActorRef, initialized: Option[Prom val SHORTID_WINDOW = 100 - val defaultRouteParams = RouteParams( - randomize = nodeParams.routerConf.randomizeRouteSelection, - maxFeeBaseMsat = nodeParams.routerConf.searchMaxFeeBaseSat * 1000, // converting sat -> msat - maxFeePct = nodeParams.routerConf.searchMaxFeePct, - routeMaxLength = nodeParams.routerConf.searchMaxRouteLength, - routeMaxCltv = nodeParams.routerConf.searchMaxCltv, - ratios = nodeParams.routerConf.searchHeuristicsEnabled match { - case false => None - case true => Some(WeightRatios( - cltvDeltaFactor = nodeParams.routerConf.searchRatioCltv, - ageFactor = nodeParams.routerConf.searchRatioChannelAge, - capacityFactor = nodeParams.routerConf.searchRatioChannelCapacity - )) - } - ) + val defaultRouteParams = getDefaultRouteParams(nodeParams.routerConf) val db = nodeParams.db.network @@ -839,6 +825,22 @@ object Router { // The default amount of routes we'll search for when findRoute is called val DEFAULT_ROUTES_COUNT = 3 + def getDefaultRouteParams(routerConf: RouterConf) = RouteParams( + randomize = routerConf.randomizeRouteSelection, + maxFeeBaseMsat = routerConf.searchMaxFeeBaseSat * 1000, // converting sat -> msat + maxFeePct = routerConf.searchMaxFeePct, + routeMaxLength = routerConf.searchMaxRouteLength, + routeMaxCltv = routerConf.searchMaxCltv, + ratios = routerConf.searchHeuristicsEnabled match { + case false => None + case true => Some(WeightRatios( + cltvDeltaFactor = routerConf.searchRatioCltv, + ageFactor = routerConf.searchRatioChannelAge, + capacityFactor = routerConf.searchRatioChannelCapacity + )) + } + ) + /** * Find a route in the graph between localNodeId and targetNodeId, returns the route. * Will perform a k-shortest path selection given the @param numRoutes and randomly select one of the result. diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/EclairImplSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/EclairImplSpec.scala index 453cf5bb7..2ab1b0b32 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/EclairImplSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/EclairImplSpec.scala @@ -84,12 +84,12 @@ class EclairImplSpec extends TestKit(ActorSystem("mySystem")) with fixture.FunSu val nodeId = PublicKey(hex"030bb6a5e0c6b203c7e2180fb78c7ba4bdce46126761d8201b91ddac089cdecc87") // standard conversion - eclair.open(nodeId, fundingSatoshis = 10000000L, pushMsat = None, fundingFeerateSatByte = Some(5), flags = None, openTimeout_opt = None) + eclair.open(nodeId, fundingSatoshis = 10000000L, pushMsat_opt = None, fundingFeerateSatByte_opt = Some(5), flags_opt = None, openTimeout_opt = None) val open = switchboard.expectMsgType[OpenChannel] assert(open.fundingTxFeeratePerKw_opt == Some(1250)) // check that minimum fee rate of 253 sat/bw is used - eclair.open(nodeId, fundingSatoshis = 10000000L, pushMsat = None, fundingFeerateSatByte = Some(1), flags = None, openTimeout_opt = None) + eclair.open(nodeId, fundingSatoshis = 10000000L, pushMsat_opt = None, fundingFeerateSatByte_opt = Some(1), flags_opt = None, openTimeout_opt = None) val open1 = switchboard.expectMsgType[OpenChannel] assert(open1.fundingTxFeeratePerKw_opt == Some(MinimumFeeratePerKw)) } @@ -116,13 +116,22 @@ class EclairImplSpec extends TestKit(ActorSystem("mySystem")) with fixture.FunSu assert(send1.paymentHash == ByteVector32.Zeroes) assert(send1.assistedRoutes == hints) - // with finalCltvExpiry and failures response + // with finalCltvExpiry eclair.send(recipientNodeId = nodeId, amountMsat = 123, paymentHash = ByteVector32.Zeroes, assistedRoutes = Seq.empty, minFinalCltvExpiry_opt = Some(96)) val send2 = paymentInitiator.expectMsgType[SendPayment] assert(send2.targetNodeId == nodeId) assert(send2.amountMsat == 123) assert(send2.paymentHash == ByteVector32.Zeroes) assert(send2.finalCltvExpiry == 96) + + // with custom route fees parameters + eclair.send(recipientNodeId = nodeId, amountMsat = 123, paymentHash = ByteVector32.Zeroes, assistedRoutes = Seq.empty, minFinalCltvExpiry_opt = None, feeThresholdSat_opt = Some(123), maxFeePct_opt = Some(4.20)) + val send3 = paymentInitiator.expectMsgType[SendPayment] + assert(send3.targetNodeId == nodeId) + assert(send3.amountMsat == 123) + assert(send3.paymentHash == ByteVector32.Zeroes) + assert(send3.routeParams.get.maxFeeBaseMsat == 123 * 1000) // conversion sat -> msat + assert(send3.routeParams.get.maxFeePct == 4.20) } test("allupdates can filter by nodeId") { f => @@ -197,7 +206,7 @@ class EclairImplSpec extends TestKit(ActorSystem("mySystem")) with fixture.FunSu val eclair = new EclairImpl(kitWithPaymentHandler) val paymentPreimage = randomBytes32 - val fResp = eclair.receive(description = "some desc", amountMsat = None, expire = None, fallbackAddress = None, paymentPreimage = Some(paymentPreimage)) + val fResp = eclair.receive(description = "some desc", amountMsat_opt = None, expire_opt = None, fallbackAddress_opt = None, paymentPreimage_opt = Some(paymentPreimage)) awaitCond({ fResp.value match { case Some(Success(pr)) => pr.paymentHash == Crypto.sha256(paymentPreimage) @@ -252,4 +261,6 @@ class EclairImplSpec extends TestKit(ActorSystem("mySystem")) with fixture.FunSu assert(send.finalCltvExpiry == 123) assert(send.paymentHash == ByteVector32.One) } + + } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala index 93346e220..60c79cd29 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala @@ -238,7 +238,7 @@ class ApiServiceSpec extends FunSuite with ScalatestRouteTest with IdiomaticMock val invoice = "lnbc12580n1pw2ywztpp554ganw404sh4yjkwnysgn3wjcxfcq7gtx53gxczkjr9nlpc3hzvqdq2wpskwctddyxqr4rqrzjqwryaup9lh50kkranzgcdnn2fgvx390wgj5jd07rwr3vxeje0glc7z9rtvqqwngqqqqqqqlgqqqqqeqqjqrrt8smgjvfj7sg38dwtr9kc9gg3era9k3t2hvq3cup0jvsrtrxuplevqgfhd3rzvhulgcxj97yjuj8gdx8mllwj4wzjd8gdjhpz3lpqqvk2plh" val eclair = mock[Eclair] - eclair.send(any, any, any, any, any, any)(any[Timeout]) returns Future.successful(UUID.randomUUID()) + eclair.send(any, any, any, any, any, any, any, any)(any[Timeout]) returns Future.successful(UUID.randomUUID()) val mockService = new MockService(eclair) Post("/payinvoice", FormData("invoice" -> invoice).toEntity) ~> @@ -247,17 +247,17 @@ class ApiServiceSpec extends FunSuite with ScalatestRouteTest with IdiomaticMock check { assert(handled) assert(status == OK) - eclair.send(any, 1258000, any, any, any, any)(any[Timeout]).wasCalled(once) + eclair.send(any, 1258000, any, any, any, any, any, any)(any[Timeout]).wasCalled(once) } - Post("/payinvoice", FormData("invoice" -> invoice, "amountMsat" -> "123").toEntity) ~> + Post("/payinvoice", FormData("invoice" -> invoice, "amountMsat" -> "123", "feeThresholdSat" -> "112233", "maxFeePct" -> "2.34").toEntity) ~> addCredentials(BasicHttpCredentials("", mockService.password)) ~> Route.seal(mockService.route) ~> check { assert(handled) assert(status == OK) - eclair.send(any, 123, any, any, any, any)(any[Timeout]).wasCalled(once) + eclair.send(any, 123, any, any, any, any, Some(112233), Some(2.34))(any[Timeout]).wasCalled(once) } } From 818199e8f4569b888d7bc06cb78e88e7fe220b1f Mon Sep 17 00:00:00 2001 From: araspitzu Date: Tue, 11 Jun 2019 17:39:52 +0200 Subject: [PATCH 04/21] Peer reconnection address from node announcements (#1009) * Use node announcements as fallback to load peer addresses during startup * Add NetworkDb.getNode to retrieve a node_announcement by nodeId * When connecting to a peer use node_announcement as fallback for its IP address * Support connection to peer via pubKey * Increase finite max of exponential backoff time to 1h. * Add peer disconnect API call --- .../main/scala/fr/acinq/eclair/Eclair.scala | 15 ++- .../eclair/api/FormParamExtractors.scala | 6 +- .../scala/fr/acinq/eclair/api/Service.scala | 15 ++- .../scala/fr/acinq/eclair/db/NetworkDb.scala | 2 + .../eclair/db/sqlite/SqliteNetworkDb.scala | 8 ++ .../main/scala/fr/acinq/eclair/io/Peer.scala | 72 +++++++++----- .../fr/acinq/eclair/io/Switchboard.scala | 16 ++- .../fr/acinq/eclair/TestkitBaseClass.scala | 3 +- .../fr/acinq/eclair/api/ApiServiceSpec.scala | 19 ++-- .../acinq/eclair/db/SqliteNetworkDbSpec.scala | 1 + .../eclair/integration/IntegrationSpec.scala | 9 +- .../scala/fr/acinq/eclair/io/PeerSpec.scala | 97 ++++++++++++++++--- .../acinq/eclair/io/PeerSpecWithLogging.scala | 43 ++++++++ 13 files changed, 248 insertions(+), 58 deletions(-) create mode 100644 eclair-core/src/test/scala/fr/acinq/eclair/io/PeerSpecWithLogging.scala diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala b/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala index 6f9c9e838..7c425328c 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala @@ -27,7 +27,7 @@ import fr.acinq.eclair.channel.Register.{Forward, ForwardShortId} import fr.acinq.eclair.channel._ import fr.acinq.eclair.db.{IncomingPayment, NetworkFee, OutgoingPayment, Stats} import fr.acinq.eclair.io.Peer.{GetPeerInfo, PeerInfo} -import fr.acinq.eclair.io.{NodeURI, Peer} +import fr.acinq.eclair.io.{NodeURI, Peer, Switchboard} import fr.acinq.eclair.payment.PaymentLifecycle._ import fr.acinq.eclair.router.{ChannelDesc, RouteRequest, RouteResponse, Router} import scodec.bits.ByteVector @@ -55,7 +55,9 @@ object TimestampQueryFilters { trait Eclair { - def connect(uri: String)(implicit timeout: Timeout): Future[String] + def connect(target: Either[NodeURI, PublicKey])(implicit timeout: Timeout): Future[String] + + def disconnect(nodeId: PublicKey)(implicit timeout: Timeout): Future[String] def open(nodeId: PublicKey, fundingSatoshis: Long, pushMsat_opt: Option[Long], fundingFeerateSatByte_opt: Option[Long], flags_opt: Option[Int], openTimeout_opt: Option[Timeout])(implicit timeout: Timeout): Future[String] @@ -109,8 +111,13 @@ class EclairImpl(appKit: Kit) extends Eclair { implicit val ec = appKit.system.dispatcher - override def connect(uri: String)(implicit timeout: Timeout): Future[String] = { - (appKit.switchboard ? Peer.Connect(NodeURI.parse(uri))).mapTo[String] + override def connect(target: Either[NodeURI, PublicKey])(implicit timeout: Timeout): Future[String] = target match { + case Left(uri) => (appKit.switchboard ? Peer.Connect(uri)).mapTo[String] + case Right(pubKey) => (appKit.switchboard ? Peer.Connect(pubKey, None)).mapTo[String] + } + + override def disconnect(nodeId: PublicKey)(implicit timeout: Timeout): Future[String] = { + (appKit.switchboard ? Peer.Disconnect(nodeId)).mapTo[String] } override def open(nodeId: PublicKey, fundingSatoshis: Long, pushMsat_opt: Option[Long], fundingFeerateSatByte_opt: Option[Long], flags_opt: Option[Int], openTimeout_opt: Option[Timeout])(implicit timeout: Timeout): Future[String] = { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/api/FormParamExtractors.scala b/eclair-core/src/main/scala/fr/acinq/eclair/api/FormParamExtractors.scala index a9e24f544..17425cf36 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/api/FormParamExtractors.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/api/FormParamExtractors.scala @@ -24,9 +24,9 @@ import akka.util.Timeout import fr.acinq.bitcoin.ByteVector32 import fr.acinq.bitcoin.Crypto.PublicKey import fr.acinq.eclair.ShortChannelId +import fr.acinq.eclair.io.NodeURI import fr.acinq.eclair.payment.PaymentRequest import scodec.bits.ByteVector - import scala.concurrent.duration._ import scala.util.{Failure, Success, Try} @@ -60,6 +60,10 @@ object FormParamExtractors { Timeout(str.toInt.seconds) } + implicit val nodeURIUnmarshaller: Unmarshaller[String, NodeURI] = Unmarshaller.strict { str => + NodeURI.parse(str) + } + implicit val pubkeyListUnmarshaller: Unmarshaller[String, List[PublicKey]] = Unmarshaller.strict { str => Try(serialization.read[List[String]](str).map { el => PublicKey(ByteVector.fromValidHex(el), checkValid = false) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/api/Service.scala b/eclair-core/src/main/scala/fr/acinq/eclair/api/Service.scala index b8d633ada..f816aa8be 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/api/Service.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/api/Service.scala @@ -30,6 +30,7 @@ import akka.http.scaladsl.server.directives.Credentials import akka.stream.scaladsl.{BroadcastHub, Flow, Keep, Source} import akka.stream.{ActorMaterializer, OverflowStrategy} import akka.util.Timeout +import com.google.common.net.HostAndPort import fr.acinq.bitcoin.ByteVector32 import fr.acinq.bitcoin.Crypto.PublicKey import fr.acinq.eclair.api.FormParamExtractors._ @@ -41,6 +42,7 @@ import fr.acinq.eclair.{Eclair, ShortChannelId} import grizzled.slf4j.Logging import org.json4s.jackson.Serialization import scodec.bits.ByteVector + import scala.concurrent.Future import scala.concurrent.duration._ @@ -135,10 +137,17 @@ trait Service extends ExtraDirectives with Logging { complete(eclairApi.getInfoResponse()) } ~ path("connect") { - formFields("uri".as[String]) { uri => - complete(eclairApi.connect(uri)) + formFields("uri".as[NodeURI]) { uri => + complete(eclairApi.connect(Left(uri))) } ~ formFields(nodeIdFormParam, "host".as[String], "port".as[Int].?) { (nodeId, host, port_opt) => - complete(eclairApi.connect(s"$nodeId@$host:${port_opt.getOrElse(NodeURI.DEFAULT_PORT)}")) + complete(eclairApi.connect(Left(NodeURI(nodeId, HostAndPort.fromParts(host, port_opt.getOrElse(NodeURI.DEFAULT_PORT)))))) + } ~ formFields(nodeIdFormParam) { nodeId => + complete(eclairApi.connect(Right(nodeId))) + } + } ~ + path("disconnect") { + formFields(nodeIdFormParam) { nodeId => + complete(eclairApi.disconnect(nodeId)) } } ~ path("open") { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/NetworkDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/NetworkDb.scala index 1387170a0..546516785 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/NetworkDb.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/NetworkDb.scala @@ -27,6 +27,8 @@ trait NetworkDb { def updateNode(n: NodeAnnouncement) + def getNode(nodeId: PublicKey): Option[NodeAnnouncement] + def removeNode(nodeId: PublicKey) def listNodes(): Seq[NodeAnnouncement] diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteNetworkDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteNetworkDb.scala index 51e29cd81..4bca51ee2 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteNetworkDb.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteNetworkDb.scala @@ -59,6 +59,14 @@ class SqliteNetworkDb(sqlite: Connection) extends NetworkDb { } } + override def getNode(nodeId: Crypto.PublicKey): Option[NodeAnnouncement] = { + using(sqlite.prepareStatement("SELECT data FROM nodes WHERE node_id=?")) { statement => + statement.setBytes(1, nodeId.toBin.toArray) + val rs = statement.executeQuery() + codecSequence(rs, nodeAnnouncementCodec).headOption + } + } + override def removeNode(nodeId: Crypto.PublicKey): Unit = { using(sqlite.prepareStatement("DELETE FROM nodes WHERE node_id=?")) { statement => statement.setBytes(1, nodeId.toBin.toArray) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/io/Peer.scala b/eclair-core/src/main/scala/fr/acinq/eclair/io/Peer.scala index eadf81caa..020ad791a 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/io/Peer.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/io/Peer.scala @@ -23,18 +23,17 @@ import java.nio.ByteOrder import akka.actor.{ActorRef, FSM, OneForOneStrategy, PoisonPill, Props, Status, SupervisorStrategy, Terminated} import akka.event.Logging.MDC import akka.util.Timeout +import com.google.common.net.HostAndPort import fr.acinq.bitcoin.Crypto.PublicKey import fr.acinq.bitcoin.{ByteVector32, DeterministicWallet, MilliSatoshi, Protocol, Satoshi} import fr.acinq.eclair.blockchain.EclairWallet import fr.acinq.eclair.channel._ import fr.acinq.eclair.crypto.TransportHandler -import fr.acinq.eclair.secureRandom import fr.acinq.eclair.router._ import fr.acinq.eclair.wire._ -import fr.acinq.eclair.{wire, _} +import fr.acinq.eclair.{secureRandom, wire, _} import scodec.Attempt import scodec.bits.ByteVector - import scala.compat.Platform import scala.concurrent.duration._ import scala.util.Random @@ -59,26 +58,34 @@ class Peer(nodeParams: NodeParams, remoteNodeId: PublicKey, authenticator: Actor } when(DISCONNECTED) { - case Event(Peer.Connect(NodeURI(_, hostAndPort)), d: DisconnectedData) => - val address = new InetSocketAddress(hostAndPort.getHost, hostAndPort.getPort) - if (d.address_opt.contains(address)) { - // we already know this address, we'll reconnect automatically - sender ! "reconnection in progress" - stay - } else { - // we immediately process explicit connection requests to new addresses - context.actorOf(Client.props(nodeParams, authenticator, address, remoteNodeId, origin_opt = Some(sender()))) - stay + case Event(Peer.Connect(_, address_opt), d: DisconnectedData) => + address_opt + .map(hostAndPort2InetSocketAddress) + .orElse(getPeerAddressFromNodeAnnouncement) match { + case None => + sender ! "no address found" + stay + case Some(address) => + if (d.address_opt.contains(address)) { + // we already know this address, we'll reconnect automatically + sender ! "reconnection in progress" + stay + } else { + // we immediately process explicit connection requests to new addresses + context.actorOf(Client.props(nodeParams, authenticator, address, remoteNodeId, origin_opt = Some(sender()))) + stay using d.copy(address_opt = Some(address)) + } } case Event(Reconnect, d: DisconnectedData) => - d.address_opt match { - case None => stay // no-op (this peer didn't initiate the connection and doesn't have the ip of the counterparty) - case _ if d.channels.isEmpty => stay // no-op (no more channels with this peer) + d.address_opt.orElse(getPeerAddressFromNodeAnnouncement) match { + case _ if d.channels.isEmpty => stay // no-op, no more channels with this peer + case None => stay // no-op, we don't know any address to this peer and we won't try reconnecting again case Some(address) => context.actorOf(Client.props(nodeParams, authenticator, address, remoteNodeId, origin_opt = None)) + log.info(s"reconnecting to $address") // exponential backoff retry with a finite max - setTimer(RECONNECT_TIMER, Reconnect, Math.min(10 + Math.pow(2, d.attempts), 60) seconds, repeat = false) + setTimer(RECONNECT_TIMER, Reconnect, Math.min(10 + Math.pow(2, d.attempts), 3600) seconds, repeat = false) stay using d.copy(attempts = d.attempts + 1) } @@ -177,6 +184,13 @@ class Peer(nodeParams: NodeParams, remoteNodeId: PublicKey, authenticator: Actor } else { stay using d.copy(channels = channels1) } + + case Event(Disconnect(nodeId), d: InitializingData) if nodeId == remoteNodeId => + log.info("disconnecting") + sender ! "disconnecting" + d.transport ! PoisonPill + stay + } when(CONNECTED) { @@ -411,7 +425,9 @@ class Peer(nodeParams: NodeParams, remoteNodeId: PublicKey, authenticator: Actor log.info(s"resuming processing of network announcements for peer") stay using d.copy(behavior = d.behavior.copy(fundingTxAlreadySpentCount = 0, ignoreNetworkAnnouncement = false)) - case Event(Disconnect, d: ConnectedData) => + case Event(Disconnect(nodeId), d: ConnectedData) if nodeId == remoteNodeId => + log.info(s"disconnecting") + sender ! "disconnecting" d.transport ! PoisonPill stay @@ -478,8 +494,8 @@ class Peer(nodeParams: NodeParams, remoteNodeId: PublicKey, authenticator: Actor onTransition { case INSTANTIATING -> DISCONNECTED if nodeParams.autoReconnect && nextStateData.address_opt.isDefined => self ! Reconnect // we reconnect right away if we just started the peer - case _ -> DISCONNECTED if nodeParams.autoReconnect && nextStateData.address_opt.isDefined => setTimer(RECONNECT_TIMER, Reconnect, 1 second, repeat = false) - case DISCONNECTED -> _ if nodeParams.autoReconnect && stateData.address_opt.isDefined => cancelTimer(RECONNECT_TIMER) + case _ -> DISCONNECTED if nodeParams.autoReconnect => setTimer(RECONNECT_TIMER, Reconnect, 1 second, repeat = false) + case DISCONNECTED -> _ if nodeParams.autoReconnect => cancelTimer(RECONNECT_TIMER) } def createNewChannel(nodeParams: NodeParams, funder: Boolean, fundingSatoshis: Long, origin_opt: Option[ActorRef]): (ActorRef, LocalParams) = { @@ -501,6 +517,11 @@ class Peer(nodeParams: NodeParams, remoteNodeId: PublicKey, authenticator: Actor stop(FSM.Normal) } + // TODO gets the first of the list, improve selection? + def getPeerAddressFromNodeAnnouncement: Option[InetSocketAddress] = { + nodeParams.db.network.getNode(remoteNodeId).flatMap(_.addresses.headOption.map(_.socketAddress)) + } + // a failing channel won't be restarted, it should handle its states override val supervisorStrategy = OneForOneStrategy(loggingEnabled = true) { case _ => SupervisorStrategy.Stop } @@ -549,9 +570,14 @@ object Peer { case object CONNECTED extends State case class Init(previousKnownAddress: Option[InetSocketAddress], storedChannels: Set[HasCommitments]) - case class Connect(uri: NodeURI) + case class Connect(nodeId: PublicKey, address_opt: Option[HostAndPort]) { + def uri: Option[NodeURI] = address_opt.map(NodeURI(nodeId, _)) + } + object Connect { + def apply(uri: NodeURI): Connect = new Connect(uri.nodeId, Some(uri.address)) + } case object Reconnect - case object Disconnect + case class Disconnect(nodeId: PublicKey) case object ResumeAnnouncements case class OpenChannel(remoteNodeId: PublicKey, fundingSatoshis: Satoshi, pushMsat: MilliSatoshi, fundingTxFeeratePerKw_opt: Option[Long], channelFlags: Option[Byte], timeout_opt: Option[Timeout]) { require(fundingSatoshis.amount < Channel.MAX_FUNDING_SATOSHIS, s"fundingSatoshis must be less than ${Channel.MAX_FUNDING_SATOSHIS}") @@ -617,4 +643,6 @@ object Peer { case _ => true // if there is a filter and message doesn't have a timestamp (e.g. channel_announcement), then we send it } } + + def hostAndPort2InetSocketAddress(hostAndPort: HostAndPort): InetSocketAddress = new InetSocketAddress(hostAndPort.getHost, hostAndPort.getPort) } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/io/Switchboard.scala b/eclair-core/src/main/scala/fr/acinq/eclair/io/Switchboard.scala index 2abc2f008..56e918cdd 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/io/Switchboard.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/io/Switchboard.scala @@ -65,7 +65,11 @@ class Switchboard(nodeParams: NodeParams, authenticator: ActorRef, watcher: Acto channels .groupBy(_.commitments.remoteParams.nodeId) .map { - case (remoteNodeId, states) => (remoteNodeId, states, peers.get(remoteNodeId)) + case (remoteNodeId, states) => + val address_opt = peers.get(remoteNodeId).orElse { + nodeParams.db.network.getNode(remoteNodeId).flatMap(_.addresses.headOption) // gets the first of the list! TODO improve selection? + } + (remoteNodeId, states, address_opt) } .foreach { case (remoteNodeId, states, nodeaddress_opt) => @@ -77,14 +81,20 @@ class Switchboard(nodeParams: NodeParams, authenticator: ActorRef, watcher: Acto def receive: Receive = { - case Peer.Connect(NodeURI(publicKey, _)) if publicKey == nodeParams.nodeId => + case Peer.Connect(publicKey, _) if publicKey == nodeParams.nodeId => sender ! Status.Failure(new RuntimeException("cannot open connection with oneself")) case c: Peer.Connect => // we create a peer if it doesn't exist - val peer = createOrGetPeer(c.uri.nodeId, previousKnownAddress = None, offlineChannels = Set.empty) + val peer = createOrGetPeer(c.nodeId, previousKnownAddress = None, offlineChannels = Set.empty) peer forward c + case d: Peer.Disconnect => + getPeer(d.nodeId) match { + case Some(peer) => peer forward d + case None => sender ! Status.Failure(new RuntimeException("peer not found")) + } + case o: Peer.OpenChannel => getPeer(o.remoteNodeId) match { case Some(peer) => peer forward o diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/TestkitBaseClass.scala b/eclair-core/src/test/scala/fr/acinq/eclair/TestkitBaseClass.scala index 949de4793..6afa9e45f 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/TestkitBaseClass.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/TestkitBaseClass.scala @@ -18,6 +18,7 @@ package fr.acinq.eclair import akka.actor.{ActorNotFound, ActorSystem, PoisonPill} import akka.testkit.TestKit +import com.typesafe.config.ConfigFactory import fr.acinq.eclair.blockchain.fee.FeeratesPerKw import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, fixture} @@ -47,4 +48,4 @@ abstract class TestkitBaseClass extends TestKit(ActorSystem("test")) with fixtur Globals.feeratesPerKw.set(FeeratesPerKw.single(1)) } -} +} \ No newline at end of file diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala index 60c79cd29..abe4b102a 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala @@ -30,6 +30,9 @@ import fr.acinq.bitcoin.Crypto.PublicKey import fr.acinq.bitcoin.{ByteVector32, Crypto, MilliSatoshi} import fr.acinq.eclair.TestConstants._ import fr.acinq.eclair._ +import fr.acinq.eclair.channel.RES_GETINFO +import fr.acinq.eclair.db.{IncomingPayment, NetworkFee, OutgoingPayment, Stats} +import fr.acinq.eclair.io.NodeURI import fr.acinq.eclair.io.Peer.PeerInfo import fr.acinq.eclair.payment.PaymentLifecycle.PaymentFailed import fr.acinq.eclair.payment._ @@ -204,35 +207,35 @@ class ApiServiceSpec extends FunSuite with ScalatestRouteTest with IdiomaticMock test("'connect' method should accept an URI and a triple with nodeId/host/port") { - val remoteNodeId = "030bb6a5e0c6b203c7e2180fb78c7ba4bdce46126761d8201b91ddac089cdecc87" - val remoteHost = "93.137.102.239" - val remoteUri = "030bb6a5e0c6b203c7e2180fb78c7ba4bdce46126761d8201b91ddac089cdecc87@93.137.102.239:9735" + val remoteNodeId = PublicKey(hex"030bb6a5e0c6b203c7e2180fb78c7ba4bdce46126761d8201b91ddac089cdecc87") + val remoteUri = NodeURI.parse("030bb6a5e0c6b203c7e2180fb78c7ba4bdce46126761d8201b91ddac089cdecc87@93.137.102.239:9735") val eclair = mock[Eclair] - eclair.connect(any[String])(any[Timeout]) returns Future.successful("connected") + eclair.connect(any[Either[NodeURI, PublicKey]])(any[Timeout]) returns Future.successful("connected") val mockService = new MockService(eclair) - Post("/connect", FormData("nodeId" -> remoteNodeId, "host" -> remoteHost).toEntity) ~> + Post("/connect", FormData("nodeId" -> remoteNodeId.toHex).toEntity) ~> addCredentials(BasicHttpCredentials("", mockService.password)) ~> Route.seal(mockService.route) ~> check { assert(handled) assert(status == OK) assert(entityAs[String] == "\"connected\"") - eclair.connect(remoteUri)(any[Timeout]).wasCalled(once) + eclair.connect(Right(remoteNodeId))(any[Timeout]).wasCalled(once) } - Post("/connect", FormData("uri" -> remoteUri).toEntity) ~> + Post("/connect", FormData("uri" -> remoteUri.toString).toEntity) ~> addCredentials(BasicHttpCredentials("", mockService.password)) ~> Route.seal(mockService.route) ~> check { assert(handled) assert(status == OK) assert(entityAs[String] == "\"connected\"") - eclair.connect(remoteUri)(any[Timeout]).wasCalled(twice) // must account for the previous, identical, invocation + eclair.connect(Left(remoteUri))(any[Timeout]).wasCalled(once) // must account for the previous, identical, invocation } } + test("'send' method should correctly forward amount parameters to EclairImpl") { val invoice = "lnbc12580n1pw2ywztpp554ganw404sh4yjkwnysgn3wjcxfcq7gtx53gxczkjr9nlpc3hzvqdq2wpskwctddyxqr4rqrzjqwryaup9lh50kkranzgcdnn2fgvx390wgj5jd07rwr3vxeje0glc7z9rtvqqwngqqqqqqqlgqqqqqeqqjqrrt8smgjvfj7sg38dwtr9kc9gg3era9k3t2hvq3cup0jvsrtrxuplevqgfhd3rzvhulgcxj97yjuj8gdx8mllwj4wzjd8gdjhpz3lpqqvk2plh" diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/db/SqliteNetworkDbSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/db/SqliteNetworkDbSpec.scala index 32dad143e..f94dcabe6 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/db/SqliteNetworkDbSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/db/SqliteNetworkDbSpec.scala @@ -47,6 +47,7 @@ class SqliteNetworkDbSpec extends FunSuite { assert(db.listNodes().toSet === Set.empty) db.addNode(node_1) db.addNode(node_1) // duplicate is ignored + assert(db.getNode(node_1.nodeId) == Some(node_1)) assert(db.listNodes().size === 1) db.addNode(node_2) db.addNode(node_3) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/IntegrationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/IntegrationSpec.scala index aaed01027..d2e56fa2c 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/IntegrationSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/IntegrationSpec.scala @@ -159,9 +159,10 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService def connect(node1: Kit, node2: Kit, fundingSatoshis: Long, pushMsat: Long) = { val sender = TestProbe() val address = node2.nodeParams.publicAddresses.head - sender.send(node1.switchboard, Peer.Connect(NodeURI( + sender.send(node1.switchboard, Peer.Connect( nodeId = node2.nodeParams.nodeId, - address = HostAndPort.fromParts(address.socketAddress.getHostString, address.socketAddress.getPort)))) + address_opt = Some(HostAndPort.fromParts(address.socketAddress.getHostString, address.socketAddress.getPort)) + )) sender.expectMsgAnyOf(10 seconds, "connected", "already connected") sender.send(node1.switchboard, Peer.OpenChannel( remoteNodeId = node2.nodeParams.nodeId, @@ -485,7 +486,7 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService sender.send(nodes("F1").switchboard, 'peers) val peers = sender.expectMsgType[Iterable[ActorRef]] // F's only node is C - peers.head ! Disconnect + peers.head ! Peer.Disconnect(nodes("C").nodeParams.nodeId) // we then wait for F to be in disconnected state awaitCond({ sender.send(nodes("F1").register, Forward(htlc.channelId, CMD_GETSTATE)) @@ -566,7 +567,7 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService sender.send(nodes("F2").switchboard, 'peers) val peers = sender.expectMsgType[Iterable[ActorRef]] // F's only node is C - peers.head ! Disconnect + peers.head ! Disconnect(nodes("C").nodeParams.nodeId) // we then wait for F to be in disconnected state awaitCond({ sender.send(nodes("F2").register, Forward(htlc.channelId, CMD_GETSTATE)) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/io/PeerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/io/PeerSpec.scala index c5bb498ff..157c6212a 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/io/PeerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/io/PeerSpec.scala @@ -16,12 +16,14 @@ package fr.acinq.eclair.io -import java.net.InetSocketAddress +import java.net.{Inet4Address, InetSocketAddress} -import akka.actor.ActorRef -import akka.testkit.{TestFSMRef, TestProbe} +import akka.actor.{ActorRef, ActorSystem, PoisonPill} +import akka.actor.FSM.{CurrentState, SubscribeTransitionCallBack, Transition} +import akka.testkit.{EventFilter, TestFSMRef, TestKit, TestProbe} import fr.acinq.bitcoin.Crypto.PublicKey import fr.acinq.eclair.TestConstants._ +import fr.acinq.eclair._ import fr.acinq.eclair.blockchain.EclairWallet import fr.acinq.eclair.channel.HasCommitments import fr.acinq.eclair.crypto.TransportHandler @@ -29,14 +31,18 @@ import fr.acinq.eclair.db.ChannelStateSpec import fr.acinq.eclair.io.Peer._ import fr.acinq.eclair.router.RoutingSyncSpec.makeFakeRoutingInfo import fr.acinq.eclair.router.{ChannelRangeQueries, ChannelRangeQueriesSpec, Rebroadcast} -import fr.acinq.eclair.wire.{Error, Ping, Pong} -import fr.acinq.eclair.{ShortChannelId, TestkitBaseClass, randomBytes, wire} -import org.scalatest.Outcome +import fr.acinq.eclair.wire.LightningMessageCodecsSpec.randomSignature +import fr.acinq.eclair.wire.{Color, Error, IPv4, NodeAddress, NodeAnnouncement, Ping, Pong} +import org.scalatest.{Outcome, Tag} +import scodec.bits.ByteVector import scala.concurrent.duration._ - class PeerSpec extends TestkitBaseClass { + + def ipv4FromInet4(address: InetSocketAddress) = IPv4.apply(address.getAddress.asInstanceOf[Inet4Address], address.getPort) + + val fakeIPAddress = NodeAddress.fromParts("1.2.3.4", 42000).get val shortChannelIds = ChannelRangeQueriesSpec.shortChannelIds.take(100) val fakeRoutingInfo = shortChannelIds.map(makeFakeRoutingInfo) val channels = fakeRoutingInfo.map(_._1).toList @@ -46,6 +52,15 @@ class PeerSpec extends TestkitBaseClass { case class FixtureParam(remoteNodeId: PublicKey, authenticator: TestProbe, watcher: TestProbe, router: TestProbe, relayer: TestProbe, connection: TestProbe, transport: TestProbe, peer: TestFSMRef[Peer.State, Peer.Data, Peer]) override protected def withFixture(test: OneArgTest): Outcome = { + val aParams = Alice.nodeParams + val aliceParams = test.tags.contains("with_node_announcements") match { + case true => + val aliceAnnouncement = NodeAnnouncement(randomSignature, ByteVector.empty, 1, Bob.nodeParams.nodeId, Color(100.toByte, 200.toByte, 300.toByte), "node-alias", fakeIPAddress :: Nil) + aParams.db.network.addNode(aliceAnnouncement) + aParams + case false => aParams + } + val authenticator = TestProbe() val watcher = TestProbe() val router = TestProbe() @@ -54,7 +69,7 @@ class PeerSpec extends TestkitBaseClass { val transport = TestProbe() val wallet: EclairWallet = null // unused val remoteNodeId = Bob.nodeParams.nodeId - val peer: TestFSMRef[Peer.State, Peer.Data, Peer] = TestFSMRef(new Peer(Alice.nodeParams, remoteNodeId, authenticator.ref, watcher.ref, router.ref, relayer.ref, wallet)) + val peer: TestFSMRef[Peer.State, Peer.Data, Peer] = TestFSMRef(new Peer(aliceParams, remoteNodeId, authenticator.ref, watcher.ref, router.ref, relayer.ref, wallet)) withFixture(test.toNoArgTest(FixtureParam(remoteNodeId, authenticator, watcher, router, relayer, connection, transport, peer))) } @@ -62,7 +77,7 @@ class PeerSpec extends TestkitBaseClass { // let's simulate a connection val probe = TestProbe() probe.send(peer, Peer.Init(None, channels)) - authenticator.send(peer, Authenticator.Authenticated(connection.ref, transport.ref, remoteNodeId, new InetSocketAddress("1.2.3.4", 42000), outgoing = true, None)) + authenticator.send(peer, Authenticator.Authenticated(connection.ref, transport.ref, remoteNodeId, fakeIPAddress.socketAddress, outgoing = true, None)) transport.expectMsgType[TransportHandler.Listener] transport.expectMsgType[wire.Init] transport.send(peer, wire.Init(Bob.nodeParams.globalFeatures, Bob.nodeParams.localFeatures)) @@ -77,7 +92,38 @@ class PeerSpec extends TestkitBaseClass { val probe = TestProbe() connect(remoteNodeId, authenticator, watcher, router, relayer, connection, transport, peer, channels = Set(ChannelStateSpec.normal)) probe.send(peer, Peer.GetPeerInfo) - probe.expectMsg(PeerInfo(remoteNodeId, "CONNECTED", Some(new InetSocketAddress("1.2.3.4", 42000)), 1)) + probe.expectMsg(PeerInfo(remoteNodeId, "CONNECTED", Some(fakeIPAddress.socketAddress), 1)) + } + + test("fail to connect if no address provided or found") { f => + import f._ + + val probe = TestProbe() + val monitor = TestProbe() + + peer ! SubscribeTransitionCallBack(monitor.ref) + + probe.send(peer, Peer.Init(None, Set.empty)) + val CurrentState(_, INSTANTIATING) = monitor.expectMsgType[CurrentState[_]] + val Transition(_, INSTANTIATING, DISCONNECTED) = monitor.expectMsgType[Transition[_]] + probe.send(peer, Peer.Connect(remoteNodeId, address_opt = None)) + probe.expectMsg(s"no address found") + } + + test("if no address was specified during connection use the one from node_announcement", Tag("with_node_announcements")) { f => + import f._ + + val probe = TestProbe() + val monitor = TestProbe() + + peer ! SubscribeTransitionCallBack(monitor.ref) + + probe.send(peer, Peer.Init(None, Set.empty)) + val CurrentState(_, INSTANTIATING) = monitor.expectMsgType[CurrentState[_]] + val Transition(_, INSTANTIATING, DISCONNECTED) = monitor.expectMsgType[Transition[_]] + + probe.send(peer, Peer.Connect(remoteNodeId, None)) + awaitCond(peer.stateData.address_opt == Some(fakeIPAddress.socketAddress)) } test("ignore connect to same address") { f => @@ -119,7 +165,7 @@ class PeerSpec extends TestkitBaseClass { awaitCond(peer.stateData.asInstanceOf[DisconnectedData].attempts == 3) } - test("disconnect if incompatible features") {f => + test("disconnect if incompatible features") { f => import f._ val probe = TestProbe() probe.watch(transport.ref) @@ -133,6 +179,33 @@ class PeerSpec extends TestkitBaseClass { probe.expectTerminated(transport.ref) } + test("handle disconnect in status INITIALIZING") { f => + import f._ + + val probe = TestProbe() + probe.send(peer, Peer.Init(None, Set(ChannelStateSpec.normal))) + authenticator.send(peer, Authenticator.Authenticated(connection.ref, transport.ref, remoteNodeId, fakeIPAddress.socketAddress, outgoing = true, None)) + + probe.send(peer, Peer.GetPeerInfo) + assert(probe.expectMsgType[Peer.PeerInfo].state == "INITIALIZING") + + probe.send(peer, Peer.Disconnect(f.remoteNodeId)) + probe.expectMsg("disconnecting") + } + + test("handle disconnect in status CONNECTED") { f => + import f._ + + val probe = TestProbe() + connect(remoteNodeId, authenticator, watcher, router, relayer, connection, transport, peer, channels = Set(ChannelStateSpec.normal)) + + probe.send(peer, Peer.GetPeerInfo) + assert(probe.expectMsgType[Peer.PeerInfo].state == "CONNECTED") + + probe.send(peer, Peer.Disconnect(f.remoteNodeId)) + probe.expectMsg("disconnecting") + } + test("reply to ping") { f => import f._ val probe = TestProbe() @@ -203,7 +276,7 @@ class PeerSpec extends TestkitBaseClass { probe.send(peer, filter) probe.send(peer, rebroadcast) // peer doesn't filter channel announcements - channels.foreach(transport.expectMsg(_)) + channels.foreach(transport.expectMsg(10 seconds, _)) // but it will only send updates and node announcements matching the filter updates.filter(u => timestamps.contains(u.timestamp)).foreach(transport.expectMsg(_)) nodes.filter(u => timestamps.contains(u.timestamp)).foreach(transport.expectMsg(_)) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/io/PeerSpecWithLogging.scala b/eclair-core/src/test/scala/fr/acinq/eclair/io/PeerSpecWithLogging.scala new file mode 100644 index 000000000..265ccf826 --- /dev/null +++ b/eclair-core/src/test/scala/fr/acinq/eclair/io/PeerSpecWithLogging.scala @@ -0,0 +1,43 @@ +package fr.acinq.eclair.io + +import akka.actor.{ActorRef, ActorSystem} +import akka.testkit.{EventFilter, TestFSMRef, TestKit, TestProbe} +import com.typesafe.config.ConfigFactory +import fr.acinq.eclair.db.ChannelStateSpec +import org.scalatest.{FunSuiteLike, Outcome, Tag} +import scala.concurrent.duration._ +import akka.testkit.{TestFSMRef, TestProbe} +import fr.acinq.eclair.TestConstants.{Alice, Bob} +import fr.acinq.eclair.blockchain.EclairWallet +import fr.acinq.eclair.wire.LightningMessageCodecsSpec.randomSignature +import fr.acinq.eclair.wire.{Color, IPv4, NodeAddress, NodeAnnouncement} +import scodec.bits.ByteVector + +class PeerSpecWithLogging extends TestKit(ActorSystem("test", ConfigFactory.parseString("""akka.loggers = ["akka.testkit.TestEventListener"]"""))) with FunSuiteLike { + + val fakeIPAddress = NodeAddress.fromParts("1.2.3.4", 42000).get + + test("reconnect using the address from node_announcement") { + val aliceParams = Alice.nodeParams + val aliceAnnouncement = NodeAnnouncement(randomSignature, ByteVector.empty, 1, Bob.nodeParams.nodeId, Color(100.toByte, 200.toByte, 300.toByte), "node-alias", fakeIPAddress :: Nil) + aliceParams.db.network.addNode(aliceAnnouncement) + val authenticator = TestProbe() + val watcher = TestProbe() + val router = TestProbe() + val relayer = TestProbe() + val wallet: EclairWallet = null // unused + val remoteNodeId = Bob.nodeParams.nodeId + val peer: TestFSMRef[Peer.State, Peer.Data, Peer] = TestFSMRef(new Peer(aliceParams, remoteNodeId, authenticator.ref, watcher.ref, router.ref, relayer.ref, wallet)) + + + val probe = TestProbe() + awaitCond({peer.stateName.toString == "INSTANTIATING"}, 10 seconds) + probe.send(peer, Peer.Init(None, Set(ChannelStateSpec.normal))) + awaitCond({peer.stateName.toString == "DISCONNECTED" && peer.stateData.address_opt.isEmpty}, 10 seconds) + EventFilter.info(message = s"reconnecting to ${fakeIPAddress.socketAddress}", occurrences = 1) intercept { + probe.send(peer, Peer.Reconnect) + } + } + + +} From 849b6bd22be6a7550c7e915e3824201ba530743d Mon Sep 17 00:00:00 2001 From: Pierre-Marie Padiou Date: Wed, 12 Jun 2019 12:29:31 +0200 Subject: [PATCH 05/21] Smarter restore logic for closing channels (#1015) If the closing type is known: - there is no need to watch the funding tx because it has already been spent and the spending tx has already reached min_depth - there is no need to attempt to publish transactions for other type of closes. * differentiate current/next remote close We can still match on the trait `RemoteClose` if we don't need that level of precision. --- .../fr/acinq/eclair/channel/Channel.scala | 45 ++- .../fr/acinq/eclair/channel/Helpers.scala | 44 ++- .../fr/acinq/eclair/payment/Auditor.scala | 2 +- .../fr/acinq/eclair/channel/HelpersSpec.scala | 296 +++++++++++++++++- 4 files changed, 349 insertions(+), 38 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 5beac4d1c..c1967c0bf 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 @@ -183,24 +183,35 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId goto(CLOSED) using closing case closing: DATA_CLOSING => // we don't put back the WatchSpent if the commitment tx has already been published and the spending tx already reached mindepth - val commitTxOutpoint = closing.commitments.commitInput.outPoint - if (closing.localCommitPublished.exists(_.irrevocablySpent.contains(commitTxOutpoint)) || - closing.remoteCommitPublished.exists(_.irrevocablySpent.contains(commitTxOutpoint)) || - closing.nextRemoteCommitPublished.exists(_.irrevocablySpent.contains(commitTxOutpoint)) || - closing.revokedCommitPublished.exists(_.irrevocablySpent.contains(commitTxOutpoint)) || - closing.futureRemoteCommitPublished.exists(_.irrevocablySpent.contains(commitTxOutpoint))) { - log.info(s"funding tx has already been spent and spending tx reached mindepth, no need to put back the watch-spent") - } else { - // TODO: should we wait for an acknowledgment from the watcher? - blockchain ! WatchSpent(self, data.commitments.commitInput.outPoint.txid, data.commitments.commitInput.outPoint.index.toInt, data.commitments.commitInput.txOut.publicKeyScript, BITCOIN_FUNDING_SPENT) - blockchain ! WatchLost(self, data.commitments.commitInput.outPoint.txid, nodeParams.minDepthBlocks, BITCOIN_FUNDING_LOST) + val closingType_opt = Closing.isClosingTypeAlreadyKnown(closing) + log.info(s"channel is closing (closingType=${closingType_opt.getOrElse("UnknownYet")})") + // if the closing type is known: + // - there is no need to watch the funding tx because it has already been spent and the spending tx has + // already reached mindepth + // - there is no need to attempt to publish transactions for other type of closes + closingType_opt match { + case Some(Closing.LocalClose) => + closing.localCommitPublished.foreach(doPublish) + case Some(Closing.CurrentRemoteClose) => + closing.remoteCommitPublished.foreach(doPublish) + case Some(Closing.NextRemoteClose) => + closing.nextRemoteCommitPublished.foreach(doPublish) + case Some(Closing.RecoveryClose) => + closing.futureRemoteCommitPublished.foreach(doPublish) + case Some(Closing.RevokedClose) => + closing.revokedCommitPublished.foreach(doPublish) + case _ => + // in all other cases we need to be ready for any type of closing + // TODO: should we wait for an acknowledgment from the watcher? + blockchain ! WatchSpent(self, data.commitments.commitInput.outPoint.txid, data.commitments.commitInput.outPoint.index.toInt, data.commitments.commitInput.txOut.publicKeyScript, BITCOIN_FUNDING_SPENT) + blockchain ! WatchLost(self, data.commitments.commitInput.outPoint.txid, nodeParams.minDepthBlocks, BITCOIN_FUNDING_LOST) + closing.mutualClosePublished.foreach(doPublish) + closing.localCommitPublished.foreach(doPublish) + closing.remoteCommitPublished.foreach(doPublish) + closing.nextRemoteCommitPublished.foreach(doPublish) + closing.revokedCommitPublished.foreach(doPublish) + closing.futureRemoteCommitPublished.foreach(doPublish) } - closing.mutualClosePublished.map(doPublish(_)) - closing.localCommitPublished.foreach(doPublish(_)) - closing.remoteCommitPublished.foreach(doPublish(_)) - closing.nextRemoteCommitPublished.foreach(doPublish(_)) - closing.revokedCommitPublished.foreach(doPublish(_)) - closing.futureRemoteCommitPublished.foreach(doPublish(_)) // no need to go OFFLINE, we can directly switch to CLOSING goto(CLOSING) using closing diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Helpers.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Helpers.scala index 835e1f67b..55f1d9ca7 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Helpers.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Helpers.scala @@ -359,7 +359,9 @@ object Helpers { sealed trait ClosingType case object MutualClose extends ClosingType case object LocalClose extends ClosingType - case object RemoteClose extends ClosingType + sealed trait RemoteClose extends ClosingType + case object CurrentRemoteClose extends RemoteClose + case object NextRemoteClose extends RemoteClose case object RecoveryClose extends ClosingType case object RevokedClose extends ClosingType // @formatter:on @@ -377,6 +379,30 @@ object Helpers { data.commitments.remoteCommit.spec.toRemoteMsat == 0 && data.commitments.remoteNextCommitInfo.isRight + /** + * As soon as a tx spending the funding tx has reached min_depth, we know what the closing type will be, before + * the whole closing process finishes(e.g. there may still be delayed or unconfirmed child transactions). It can + * save us from attempting to publish some transactions. + * + * Note that we can't tell for mutual close before it is already final, because only one tx needs to be confirmed. + * + * @param closing channel state data + * @return the channel closing type, if applicable + */ + def isClosingTypeAlreadyKnown(closing: DATA_CLOSING): Option[ClosingType] = closing match { + case _ if closing.localCommitPublished.exists(lcp => lcp.irrevocablySpent.values.toSet.contains(lcp.commitTx.txid)) => + Some(LocalClose) + case _ if closing.remoteCommitPublished.exists(rcp => rcp.irrevocablySpent.values.toSet.contains(rcp.commitTx.txid)) => + Some(CurrentRemoteClose) + case _ if closing.nextRemoteCommitPublished.exists(rcp => rcp.irrevocablySpent.values.toSet.contains(rcp.commitTx.txid)) => + Some(NextRemoteClose) + case _ if closing.futureRemoteCommitPublished.exists(rcp => rcp.irrevocablySpent.values.toSet.contains(rcp.commitTx.txid)) => + Some(RecoveryClose) + case _ if closing.revokedCommitPublished.exists(rcp => rcp.irrevocablySpent.values.toSet.contains(rcp.commitTx.txid)) => + Some(RevokedClose) + case _ => None // we don't know yet what the closing type will be + } + /** * Checks if a channel is closed (i.e. its closing tx has been confirmed) * @@ -391,9 +417,9 @@ object Helpers { case closing: DATA_CLOSING if closing.localCommitPublished.exists(Closing.isLocalCommitDone) => Some(LocalClose) case closing: DATA_CLOSING if closing.remoteCommitPublished.exists(Closing.isRemoteCommitDone) => - Some(RemoteClose) + Some(CurrentRemoteClose) case closing: DATA_CLOSING if closing.nextRemoteCommitPublished.exists(Closing.isRemoteCommitDone) => - Some(RemoteClose) + Some(NextRemoteClose) case closing: DATA_CLOSING if closing.futureRemoteCommitPublished.exists(Closing.isRemoteCommitDone) => Some(RecoveryClose) case closing: DATA_CLOSING if closing.revokedCommitPublished.exists(Closing.isRevokedCommitDone) => @@ -854,18 +880,6 @@ object Helpers { }).toSet.flatten } - /** - * Tells if we were the origin of this outgoing htlc - * - * @param htlcId - * @param originChannels - * @return - */ - def isSentByLocal(htlcId: Long, originChannels: Map[Long, Origin]) = originChannels.get(htlcId) match { - case Some(Local(_, _)) => true - case _ => false - } - /** * As soon as a local or remote commitment reaches min_depth, we know which htlcs will be settled on-chain (whether * or not they actually have an output in the commitment tx). diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/Auditor.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/Auditor.scala index 888e5e961..6136b928e 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/Auditor.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/Auditor.scala @@ -65,7 +65,7 @@ class Auditor(nodeParams: NodeParams) extends Actor with ActorLogging { val event = e.closingType match { case MutualClose => "mutual" case LocalClose => "local" - case RemoteClose => "remote" + case _: RemoteClose => "remote" // can be current or next case RecoveryClose => "recovery" case RevokedClose => "revoked" } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/HelpersSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/HelpersSpec.scala index 5e1c5dcde..37fd3ee66 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/HelpersSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/HelpersSpec.scala @@ -16,7 +16,9 @@ package fr.acinq.eclair.channel -import fr.acinq.eclair.channel.Helpers.Funding +import fr.acinq.bitcoin.Transaction +import fr.acinq.eclair.channel.Helpers.Closing.{CurrentRemoteClose, LocalClose, MutualClose, NextRemoteClose, RecoveryClose, RevokedClose} +import fr.acinq.eclair.channel.Helpers.{Closing, Funding} import org.scalatest.FunSuite import scala.compat.Platform @@ -50,10 +52,294 @@ class HelpersSpec extends FunSuite { test("compute refresh delay") { import org.scalatest.Matchers._ implicit val log = akka.event.NoLogging - Helpers.nextChannelUpdateRefresh(1544400000).toSeconds should equal (0) - Helpers.nextChannelUpdateRefresh((Platform.currentTime.milliseconds - 9.days).toSeconds).toSeconds should equal (24 * 3600L +- 100) - Helpers.nextChannelUpdateRefresh((Platform.currentTime.milliseconds - 3.days).toSeconds).toSeconds should equal (7 * 24 * 3600L +- 100) - Helpers.nextChannelUpdateRefresh(Platform.currentTime.milliseconds.toSeconds).toSeconds should equal (10 * 24 * 3600L +- 100) + Helpers.nextChannelUpdateRefresh(1544400000).toSeconds should equal(0) + Helpers.nextChannelUpdateRefresh((Platform.currentTime.milliseconds - 9.days).toSeconds).toSeconds should equal(24 * 3600L +- 100) + Helpers.nextChannelUpdateRefresh((Platform.currentTime.milliseconds - 3.days).toSeconds).toSeconds should equal(7 * 24 * 3600L +- 100) + Helpers.nextChannelUpdateRefresh(Platform.currentTime.milliseconds.toSeconds).toSeconds should equal(10 * 24 * 3600L +- 100) + } + + test("tell closing type") { + val tx1 = Transaction.read("010000000110f01d4a4228ef959681feb1465c2010d0135be88fd598135b2e09d5413bf6f1000000006a473044022074658623424cebdac8290488b76f893cfb17765b7a3805e773e6770b7b17200102202892cfa9dda662d5eac394ba36fcfd1ea6c0b8bb3230ab96220731967bbdb90101210372d437866d9e4ead3d362b01b615d24cc0d5152c740d51e3c55fb53f6d335d82ffffffff01408b0700000000001976a914678db9a7caa2aca887af1177eda6f3d0f702df0d88ac00000000") + val tx2 = Transaction.read("0100000001be43e9788523ed4de0b24a007a90009bc25e667ddac0e9ee83049be03e220138000000006b483045022100f74dd6ad3e6a00201d266a0ed860a6379c6e68b473970423f3fc8a15caa1ea0f022065b4852c9da230d9e036df743cb743601ca5229e1cb610efdd99769513f2a2260121020636de7755830fb4a3f136e97ecc6c58941611957ba0364f01beae164b945b2fffffffff0150f80c000000000017a9146809053148799a10480eada3d56d15edf4a648c88700000000") + val tx3 = Transaction.read("0100000002b8682539550b3182966ecaca3d1fd5b2a96d0966a0fded143aaf771cbaf4222b000000006b483045022100c4484511ea7d9cf989797ca98e403c93372ded754ce30737af4914a222c84e8e022011648b42f8756ef4b83aa4f49e6b77f86ce7c54e1b70f25a16a94c4551c99cff012102506d400d2168a4a272b026d8b95ecb822cccd60277fb7268a6873fef0a85fe96ffffffff5a68052b6c23f6f718e09ffe56378ee90ad438b94c99b398c8d9e581a3c049d0300000006b483045022100d1b0eebc8250ebbb2d692c1e293260387b748115cf4cf892891ca4a1e81029cf02202fb5daa7647355e2c86d3f8bcfde7691a163f0dd99a002aca97f0a17cc72c5da012102fe1ec7be2f1e974c7e75932c0187f61667fa2825c4f79ccb964a83f48cce442cffffffff02ba100000000000001976a914a14c305babbd3d6984a20899f078980f078d433288acc8370800000000001976a914e0b4609a38d1a4dd1196f8c66e879b4923f9ea7388ac00000000") + val tx4 = Transaction.read("0200000001c8a8934fb38a44b969528252bc37be66ee166c7897c57384d1e561449e110c93010000006b483045022100dc6c50f445ed53d2fb41067fdcb25686fe79492d90e6e5db43235726ace247210220773d35228af0800c257970bee9cf75175d75217de09a8ecd83521befd040c4ca012102082b751372fe7e3b012534afe0bb8d1f2f09c724b1a10a813ce704e5b9c217ccfdffffff0247ba2300000000001976a914f97a7641228e6b17d4b0b08252ae75bd62a95fe788ace3de24000000000017a914a9fefd4b9a9282a1d7a17d2f14ac7d1eb88141d287f7d50800") + val tx5 = Transaction.read("010000000235a2f5c4fd48672534cce1ac063047edc38683f43c5a883f815d6026cb5f8321020000006a47304402206be5fd61b1702599acf51941560f0a1e1965aa086634b004967747f79788bd6e022002f7f719a45b8b5e89129c40a9d15e4a8ee1e33be3a891cf32e859823ecb7a510121024756c5adfbc0827478b0db042ce09d9b98e21ad80d036e73bd8e7f0ecbc254a2ffffffffb2387d3125bb8c84a2da83f4192385ce329283661dfc70191f4112c67ce7b4d0000000006b483045022100a2c737eab1c039f79238767ccb9bb3e81160e965ef0fc2ea79e8360c61b7c9f702202348b0f2c0ea2a757e25d375d9be183200ce0a79ec81d6a4ebb2ae4dc31bc3c9012102db16a822e2ec3706c58fc880c08a3617c61d8ef706cc8830cfe4561d9a5d52f0ffffffff01808d5b00000000001976a9141210c32def6b64d0d77ba8d99adeb7e9f91158b988ac00000000") + val tx6 = Transaction.read("0100000001b14ba6952c83f6f8c382befbf4e44270f13e479d5a5ff3862ac3a112f103ff2a010000006b4830450221008b097fd69bfa3715fc5e119a891933c091c55eabd3d1ddae63a1c2cc36dc9a3e02205666d5299fa403a393bcbbf4b05f9c0984480384796cdebcf69171674d00809c01210335b592484a59a44f40998d65a94f9e2eecca47e8d1799342112a59fc96252830ffffffff024bf308000000000017a914440668d018e5e0ba550d6e042abcf726694f515c8798dd1801000000001976a91453a503fe151dd32e0503bd9a2fbdbf4f9a3af1da88ac00000000") + + // only mutual close + assert(Closing.isClosingTypeAlreadyKnown( + DATA_CLOSING( + commitments = null, + mutualCloseProposed = tx1 :: tx2 :: tx3 :: Nil, + mutualClosePublished = tx2 :: tx3 :: Nil, + localCommitPublished = None, + remoteCommitPublished = None, + nextRemoteCommitPublished = None, + futureRemoteCommitPublished = None, + revokedCommitPublished = Nil) + ).isEmpty) + + // mutual + local close, but local commit tx isn't confirmed + assert(Closing.isClosingTypeAlreadyKnown( + DATA_CLOSING( + commitments = null, + mutualCloseProposed = tx1 :: Nil, + mutualClosePublished = tx1 :: Nil, + localCommitPublished = Some(LocalCommitPublished( + commitTx = tx2, + claimMainDelayedOutputTx = Some(tx3), + htlcSuccessTxs = Nil, + htlcTimeoutTxs = Nil, + claimHtlcDelayedTxs = Nil, + irrevocablySpent = Map.empty + )), + remoteCommitPublished = None, + nextRemoteCommitPublished = None, + futureRemoteCommitPublished = None, + revokedCommitPublished = Nil) + ).isEmpty) + + // mutual + local close, local commit tx confirmed + assert(Closing.isClosingTypeAlreadyKnown( + DATA_CLOSING( + commitments = null, + mutualCloseProposed = tx1 :: Nil, + mutualClosePublished = tx1 :: Nil, + localCommitPublished = Some(LocalCommitPublished( + commitTx = tx2, + claimMainDelayedOutputTx = Some(tx3), + htlcSuccessTxs = Nil, + htlcTimeoutTxs = Nil, + claimHtlcDelayedTxs = Nil, + irrevocablySpent = Map( + tx2.txIn.head.outPoint -> tx2.txid + ) + )), + remoteCommitPublished = None, + nextRemoteCommitPublished = None, + futureRemoteCommitPublished = None, + revokedCommitPublished = Nil) + ).contains(LocalClose)) + + // local close + remote close, none is confirmed + assert(Closing.isClosingTypeAlreadyKnown( + DATA_CLOSING( + commitments = null, + mutualCloseProposed = Nil, + mutualClosePublished = Nil, + localCommitPublished = Some(LocalCommitPublished( + commitTx = tx2, + claimMainDelayedOutputTx = None, + htlcSuccessTxs = Nil, + htlcTimeoutTxs = Nil, + claimHtlcDelayedTxs = Nil, + irrevocablySpent = Map.empty + )), + remoteCommitPublished = Some(RemoteCommitPublished( + commitTx = tx3, + claimMainOutputTx = None, + claimHtlcSuccessTxs = Nil, + claimHtlcTimeoutTxs = Nil, + irrevocablySpent = Map.empty + )), + nextRemoteCommitPublished = None, + futureRemoteCommitPublished = None, + revokedCommitPublished = Nil) + ).isEmpty) + + // mutual + local + remote close, remote commit tx confirmed + assert(Closing.isClosingTypeAlreadyKnown( + DATA_CLOSING( + commitments = null, + mutualCloseProposed = tx1 :: Nil, + mutualClosePublished = tx1 :: Nil, + localCommitPublished = Some(LocalCommitPublished( + commitTx = tx2, + claimMainDelayedOutputTx = None, + htlcSuccessTxs = Nil, + htlcTimeoutTxs = Nil, + claimHtlcDelayedTxs = Nil, + irrevocablySpent = Map.empty + )), + remoteCommitPublished = Some(RemoteCommitPublished( + commitTx = tx3, + claimMainOutputTx = None, + claimHtlcSuccessTxs = Nil, + claimHtlcTimeoutTxs = Nil, + irrevocablySpent = Map( + tx3.txIn.head.outPoint -> tx3.txid + ) + )), + nextRemoteCommitPublished = None, + futureRemoteCommitPublished = None, + revokedCommitPublished = Nil) + ).contains(CurrentRemoteClose)) + + // mutual + local + remote + next remote close, next remote commit tx confirmed + assert(Closing.isClosingTypeAlreadyKnown( + DATA_CLOSING( + commitments = null, + mutualCloseProposed = tx1 :: Nil, + mutualClosePublished = tx1 :: Nil, + localCommitPublished = Some(LocalCommitPublished( + commitTx = tx2, + claimMainDelayedOutputTx = None, + htlcSuccessTxs = Nil, + htlcTimeoutTxs = Nil, + claimHtlcDelayedTxs = Nil, + irrevocablySpent = Map.empty + )), + remoteCommitPublished = Some(RemoteCommitPublished( + commitTx = tx3, + claimMainOutputTx = None, + claimHtlcSuccessTxs = Nil, + claimHtlcTimeoutTxs = Nil, + irrevocablySpent = Map.empty + )), + nextRemoteCommitPublished = Some(RemoteCommitPublished( + commitTx = tx4, + claimMainOutputTx = Some(tx5), + claimHtlcSuccessTxs = Nil, + claimHtlcTimeoutTxs = Nil, + irrevocablySpent = Map( + tx4.txIn.head.outPoint -> tx4.txid + ) + )), + futureRemoteCommitPublished = None, + revokedCommitPublished = Nil) + ).contains(NextRemoteClose)) + + // future remote close, not confirmed + assert(Closing.isClosingTypeAlreadyKnown( + DATA_CLOSING( + commitments = null, + mutualCloseProposed = Nil, + mutualClosePublished = Nil, + localCommitPublished = None, + remoteCommitPublished = None, + nextRemoteCommitPublished = None, + futureRemoteCommitPublished = Some(RemoteCommitPublished( + commitTx = tx4, + claimMainOutputTx = Some(tx5), + claimHtlcSuccessTxs = Nil, + claimHtlcTimeoutTxs = Nil, + irrevocablySpent = Map.empty + )), + revokedCommitPublished = Nil) + ).isEmpty) + + // future remote close, confirmed + assert(Closing.isClosingTypeAlreadyKnown( + DATA_CLOSING( + commitments = null, + mutualCloseProposed = Nil, + mutualClosePublished = Nil, + localCommitPublished = None, + remoteCommitPublished = None, + nextRemoteCommitPublished = None, + futureRemoteCommitPublished = Some(RemoteCommitPublished( + commitTx = tx4, + claimMainOutputTx = Some(tx5), + claimHtlcSuccessTxs = Nil, + claimHtlcTimeoutTxs = Nil, + irrevocablySpent = Map( + tx4.txIn.head.outPoint -> tx4.txid + ) + )), + revokedCommitPublished = Nil) + ).contains(RecoveryClose)) + + // local close + revoked close, none confirmed + assert(Closing.isClosingTypeAlreadyKnown( + DATA_CLOSING( + commitments = null, + mutualCloseProposed = Nil, + mutualClosePublished = Nil, + localCommitPublished = Some(LocalCommitPublished( + commitTx = tx1, + claimMainDelayedOutputTx = None, + htlcSuccessTxs = Nil, + htlcTimeoutTxs = Nil, + claimHtlcDelayedTxs = Nil, + irrevocablySpent = Map.empty + )), + remoteCommitPublished = None, + nextRemoteCommitPublished = None, + futureRemoteCommitPublished = None, + revokedCommitPublished = + RevokedCommitPublished( + commitTx = tx2, + claimMainOutputTx = Some(tx3), + mainPenaltyTx = None, + htlcPenaltyTxs = Nil, + claimHtlcDelayedPenaltyTxs = Nil, + irrevocablySpent = Map.empty + ) :: + RevokedCommitPublished( + commitTx = tx4, + claimMainOutputTx = Some(tx5), + mainPenaltyTx = None, + htlcPenaltyTxs = Nil, + claimHtlcDelayedPenaltyTxs = Nil, + irrevocablySpent = Map.empty + ) :: + RevokedCommitPublished( + commitTx = tx6, + claimMainOutputTx = None, + mainPenaltyTx = None, + htlcPenaltyTxs = Nil, + claimHtlcDelayedPenaltyTxs = Nil, + irrevocablySpent = Map.empty + ) :: Nil + ) + ).isEmpty) + + // local close + revoked close, one revoked confirmed + assert(Closing.isClosingTypeAlreadyKnown( + DATA_CLOSING( + commitments = null, + mutualCloseProposed = Nil, + mutualClosePublished = Nil, + localCommitPublished = Some(LocalCommitPublished( + commitTx = tx1, + claimMainDelayedOutputTx = None, + htlcSuccessTxs = Nil, + htlcTimeoutTxs = Nil, + claimHtlcDelayedTxs = Nil, + irrevocablySpent = Map.empty + )), + remoteCommitPublished = None, + nextRemoteCommitPublished = None, + futureRemoteCommitPublished = None, + revokedCommitPublished = + RevokedCommitPublished( + commitTx = tx2, + claimMainOutputTx = Some(tx3), + mainPenaltyTx = None, + htlcPenaltyTxs = Nil, + claimHtlcDelayedPenaltyTxs = Nil, + irrevocablySpent = Map.empty + ) :: + RevokedCommitPublished( + commitTx = tx4, + claimMainOutputTx = Some(tx5), + mainPenaltyTx = None, + htlcPenaltyTxs = Nil, + claimHtlcDelayedPenaltyTxs = Nil, + irrevocablySpent = Map( + tx4.txIn.head.outPoint -> tx4.txid + ) + ) :: + RevokedCommitPublished( + commitTx = tx6, + claimMainOutputTx = None, + mainPenaltyTx = None, + htlcPenaltyTxs = Nil, + claimHtlcDelayedPenaltyTxs = Nil, + irrevocablySpent = Map.empty + ) :: Nil + ) + ).contains(RevokedClose)) } From db334380b3a34c46ac6806c70bcc13e8155a6740 Mon Sep 17 00:00:00 2001 From: Pierre-Marie Padiou Date: Thu, 13 Jun 2019 18:08:34 +0200 Subject: [PATCH 06/21] Clean pending htlcs db (#1027) We store `CMD_FULFILL_HTLC`/`CMD_FAIL_HTLC`/`CMD_FAIL_MALFORMED_HTLC` in a database (see `CommandBuffer`) because we don't want to lose preimages, or to forget to fail incoming htlcs, which would lead to unwanted channel closings. But we currently only clean up this database on success, and because of the way our watcher works, in a scenario where a downstream channel has gone to the blockchain, it may send several times the same command. Only the first one will be acked and cleaned up by the upstream channel, causing the remaining commands to stay forever in the "pending relay db". With this change we clean up the commands when they fail too. We also clean up the pending relay db on startup. --- .../fr/acinq/eclair/channel/Channel.scala | 30 +++++++++--- .../fr/acinq/eclair/db/PendingRelayDb.scala | 2 + .../db/sqlite/SqlitePendingRelayDb.scala | 17 ++++++- .../fr/acinq/eclair/io/Switchboard.scala | 49 ++++++++++++++++--- .../acinq/eclair/payment/CommandBuffer.scala | 13 +++-- .../channel/states/e/NormalStateSpec.scala | 34 ++++++++++++- .../channel/states/f/ShutdownStateSpec.scala | 33 ++++++++++++- .../eclair/db/SqlitePendingRelayDbSpec.scala | 3 +- 8 files changed, 161 insertions(+), 20 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 c1967c0bf..863b91d89 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 @@ -625,7 +625,10 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId case Success((commitments1, fulfill)) => if (c.commit) self ! CMD_SIGN handleCommandSuccess(sender, d.copy(commitments = commitments1)) sending fulfill - case Failure(cause) => handleCommandError(cause, c) + case Failure(cause) => + // we can clean up the command right away in case of failure + relayer ! CommandBuffer.CommandAck(d.channelId, c.id) + handleCommandError(cause, c) } case Event(fulfill: UpdateFulfillHtlc, d: DATA_NORMAL) => @@ -643,7 +646,10 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId case Success((commitments1, fail)) => if (c.commit) self ! CMD_SIGN handleCommandSuccess(sender, d.copy(commitments = commitments1)) sending fail - case Failure(cause) => handleCommandError(cause, c) + case Failure(cause) => + // we can clean up the command right away in case of failure + relayer ! CommandBuffer.CommandAck(d.channelId, c.id) + handleCommandError(cause, c) } case Event(c: CMD_FAIL_MALFORMED_HTLC, d: DATA_NORMAL) => @@ -651,7 +657,10 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId case Success((commitments1, fail)) => if (c.commit) self ! CMD_SIGN handleCommandSuccess(sender, d.copy(commitments = commitments1)) sending fail - case Failure(cause) => handleCommandError(cause, c) + case Failure(cause) => + // we can clean up the command right away in case of failure + relayer ! CommandBuffer.CommandAck(d.channelId, c.id) + handleCommandError(cause, c) } case Event(fail: UpdateFailHtlc, d: DATA_NORMAL) => @@ -975,7 +984,10 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId case Success((commitments1, fulfill)) => if (c.commit) self ! CMD_SIGN handleCommandSuccess(sender, d.copy(commitments = commitments1)) sending fulfill - case Failure(cause) => handleCommandError(cause, c) + case Failure(cause) => + // we can clean up the command right away in case of failure + relayer ! CommandBuffer.CommandAck(d.channelId, c.id) + handleCommandError(cause, c) } case Event(fulfill: UpdateFulfillHtlc, d: DATA_SHUTDOWN) => @@ -993,7 +1005,10 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId case Success((commitments1, fail)) => if (c.commit) self ! CMD_SIGN handleCommandSuccess(sender, d.copy(commitments = commitments1)) sending fail - case Failure(cause) => handleCommandError(cause, c) + case Failure(cause) => + // we can clean up the command right away in case of failure + relayer ! CommandBuffer.CommandAck(d.channelId, c.id) + handleCommandError(cause, c) } case Event(c: CMD_FAIL_MALFORMED_HTLC, d: DATA_SHUTDOWN) => @@ -1001,7 +1016,10 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId case Success((commitments1, fail)) => if (c.commit) self ! CMD_SIGN handleCommandSuccess(sender, d.copy(commitments = commitments1)) sending fail - case Failure(cause) => handleCommandError(cause, c) + case Failure(cause) => + // we can clean up the command right away in case of failure + relayer ! CommandBuffer.CommandAck(d.channelId, c.id) + handleCommandError(cause, c) } case Event(fail: UpdateFailHtlc, d: DATA_SHUTDOWN) => diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/PendingRelayDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/PendingRelayDb.scala index 34ddf3d84..f313eba6e 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/PendingRelayDb.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/PendingRelayDb.scala @@ -39,6 +39,8 @@ trait PendingRelayDb { def listPendingRelay(channelId: ByteVector32): Seq[Command] + def listPendingRelay(): Set[(ByteVector32, Long)] + def close(): Unit } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqlitePendingRelayDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqlitePendingRelayDb.scala index 3ae1eabed..b0621ac5e 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqlitePendingRelayDb.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqlitePendingRelayDb.scala @@ -21,11 +21,15 @@ import java.sql.Connection import fr.acinq.bitcoin.ByteVector32 import fr.acinq.eclair.channel.Command import fr.acinq.eclair.db.PendingRelayDb -import fr.acinq.eclair.db.sqlite.SqliteUtils.{codecSequence, getVersion, using} import fr.acinq.eclair.wire.CommandCodecs.cmdCodec +import scala.collection.immutable.Queue + class SqlitePendingRelayDb(sqlite: Connection) extends PendingRelayDb { + import SqliteUtils.ExtendedResultSet._ + import SqliteUtils._ + val DB_NAME = "pending_relay" val CURRENT_VERSION = 1 @@ -60,5 +64,16 @@ class SqlitePendingRelayDb(sqlite: Connection) extends PendingRelayDb { } } + override def listPendingRelay(): Set[(ByteVector32, Long)] = { + using(sqlite.prepareStatement("SELECT channel_id, htlc_id FROM pending_relay")) { statement => + val rs = statement.executeQuery() + var q: Queue[(ByteVector32, Long)] = Queue() + while (rs.next()) { + q = q :+ (rs.getByteVector32("channel_id"), rs.getLong("htlc_id")) + } + q.toSet + } + } + override def close(): Unit = sqlite.close() } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/io/Switchboard.scala b/eclair-core/src/main/scala/fr/acinq/eclair/io/Switchboard.scala index 56e918cdd..33bb05071 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/io/Switchboard.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/io/Switchboard.scala @@ -19,11 +19,13 @@ package fr.acinq.eclair.io import java.net.InetSocketAddress import akka.actor.{Actor, ActorLogging, ActorRef, OneForOneStrategy, Props, Status, SupervisorStrategy} +import fr.acinq.bitcoin.ByteVector32 import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey} import fr.acinq.eclair.NodeParams import fr.acinq.eclair.blockchain.EclairWallet import fr.acinq.eclair.channel.Helpers.Closing import fr.acinq.eclair.channel.{HasCommitments, _} +import fr.acinq.eclair.db.PendingRelayDb import fr.acinq.eclair.payment.Relayer.RelayPayload import fr.acinq.eclair.payment.{Relayed, Relayer} import fr.acinq.eclair.router.Rebroadcast @@ -62,6 +64,8 @@ class Switchboard(nodeParams: NodeParams, authenticator: ActorRef, watcher: Acto brokenHtlcKiller ! brokenHtlcs } + cleanupRelayDb(channels, nodeParams.db.pendingRelay) + channels .groupBy(_.commitments.remoteParams.nodeId) .map { @@ -163,15 +167,12 @@ object Switchboard extends Logging { * get closed, which is a major inconvenience. * * This check will detect this and will allow us to fast-fail HTLCs and thus preserve channels. - * - * @param channels - * @return */ def checkBrokenHtlcsLink(channels: Seq[HasCommitments], privateKey: PrivateKey): Seq[UpdateAddHtlc] = { - // We are interested in incoming HTLCs, that have been *cross-signed*. They signed it first, so the HTLC will first - // appear in our commitment tx, and later on in their commitment when we subsequently sign it. - // That's why we need to look in *their* commitment with direction=OUT. + // We are interested in incoming HTLCs, that have been *cross-signed* (otherwise they wouldn't have been relayed). + // They signed it first, so the HTLC will first appear in our commitment tx, and later on in their commitment when + // we subsequently sign it. That's why we need to look in *their* commitment with direction=OUT. val htlcs_in = channels .flatMap(_.commitments.remoteCommit.spec.htlcs) .filter(_.direction == OUT) @@ -192,6 +193,42 @@ object Switchboard extends Logging { htlcs_broken } + /** + * We store [[CMD_FULFILL_HTLC]]/[[CMD_FAIL_HTLC]]/[[CMD_FAIL_MALFORMED_HTLC]] + * in a database (see [[fr.acinq.eclair.payment.CommandBuffer]]) because we + * don't want to lose preimages, or to forget to fail incoming htlcs, which + * would lead to unwanted channel closings. + * + * Because of the way our watcher works, in a scenario where a downstream + * channel has gone to the blockchain, it may send several times the same + * command, and the upstream channel may have disappeared in the meantime. + * + * That's why we need to periodically clean up the pending relay db. + */ + def cleanupRelayDb(channels: Seq[HasCommitments], relayDb: PendingRelayDb): Int = { + + // We are interested in incoming HTLCs, that have been *cross-signed* (otherwise they wouldn't have been relayed). + // If the HTLC is not in their commitment, it means that we have already fulfilled/failed it and that we can remove + // the command from the pending relay db. + val channel2Htlc: Set[(ByteVector32, Long)] = + channels + .flatMap(_.commitments.remoteCommit.spec.htlcs) + .filter(_.direction == OUT) + .map(htlc => (htlc.add.channelId, htlc.add.id)) + .toSet + + val pendingRelay: Set[(ByteVector32, Long)] = relayDb.listPendingRelay() + + val toClean = pendingRelay -- channel2Htlc + + toClean.foreach { + case (channelId, htlcId) => + logger.info(s"cleaning up channelId=$channelId htlcId=$htlcId from relay db") + relayDb.removePendingRelay(channelId, htlcId) + } + toClean.size + } + } class HtlcReaper extends Actor with ActorLogging { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/CommandBuffer.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/CommandBuffer.scala index 46fd822fd..2b11d9231 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/CommandBuffer.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/CommandBuffer.scala @@ -21,6 +21,11 @@ import fr.acinq.bitcoin.ByteVector32 import fr.acinq.eclair.NodeParams import fr.acinq.eclair.channel._ +/** + * We store [[CMD_FULFILL_HTLC]]/[[CMD_FAIL_HTLC]]/[[CMD_FAIL_MALFORMED_HTLC]] + * in a database because we don't want to lose preimages, or to forget to fail + * incoming htlcs, which would lead to unwanted channel closings. + */ class CommandBuffer(nodeParams: NodeParams, register: ActorRef) extends Actor with ActorLogging { import CommandBuffer._ @@ -43,12 +48,12 @@ class CommandBuffer(nodeParams: NodeParams, register: ActorRef) extends Actor wi case ChannelStateChanged(channel, _, _, WAIT_FOR_INIT_INTERNAL | OFFLINE | SYNCING, NORMAL | SHUTDOWN | CLOSING, d: HasCommitments) => import d.channelId - // if channel is in a state where it can have pending htlcs, we send them the fulfills we know of + // if channel is in a state where it can have pending htlcs, we send them the fulfills/fails we know of pendingRelay.listPendingRelay(channelId) match { case Nil => () - case msgs => - log.info(s"re-sending ${msgs.size} unacked fulfills/fails to channel $channelId") - msgs.foreach(channel ! _) // they all have commit = false + case cmds => + log.info(s"re-sending ${cmds.size} unacked fulfills/fails to channel $channelId") + cmds.foreach(channel ! _) // they all have commit = false // better to sign once instead of after each fulfill channel ! CMD_SIGN } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalStateSpec.scala index a4cca3bad..0d8613225 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalStateSpec.scala @@ -1046,6 +1046,16 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods { assert(initialState == bob.stateData) } + test("recv CMD_FULFILL_HTLC (acknowledge in case of failure)") { f => + import f._ + val sender = TestProbe() + val initialState = bob.stateData.asInstanceOf[DATA_NORMAL] + + sender.send(bob, CMD_FULFILL_HTLC(42, randomBytes32)) // this will fail + sender.expectMsg(Failure(UnknownHtlcId(channelId(bob), 42))) + relayerB.expectMsg(CommandBuffer.CommandAck(initialState.channelId, 42)) + } + test("recv UpdateFulfillHtlc") { f => import f._ val sender = TestProbe() @@ -1148,6 +1158,17 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods { assert(initialState == bob.stateData) } + test("recv CMD_FAIL_HTLC (acknowledge in case of failure)") { f => + import f._ + val sender = TestProbe() + val r = randomBytes32 + val initialState = bob.stateData.asInstanceOf[DATA_NORMAL] + + sender.send(bob, CMD_FAIL_HTLC(42, Right(PermanentChannelFailure))) // this will fail + sender.expectMsg(Failure(UnknownHtlcId(channelId(bob), 42))) + relayerB.expectMsg(CommandBuffer.CommandAck(initialState.channelId, 42)) + } + test("recv CMD_FAIL_MALFORMED_HTLC") { f => import f._ val sender = TestProbe() @@ -1168,12 +1189,13 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods { import f._ val sender = TestProbe() val initialState = bob.stateData.asInstanceOf[DATA_NORMAL] + sender.send(bob, CMD_FAIL_MALFORMED_HTLC(42, ByteVector32.Zeroes, FailureMessageCodecs.BADONION)) sender.expectMsg(Failure(UnknownHtlcId(channelId(bob), 42))) assert(initialState == bob.stateData) } - test("recv CMD_FAIL_HTLC (invalid failure_code)") { f => + test("recv CMD_FAIL_MALFORMED_HTLC (invalid failure_code)") { f => import f._ val sender = TestProbe() val initialState = bob.stateData.asInstanceOf[DATA_NORMAL] @@ -1182,6 +1204,16 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods { assert(initialState == bob.stateData) } + test("recv CMD_FAIL_MALFORMED_HTLC (acknowledge in case of failure)") { f => + import f._ + val sender = TestProbe() + val initialState = bob.stateData.asInstanceOf[DATA_NORMAL] + + sender.send(bob, CMD_FAIL_MALFORMED_HTLC(42, ByteVector32.Zeroes, FailureMessageCodecs.BADONION)) // this will fail + sender.expectMsg(Failure(UnknownHtlcId(channelId(bob), 42))) + relayerB.expectMsg(CommandBuffer.CommandAck(initialState.channelId, 42)) + } + test("recv UpdateFailHtlc") { f => import f._ val sender = TestProbe() diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/f/ShutdownStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/f/ShutdownStateSpec.scala index a7040affe..a1c80fd03 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/f/ShutdownStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/f/ShutdownStateSpec.scala @@ -140,6 +140,16 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods { assert(initialState == bob.stateData) } + test("recv CMD_FULFILL_HTLC (acknowledge in case of failure)") { f => + import f._ + val sender = TestProbe() + val initialState = bob.stateData.asInstanceOf[DATA_SHUTDOWN] + + sender.send(bob, CMD_FULFILL_HTLC(42, randomBytes32)) // this will fail + sender.expectMsg(Failure(UnknownHtlcId(channelId(bob), 42))) + relayerB.expectMsg(CommandBuffer.CommandAck(initialState.channelId, 42)) + } + test("recv UpdateFulfillHtlc") { f => import f._ val sender = TestProbe() @@ -203,6 +213,16 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods { assert(initialState == bob.stateData) } + test("recv CMD_FAIL_HTLC (acknowledge in case of failure)") { f => + import f._ + val sender = TestProbe() + val r = randomBytes32 + val initialState = bob.stateData.asInstanceOf[DATA_SHUTDOWN] + sender.send(bob, CMD_FAIL_HTLC(42, Right(PermanentChannelFailure))) // this will fail + sender.expectMsg(Failure(UnknownHtlcId(channelId(bob), 42))) + relayerB.expectMsg(CommandBuffer.CommandAck(initialState.channelId, 42)) + } + test("recv CMD_FAIL_MALFORMED_HTLC") { f => import f._ val sender = TestProbe() @@ -224,7 +244,7 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods { assert(initialState == bob.stateData) } - test("recv CMD_FAIL_HTLC (invalid failure_code)") { f => + test("recv CMD_FAIL_MALFORMED_HTLC (invalid failure_code)") { f => import f._ val sender = TestProbe() val initialState = bob.stateData.asInstanceOf[DATA_SHUTDOWN] @@ -233,6 +253,17 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods { assert(initialState == bob.stateData) } + test("recv CMD_FAIL_MALFORMED_HTLC (acknowledge in case of failure)") { f => + import f._ + val sender = TestProbe() + val r = randomBytes32 + val initialState = bob.stateData.asInstanceOf[DATA_SHUTDOWN] + + sender.send(bob, CMD_FAIL_MALFORMED_HTLC(42, ByteVector32.Zeroes, FailureMessageCodecs.BADONION)) // this will fail + sender.expectMsg(Failure(UnknownHtlcId(channelId(bob), 42))) + relayerB.expectMsg(CommandBuffer.CommandAck(initialState.channelId, 42)) + } + test("recv UpdateFailHtlc") { f => import f._ val sender = TestProbe() diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/db/SqlitePendingRelayDbSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/db/SqlitePendingRelayDbSpec.scala index 6ae45b54b..1c2f579c1 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/db/SqlitePendingRelayDbSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/db/SqlitePendingRelayDbSpec.scala @@ -54,8 +54,9 @@ class SqlitePendingRelayDbSpec extends FunSuite { db.addPendingRelay(channelId2, msg1.id, msg1) assert(db.listPendingRelay(channelId1).toSet === Set(msg0, msg1, msg2, msg3, msg4)) assert(db.listPendingRelay(channelId2).toSet === Set(msg0, msg1)) + assert(db.listPendingRelay === Set((channelId1, msg0.id), (channelId1, msg1.id), (channelId1, msg2.id), (channelId1, msg3.id), (channelId1, msg4.id), (channelId2, msg0.id), (channelId2, msg1.id))) db.removePendingRelay(channelId1, msg1.id) - assert(db.listPendingRelay(channelId1).toSet === Set(msg0, msg2, msg3, msg4)) + assert(db.listPendingRelay === Set((channelId1, msg0.id), (channelId1, msg2.id), (channelId1, msg3.id), (channelId1, msg4.id), (channelId2, msg0.id), (channelId2, msg1.id))) } } From 199d6041c59f45427ba4ed0aebb9be580a65f87d Mon Sep 17 00:00:00 2001 From: araspitzu Date: Thu, 13 Jun 2019 18:10:09 +0200 Subject: [PATCH 07/21] Output txid and serialized tx in transaction json serializer (#1033) --- .../scala/fr/acinq/eclair/api/JsonSerializers.scala | 10 ++++++++-- .../fr/acinq/eclair/api/JsonSerializersSpec.scala | 12 ++++++++++-- 2 files changed, 18 insertions(+), 4 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/api/JsonSerializers.scala b/eclair-core/src/main/scala/fr/acinq/eclair/api/JsonSerializers.scala index 82c8ca10f..a0e6c984d 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/api/JsonSerializers.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/api/JsonSerializers.scala @@ -85,11 +85,17 @@ class ScalarSerializer extends CustomSerializer[Scalar](format => ({ null }, { })) class TransactionSerializer extends CustomSerializer[TransactionWithInputInfo](ser = format => ({ null }, { - case x: Transaction => JString(x.toString()) + case x: Transaction => JObject(List( + JField("txid", JString(x.txid.toHex)), + JField("tx", JString(x.toString())) + )) })) class TransactionWithInputInfoSerializer extends CustomSerializer[TransactionWithInputInfo](ser = format => ({ null }, { - case x: TransactionWithInputInfo => JString(x.tx.toString()) + case x: TransactionWithInputInfo => JObject(List( + JField("txid", JString(x.tx.txid.toHex)), + JField("tx", JString(x.tx.toString())) + )) })) class InetSocketAddressSerializer extends CustomSerializer[InetSocketAddress](format => ({ null }, { diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/api/JsonSerializersSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/api/JsonSerializersSpec.scala index e4eb282f4..4313c853e 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/api/JsonSerializersSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/api/JsonSerializersSpec.scala @@ -19,10 +19,9 @@ package fr.acinq.eclair.api import java.net.InetAddress import java.util.UUID -import fr.acinq.bitcoin.{MilliSatoshi, OutPoint} +import fr.acinq.bitcoin.{ByteVector32, MilliSatoshi, OutPoint, Transaction} import fr.acinq.eclair._ import fr.acinq.eclair.payment.{PaymentRequest, PaymentSettlingOnChain} -import fr.acinq.bitcoin.{ByteVector32, OutPoint} import fr.acinq.eclair.api.JsonSupport.CustomTypeHints import fr.acinq.eclair.payment.PaymentRequest import fr.acinq.eclair.transactions.{IN, OUT} @@ -82,4 +81,13 @@ class JsonSerializersSpec extends FunSuite with Matchers { val e1 = PaymentSettlingOnChain(UUID.randomUUID, MilliSatoshi(42), randomBytes32) assert(Serialization.writePretty(e1).contains("\"type\" : \"payment-settling-onchain\"")) } + + test("transaction serializer") { + implicit val formats = JsonSupport.formats + + val tx = Transaction.read("0200000001c8a8934fb38a44b969528252bc37be66ee166c7897c57384d1e561449e110c93010000006b483045022100dc6c50f445ed53d2fb41067fdcb25686fe79492d90e6e5db43235726ace247210220773d35228af0800c257970bee9cf75175d75217de09a8ecd83521befd040c4ca012102082b751372fe7e3b012534afe0bb8d1f2f09c724b1a10a813ce704e5b9c217ccfdffffff0247ba2300000000001976a914f97a7641228e6b17d4b0b08252ae75bd62a95fe788ace3de24000000000017a914a9fefd4b9a9282a1d7a17d2f14ac7d1eb88141d287f7d50800") + + assert(JsonSupport.serialization.write(tx) == "{\"txid\":\"3ef63b5d297c9dcf93f33b45b9f102733c36e8ef61da1ccf2bc132a10584be18\",\"tx\":\"0200000001c8a8934fb38a44b969528252bc37be66ee166c7897c57384d1e561449e110c93010000006b483045022100dc6c50f445ed53d2fb41067fdcb25686fe79492d90e6e5db43235726ace247210220773d35228af0800c257970bee9cf75175d75217de09a8ecd83521befd040c4ca012102082b751372fe7e3b012534afe0bb8d1f2f09c724b1a10a813ce704e5b9c217ccfdffffff0247ba2300000000001976a914f97a7641228e6b17d4b0b08252ae75bd62a95fe788ace3de24000000000017a914a9fefd4b9a9282a1d7a17d2f14ac7d1eb88141d287f7d50800\"}") + + } } From 31022ceca52e5e16e0353de3cdbb3c3330d2e574 Mon Sep 17 00:00:00 2001 From: Fabrice Drouin Date: Fri, 14 Jun 2019 11:52:01 +0200 Subject: [PATCH 08/21] Use compact encoding for signatures (64 bytes) instead of DER (#1014) * use 64B representation instead of DER for sigs It is more compact, and as an added bonus it frees us from the completely unrelated Bitcoin-specific `0x01` trailing sig hash. Note that we already used the 64B representation for storage everywhere, except in `ChannelCodecs.htlcTxAndSigsCodec`, which required a backward compatibility codec. Added a nonreg test for this. * Use updated secp256k1 JNI bindings * Replace scalar with private key and point with public key We now use the simplified/unified design proposed in bitcoin-lib where: - there are no more specific types for scalar/point - private and public keys are compressed unless explicitly requested * Generate and use 32 bytes seeds (and not 33) We used serialized random private keys which were represented a 33 bytes (with a 01 suffix). Using random 32 bytes values is more consistent. We must make sure that upgraded apps that already have a 33 bytes seed will still generate the same secrets, which is why LocalKeyManager still uses the 01 suffix when needed --- .../scala/fr/acinq/eclair/NodeParams.scala | 2 +- .../fr/acinq/eclair/api/JsonSerializers.scala | 19 ++- .../fr/acinq/eclair/api/OldService.scala | 2 +- .../blockchain/electrum/ElectrumWallet.scala | 4 +- .../blockchain/electrum/ElectrumWatcher.scala | 2 +- .../fr/acinq/eclair/channel/Channel.scala | 9 +- .../eclair/channel/ChannelExceptions.scala | 4 +- .../acinq/eclair/channel/ChannelTypes.scala | 14 +-- .../fr/acinq/eclair/channel/Commitments.scala | 18 +-- .../fr/acinq/eclair/channel/Helpers.scala | 20 ++-- .../fr/acinq/eclair/crypto/Generators.scala | 32 ++--- .../fr/acinq/eclair/crypto/KeyManager.scala | 16 +-- .../acinq/eclair/crypto/LocalKeyManager.scala | 18 +-- .../scala/fr/acinq/eclair/crypto/Noise.scala | 3 +- .../scala/fr/acinq/eclair/crypto/Sphinx.scala | 16 +-- .../eclair/db/sqlite/SqliteAuditDb.scala | 8 +- .../eclair/db/sqlite/SqliteNetworkDb.scala | 8 +- .../eclair/db/sqlite/SqlitePeersDb.scala | 6 +- .../fr/acinq/eclair/io/Authenticator.scala | 4 +- .../main/scala/fr/acinq/eclair/package.scala | 4 +- .../eclair/payment/PaymentLifecycle.scala | 16 +-- .../acinq/eclair/payment/PaymentRequest.scala | 18 ++- .../acinq/eclair/router/Announcements.scala | 27 +++-- .../scala/fr/acinq/eclair/router/Router.scala | 10 +- .../acinq/eclair/transactions/Scripts.scala | 103 ++++------------- .../eclair/transactions/Transactions.scala | 85 ++++++++------ .../fr/acinq/eclair/wire/ChannelCodecs.scala | 30 +++-- .../eclair/wire/LightningMessageCodecs.scala | 109 ++++++------------ .../eclair/wire/LightningMessageTypes.scala | 60 +++++----- .../src/test/resources/normal_data_htlcs.bin | 1 + .../scala/fr/acinq/eclair/PackageSpec.scala | 4 +- .../scala/fr/acinq/eclair/StartupSpec.scala | 2 +- .../scala/fr/acinq/eclair/TestConstants.scala | 4 +- .../fr/acinq/eclair/api/ApiServiceSpec.scala | 2 +- .../acinq/eclair/blockchain/TestWallet.scala | 12 -- .../electrum/ElectrumWalletBasicSpec.scala | 4 +- .../electrum/ElectrumWatcherSpec.scala | 4 +- .../b/WaitForFundingSignedStateSpec.scala | 4 +- .../channel/states/e/NormalStateSpec.scala | 28 ++--- .../channel/states/e/OfflineStateSpec.scala | 10 +- .../channel/states/f/ShutdownStateSpec.scala | 24 ++-- .../states/g/NegotiatingStateSpec.scala | 4 +- .../channel/states/h/ClosingStateSpec.scala | 8 +- .../acinq/eclair/crypto/GeneratorsSpec.scala | 19 +-- .../eclair/crypto/LocalKeyManagerSpec.scala | 19 ++- .../fr/acinq/eclair/crypto/SphinxSpec.scala | 12 +- .../fr/acinq/eclair/db/ChannelStateSpec.scala | 43 ++++++- .../acinq/eclair/db/SqliteNetworkDbSpec.scala | 4 +- .../eclair/integration/IntegrationSpec.scala | 2 +- .../scala/fr/acinq/eclair/io/PeerSpec.scala | 3 +- .../acinq/eclair/io/PeerSpecWithLogging.scala | 5 +- .../eclair/payment/HtlcGenerationSpec.scala | 2 +- .../eclair/payment/PaymentRequestSpec.scala | 2 +- .../acinq/eclair/router/BaseRouterSpec.scala | 2 +- .../eclair/router/RouteCalculationSpec.scala | 26 ++--- .../transactions/ClaimReceivedHtlcSpec.scala | 36 ++++-- .../transactions/ClaimSentHtlcSpec.scala | 35 ++++-- .../eclair/transactions/TestVectorsSpec.scala | 26 ++--- .../transactions/TransactionsSpec.scala | 34 +++--- .../acinq/eclair/wire/ChannelCodecsSpec.scala | 8 +- .../wire/FailureMessageCodecsSpec.scala | 8 +- .../wire/LightningMessageCodecsSpec.scala | 89 ++++---------- pom.xml | 2 +- 63 files changed, 554 insertions(+), 601 deletions(-) create mode 100644 eclair-core/src/test/resources/normal_data_htlcs.bin diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala b/eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala index c1ad5b342..c5977a481 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/NodeParams.scala @@ -109,7 +109,7 @@ object NodeParams { case true => ByteVector(Files.readAllBytes(seedPath.toPath)) case false => datadir.mkdirs() - val seed = randomKey.toBin + val seed = randomBytes32 Files.write(seedPath.toPath, seed.toArray) seed } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/api/JsonSerializers.scala b/eclair-core/src/main/scala/fr/acinq/eclair/api/JsonSerializers.scala index a0e6c984d..3ccecc466 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/api/JsonSerializers.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/api/JsonSerializers.scala @@ -18,11 +18,12 @@ package fr.acinq.eclair.api import java.net.InetSocketAddress import java.util.UUID + import com.google.common.net.HostAndPort import de.heikoseeberger.akkahttpjson4s.Json4sSupport import de.heikoseeberger.akkahttpjson4s.Json4sSupport.ShouldWritePretty -import fr.acinq.bitcoin.Crypto.{Point, PrivateKey, PublicKey, Scalar} -import fr.acinq.bitcoin.{ByteVector32, MilliSatoshi, OutPoint, Transaction} +import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey} +import fr.acinq.bitcoin.{ByteVector32, ByteVector64, MilliSatoshi, OutPoint, Transaction} import fr.acinq.eclair.channel.State import fr.acinq.eclair.crypto.ShaChain import fr.acinq.eclair.db.OutgoingPaymentStatus @@ -48,6 +49,10 @@ class ByteVector32Serializer extends CustomSerializer[ByteVector32](format => ({ case x: ByteVector32 => JString(x.toHex) })) +class ByteVector64Serializer extends CustomSerializer[ByteVector64](format => ({ null }, { + case x: ByteVector64 => JString(x.toHex) +})) + class UInt64Serializer extends CustomSerializer[UInt64](format => ({ null }, { case x: UInt64 => JInt(x.toBigInt) })) @@ -76,13 +81,6 @@ class PrivateKeySerializer extends CustomSerializer[PrivateKey](format => ({ nul case x: PrivateKey => JString("XXX") })) -class PointSerializer extends CustomSerializer[Point](format => ({ null }, { - case x: Point => JString(x.toString()) -})) - -class ScalarSerializer extends CustomSerializer[Scalar](format => ({ null }, { - case x: Scalar => JString("XXX") -})) class TransactionSerializer extends CustomSerializer[TransactionWithInputInfo](ser = format => ({ null }, { case x: Transaction => JObject(List( @@ -184,6 +182,7 @@ object JsonSupport extends Json4sSupport { implicit val formats = org.json4s.DefaultFormats + new ByteVectorSerializer + new ByteVector32Serializer + + new ByteVector64Serializer + new UInt64Serializer + new MilliSatoshiSerializer + new ShortChannelIdSerializer + @@ -191,8 +190,6 @@ object JsonSupport extends Json4sSupport { new ShaChainSerializer + new PublicKeySerializer + new PrivateKeySerializer + - new ScalarSerializer + - new PointSerializer + new TransactionSerializer + new TransactionWithInputInfoSerializer + new InetSocketAddressSerializer + diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/api/OldService.scala b/eclair-core/src/main/scala/fr/acinq/eclair/api/OldService.scala index a237af9e7..89910d457 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/api/OldService.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/api/OldService.scala @@ -73,7 +73,7 @@ trait OldService extends Logging { def scheduler: Scheduler implicit val serialization = jackson.Serialization - implicit val formats = org.json4s.DefaultFormats + new ByteVectorSerializer + new ByteVector32Serializer + new UInt64Serializer + new MilliSatoshiSerializer + new ShortChannelIdSerializer + new StateSerializer + new ShaChainSerializer + new PublicKeySerializer + new PrivateKeySerializer + new ScalarSerializer + new PointSerializer + new TransactionSerializer + new TransactionWithInputInfoSerializer + new InetSocketAddressSerializer + new OutPointSerializer + new OutPointKeySerializer + new InputInfoSerializer + new ColorSerializer + new RouteResponseSerializer + new ThrowableSerializer + new FailureMessageSerializer + new NodeAddressSerializer + new DirectionSerializer +new PaymentRequestSerializer + implicit val formats = org.json4s.DefaultFormats + new ByteVectorSerializer + new ByteVector32Serializer + new ByteVector64Serializer + new UInt64Serializer + new MilliSatoshiSerializer + new ShortChannelIdSerializer + new StateSerializer + new ShaChainSerializer + new PublicKeySerializer + new PrivateKeySerializer + new TransactionSerializer + new TransactionWithInputInfoSerializer + new InetSocketAddressSerializer + new OutPointSerializer + new OutPointKeySerializer + new InputInfoSerializer + new ColorSerializer + new RouteResponseSerializer + new ThrowableSerializer + new FailureMessageSerializer + new NodeAddressSerializer + new DirectionSerializer +new PaymentRequestSerializer implicit val timeout = Timeout(60 seconds) implicit val shouldWritePretty: ShouldWritePretty = ShouldWritePretty.True diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/electrum/ElectrumWallet.scala b/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/electrum/ElectrumWallet.scala index 3ea0db2b7..912fd1206 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/electrum/ElectrumWallet.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/electrum/ElectrumWallet.scala @@ -930,7 +930,7 @@ object ElectrumWallet { // we use dummy signature here, because the result is only used to estimate fees val sig = ByteVector.fill(71)(1) val sigScript = Script.write(OP_PUSHDATA(Script.write(Script.pay2wpkh(utxo.key.publicKey))) :: Nil) - val witness = ScriptWitness(sig :: utxo.key.publicKey.toBin :: Nil) + val witness = ScriptWitness(sig :: utxo.key.publicKey.value :: Nil) TxIn(utxo.outPoint, signatureScript = sigScript, sequence = TxIn.SEQUENCE_FINAL, witness = witness) }) @@ -1014,7 +1014,7 @@ object ElectrumWallet { val key = utxo.key val sig = Transaction.signInput(tx, i, Script.pay2pkh(key.publicKey), SIGHASH_ALL, Satoshi(utxo.item.value), SigVersion.SIGVERSION_WITNESS_V0, key.privateKey) val sigScript = Script.write(OP_PUSHDATA(Script.write(Script.pay2wpkh(key.publicKey))) :: Nil) - val witness = ScriptWitness(sig :: key.publicKey.toBin :: Nil) + val witness = ScriptWitness(sig :: key.publicKey.value :: Nil) txIn.copy(signatureScript = sigScript, witness = witness) }) } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/electrum/ElectrumWatcher.scala b/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/electrum/ElectrumWatcher.scala index 608717b22..f0195096f 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/electrum/ElectrumWatcher.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/electrum/ElectrumWatcher.scala @@ -37,7 +37,7 @@ class ElectrumWatcher(client: ActorRef) extends Actor with Stash with ActorLoggi override def unhandled(message: Any): Unit = message match { case ValidateRequest(c) => log.info(s"blindly validating channel=$c") - val pubkeyScript = Script.write(Script.pay2wsh(Scripts.multiSig2of2(PublicKey(c.bitcoinKey1), PublicKey(c.bitcoinKey2)))) + val pubkeyScript = Script.write(Script.pay2wsh(Scripts.multiSig2of2(c.bitcoinKey1, c.bitcoinKey2))) val TxCoordinates(_, _, outputIndex) = ShortChannelId.coordinates(c.shortChannelId) val fakeFundingTx = Transaction( version = 2, 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 863b91d89..213306add 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 @@ -19,7 +19,7 @@ package fr.acinq.eclair.channel import akka.actor.{ActorRef, FSM, OneForOneStrategy, Props, Status, SupervisorStrategy} import akka.event.Logging.MDC import akka.pattern.pipe -import fr.acinq.bitcoin.Crypto.{PublicKey, Scalar, sha256} +import fr.acinq.bitcoin.Crypto.{PublicKey, PrivateKey, sha256} import fr.acinq.bitcoin._ import fr.acinq.eclair._ import fr.acinq.eclair.blockchain._ @@ -30,7 +30,6 @@ import fr.acinq.eclair.payment._ import fr.acinq.eclair.router.Announcements import fr.acinq.eclair.transactions._ import fr.acinq.eclair.wire.{ChannelReestablish, _} -import scodec.bits.ByteVector import scala.compat.Platform import scala.concurrent.ExecutionContext @@ -224,7 +223,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId // we rebuild a new channel_update with values from the configuration because they may have changed while eclair was down val candidateChannelUpdate = Announcements.makeChannelUpdate(nodeParams.chainHash, nodeParams.privateKey, remoteNodeId, normal.channelUpdate.shortChannelId, nodeParams.expiryDeltaBlocks, normal.commitments.remoteParams.htlcMinimumMsat, normal.channelUpdate.feeBaseMsat, normal.channelUpdate.feeProportionalMillionths, normal.commitments.localCommit.spec.totalFunds, enable = Announcements.isEnabled(normal.channelUpdate.channelFlags)) - val channelUpdate1 = if (candidateChannelUpdate.copy(signature = ByteVector.empty, timestamp = 0) == normal.channelUpdate.copy(signature = ByteVector.empty, timestamp = 0)) { + val channelUpdate1 = if (Announcements.areSame(candidateChannelUpdate, normal.channelUpdate)) { // if there was no configuration change we keep the existing channel update normal.channelUpdate } else { @@ -929,7 +928,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId val age = Platform.currentTime.milliseconds - d.channelUpdate.timestamp.seconds val channelUpdate1 = Announcements.makeChannelUpdate(nodeParams.chainHash, nodeParams.privateKey, remoteNodeId, d.shortChannelId, d.channelUpdate.cltvExpiryDelta, d.channelUpdate.htlcMinimumMsat, d.channelUpdate.feeBaseMsat, d.channelUpdate.feeProportionalMillionths, d.commitments.localCommit.spec.totalFunds, enable = Helpers.aboveReserve(d.commitments)) reason match { - case Reconnected if channelUpdate1.copy(signature = ByteVector.empty, timestamp = 0) == d.channelUpdate.copy(signature = ByteVector.empty, timestamp = 0) && age < REFRESH_CHANNEL_UPDATE_INTERVAL => + case Reconnected if Announcements.areSame(channelUpdate1, d.channelUpdate) && age < REFRESH_CHANNEL_UPDATE_INTERVAL => // we already sent an identical channel_update not long ago (flapping protection in case we keep being disconnected/reconnected) log.info(s"not sending a new identical channel_update, current one was created {} days ago", age.toDays) stay @@ -1414,7 +1413,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId channelId = d.channelId, nextLocalCommitmentNumber = d.commitments.localCommit.index + 1, nextRemoteRevocationNumber = d.commitments.remoteCommit.index, - yourLastPerCommitmentSecret = Some(Scalar(yourLastPerCommitmentSecret)), + yourLastPerCommitmentSecret = Some(PrivateKey(yourLastPerCommitmentSecret)), myCurrentPerCommitmentPoint = Some(myCurrentPerCommitmentPoint) ) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelExceptions.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelExceptions.scala index ab5efa9a1..f5b45e77c 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelExceptions.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelExceptions.scala @@ -16,7 +16,7 @@ package fr.acinq.eclair.channel -import fr.acinq.bitcoin.Crypto.Scalar +import fr.acinq.bitcoin.Crypto.PrivateKey import fr.acinq.bitcoin.{ByteVector32, Transaction} import fr.acinq.eclair.{ShortChannelId, UInt64} import fr.acinq.eclair.payment.Origin @@ -74,7 +74,7 @@ case class CannotSignWithoutChanges (override val channelId: ByteVect case class CannotSignBeforeRevocation (override val channelId: ByteVector32) extends ChannelException(channelId, "cannot sign until next revocation hash is received") case class UnexpectedRevocation (override val channelId: ByteVector32) extends ChannelException(channelId, "received unexpected RevokeAndAck message") case class InvalidRevocation (override val channelId: ByteVector32) extends ChannelException(channelId, "invalid revocation") -case class InvalidRevokedCommitProof (override val channelId: ByteVector32, ourCommitmentNumber: Long, theirCommitmentNumber: Long, perCommitmentSecret: Scalar) extends ChannelException(channelId, s"counterparty claimed that we have a revoked commit but their proof doesn't check out: ourCommitmentNumber=$ourCommitmentNumber theirCommitmentNumber=$theirCommitmentNumber perCommitmentSecret=$perCommitmentSecret") +case class InvalidRevokedCommitProof (override val channelId: ByteVector32, ourCommitmentNumber: Long, theirCommitmentNumber: Long, perCommitmentSecret: PrivateKey) extends ChannelException(channelId, s"counterparty claimed that we have a revoked commit but their proof doesn't check out: ourCommitmentNumber=$ourCommitmentNumber theirCommitmentNumber=$theirCommitmentNumber perCommitmentSecret=$perCommitmentSecret") case class CommitmentSyncError (override val channelId: ByteVector32) extends ChannelException(channelId, "commitment sync error") case class RevocationSyncError (override val channelId: ByteVector32) extends ChannelException(channelId, "revocation sync error") case class InvalidFailureCode (override val channelId: ByteVector32) extends ChannelException(channelId, "UpdateFailMalformedHtlc message doesn't have BADONION bit set") diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelTypes.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelTypes.scala index e538eceea..1c65b9016 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelTypes.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/ChannelTypes.scala @@ -19,7 +19,7 @@ package fr.acinq.eclair.channel import java.util.UUID import akka.actor.ActorRef -import fr.acinq.bitcoin.Crypto.{Point, PublicKey} +import fr.acinq.bitcoin.Crypto.{PublicKey} import fr.acinq.bitcoin.{ByteVector32, DeterministicWallet, OutPoint, Satoshi, Transaction} import fr.acinq.eclair.crypto.Sphinx import fr.acinq.eclair.transactions.CommitmentSpec @@ -149,8 +149,8 @@ case class RevokedCommitPublished(commitTx: Transaction, claimMainOutputTx: Opti final case class DATA_WAIT_FOR_OPEN_CHANNEL(initFundee: INPUT_INIT_FUNDEE) extends Data final case class DATA_WAIT_FOR_ACCEPT_CHANNEL(initFunder: INPUT_INIT_FUNDER, lastSent: OpenChannel) extends Data -final case class DATA_WAIT_FOR_FUNDING_INTERNAL(temporaryChannelId: ByteVector32, localParams: LocalParams, remoteParams: RemoteParams, fundingSatoshis: Long, pushMsat: Long, initialFeeratePerKw: Long, remoteFirstPerCommitmentPoint: Point, lastSent: OpenChannel) extends Data -final case class DATA_WAIT_FOR_FUNDING_CREATED(temporaryChannelId: ByteVector32, localParams: LocalParams, remoteParams: RemoteParams, fundingSatoshis: Long, pushMsat: Long, initialFeeratePerKw: Long, remoteFirstPerCommitmentPoint: Point, channelFlags: Byte, lastSent: AcceptChannel) extends Data +final case class DATA_WAIT_FOR_FUNDING_INTERNAL(temporaryChannelId: ByteVector32, localParams: LocalParams, remoteParams: RemoteParams, fundingSatoshis: Long, pushMsat: Long, initialFeeratePerKw: Long, remoteFirstPerCommitmentPoint: PublicKey, lastSent: OpenChannel) extends Data +final case class DATA_WAIT_FOR_FUNDING_CREATED(temporaryChannelId: ByteVector32, localParams: LocalParams, remoteParams: RemoteParams, fundingSatoshis: Long, pushMsat: Long, initialFeeratePerKw: Long, remoteFirstPerCommitmentPoint: PublicKey, channelFlags: Byte, lastSent: AcceptChannel) extends Data final case class DATA_WAIT_FOR_FUNDING_SIGNED(channelId: ByteVector32, localParams: LocalParams, remoteParams: RemoteParams, fundingTx: Transaction, fundingTxFee: Satoshi, localSpec: CommitmentSpec, localCommitTx: CommitTx, remoteCommit: RemoteCommit, channelFlags: Byte, lastSent: FundingCreated) extends Data final case class DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments: Commitments, fundingTx: Option[Transaction], @@ -209,10 +209,10 @@ final case class RemoteParams(nodeId: PublicKey, toSelfDelay: Int, maxAcceptedHtlcs: Int, fundingPubKey: PublicKey, - revocationBasepoint: Point, - paymentBasepoint: Point, - delayedPaymentBasepoint: Point, - htlcBasepoint: Point, + revocationBasepoint: PublicKey, + paymentBasepoint: PublicKey, + delayedPaymentBasepoint: PublicKey, + htlcBasepoint: PublicKey, globalFeatures: ByteVector, localFeatures: ByteVector) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Commitments.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Commitments.scala index ee0e47f09..8d15614fb 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Commitments.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Commitments.scala @@ -17,8 +17,8 @@ package fr.acinq.eclair.channel import akka.event.LoggingAdapter -import fr.acinq.bitcoin.Crypto.{Point, PrivateKey, sha256} -import fr.acinq.bitcoin.{ByteVector32, Crypto, Satoshi} +import fr.acinq.bitcoin.Crypto.{PublicKey, PrivateKey, sha256} +import fr.acinq.bitcoin.{ByteVector32, ByteVector64, Crypto, Satoshi} import fr.acinq.eclair.crypto.{Generators, KeyManager, ShaChain, Sphinx} import fr.acinq.eclair.payment._ import fr.acinq.eclair.transactions.Transactions._ @@ -35,10 +35,10 @@ case class LocalChanges(proposed: List[UpdateMessage], signed: List[UpdateMessag } case class RemoteChanges(proposed: List[UpdateMessage], acked: List[UpdateMessage], signed: List[UpdateMessage]) case class Changes(ourChanges: LocalChanges, theirChanges: RemoteChanges) -case class HtlcTxAndSigs(txinfo: TransactionWithInputInfo, localSig: ByteVector, remoteSig: ByteVector) +case class HtlcTxAndSigs(txinfo: TransactionWithInputInfo, localSig: ByteVector64, remoteSig: ByteVector64) case class PublishableTxs(commitTx: CommitTx, htlcTxsAndSigs: List[HtlcTxAndSigs]) case class LocalCommit(index: Long, spec: CommitmentSpec, publishableTxs: PublishableTxs) -case class RemoteCommit(index: Long, spec: CommitmentSpec, txid: ByteVector32, remotePerCommitmentPoint: Point) +case class RemoteCommit(index: Long, spec: CommitmentSpec, txid: ByteVector32, remotePerCommitmentPoint: PublicKey) case class WaitingForRevocation(nextRemoteCommit: RemoteCommit, sent: CommitSig, sentAfterLocalCommitIndex: Long, reSignAsap: Boolean = false) // @formatter:on @@ -56,7 +56,7 @@ case class Commitments(localParams: LocalParams, remoteParams: RemoteParams, localChanges: LocalChanges, remoteChanges: RemoteChanges, localNextHtlcId: Long, remoteNextHtlcId: Long, originChannels: Map[Long, Origin], // for outgoing htlcs relayed through us, the id of the previous channel - remoteNextCommitInfo: Either[WaitingForRevocation, Point], + remoteNextCommitInfo: Either[WaitingForRevocation, PublicKey], commitInput: InputInfo, remotePerCommitmentSecrets: ShaChain, channelId: ByteVector32) { @@ -466,7 +466,7 @@ object Commitments { import commitments._ // we receive a revocation because we just sent them a sig for their next commit tx remoteNextCommitInfo match { - case Left(_) if revocation.perCommitmentSecret.toPoint != remoteCommit.remotePerCommitmentPoint => + case Left(_) if revocation.perCommitmentSecret.publicKey != remoteCommit.remotePerCommitmentPoint => throw InvalidRevocation(commitments.channelId) case Left(WaitingForRevocation(theirNextCommit, _, _, _)) => val forwards = commitments.remoteChanges.signed collect { @@ -495,7 +495,7 @@ object Commitments { remoteChanges = remoteChanges.copy(signed = Nil), remoteCommit = theirNextCommit, remoteNextCommitInfo = Right(revocation.nextPerCommitmentPoint), - remotePerCommitmentSecrets = commitments.remotePerCommitmentSecrets.addHash(revocation.perCommitmentSecret.toBin, 0xFFFFFFFFFFFFL - commitments.remoteCommit.index), + remotePerCommitmentSecrets = commitments.remotePerCommitmentSecrets.addHash(revocation.perCommitmentSecret.value, 0xFFFFFFFFFFFFL - commitments.remoteCommit.index), originChannels = originChannels1) (commitments1, forwards) case Right(_) => @@ -503,7 +503,7 @@ object Commitments { } } - def makeLocalTxs(keyManager: KeyManager, commitTxNumber: Long, localParams: LocalParams, remoteParams: RemoteParams, commitmentInput: InputInfo, localPerCommitmentPoint: Point, spec: CommitmentSpec): (CommitTx, Seq[HtlcTimeoutTx], Seq[HtlcSuccessTx]) = { + def makeLocalTxs(keyManager: KeyManager, commitTxNumber: Long, localParams: LocalParams, remoteParams: RemoteParams, commitmentInput: InputInfo, localPerCommitmentPoint: PublicKey, spec: CommitmentSpec): (CommitTx, Seq[HtlcTimeoutTx], Seq[HtlcSuccessTx]) = { val localDelayedPaymentPubkey = Generators.derivePubKey(keyManager.delayedPaymentPoint(localParams.channelKeyPath).publicKey, localPerCommitmentPoint) val localHtlcPubkey = Generators.derivePubKey(keyManager.htlcPoint(localParams.channelKeyPath).publicKey, localPerCommitmentPoint) val remotePaymentPubkey = Generators.derivePubKey(remoteParams.paymentBasepoint, localPerCommitmentPoint) @@ -514,7 +514,7 @@ object Commitments { (commitTx, htlcTimeoutTxs, htlcSuccessTxs) } - def makeRemoteTxs(keyManager: KeyManager, commitTxNumber: Long, localParams: LocalParams, remoteParams: RemoteParams, commitmentInput: InputInfo, remotePerCommitmentPoint: Point, spec: CommitmentSpec): (CommitTx, Seq[HtlcTimeoutTx], Seq[HtlcSuccessTx]) = { + def makeRemoteTxs(keyManager: KeyManager, commitTxNumber: Long, localParams: LocalParams, remoteParams: RemoteParams, commitmentInput: InputInfo, remotePerCommitmentPoint: PublicKey, spec: CommitmentSpec): (CommitTx, Seq[HtlcTimeoutTx], Seq[HtlcSuccessTx]) = { val localPaymentPubkey = Generators.derivePubKey(keyManager.paymentPoint(localParams.channelKeyPath).publicKey, remotePerCommitmentPoint) val localHtlcPubkey = Generators.derivePubKey(keyManager.htlcPoint(localParams.channelKeyPath).publicKey, remotePerCommitmentPoint) val remoteDelayedPaymentPubkey = Generators.derivePubKey(remoteParams.delayedPaymentBasepoint, remotePerCommitmentPoint) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Helpers.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Helpers.scala index 55f1d9ca7..8f88d7139 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Helpers.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Helpers.scala @@ -17,7 +17,7 @@ package fr.acinq.eclair.channel import akka.event.LoggingAdapter -import fr.acinq.bitcoin.Crypto.{Point, PrivateKey, PublicKey, Scalar, ripemd160, sha256} +import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey, ripemd160, sha256} import fr.acinq.bitcoin.Script._ import fr.acinq.bitcoin.{OutPoint, _} import fr.acinq.eclair.blockchain.EclairWallet @@ -251,7 +251,7 @@ object Helpers { * @param remoteFirstPerCommitmentPoint * @return (localSpec, localTx, remoteSpec, remoteTx, fundingTxOutput) */ - def makeFirstCommitTxs(keyManager: KeyManager, temporaryChannelId: ByteVector32, localParams: LocalParams, remoteParams: RemoteParams, fundingSatoshis: Long, pushMsat: Long, initialFeeratePerKw: Long, fundingTxHash: ByteVector32, fundingTxOutputIndex: Int, remoteFirstPerCommitmentPoint: Point, maxFeerateMismatch: Double): (CommitmentSpec, CommitTx, CommitmentSpec, CommitTx) = { + def makeFirstCommitTxs(keyManager: KeyManager, temporaryChannelId: ByteVector32, localParams: LocalParams, remoteParams: RemoteParams, fundingSatoshis: Long, pushMsat: Long, initialFeeratePerKw: Long, fundingTxHash: ByteVector32, fundingTxOutputIndex: Int, remoteFirstPerCommitmentPoint: PublicKey, maxFeerateMismatch: Double): (CommitmentSpec, CommitTx, CommitmentSpec, CommitTx) = { val toLocalMsat = if (localParams.isFunder) fundingSatoshis * 1000 - pushMsat else pushMsat val toRemoteMsat = if (localParams.isFunder) pushMsat else fundingSatoshis * 1000 - pushMsat @@ -428,7 +428,7 @@ object Helpers { } // used only to compute tx weights and estimate fees - lazy val dummyPublicKey = PrivateKey(ByteVector32(ByteVector.fill(32)(1)), true).publicKey + lazy val dummyPublicKey = PrivateKey(ByteVector32(ByteVector.fill(32)(1))).publicKey def isValidFinalScriptPubkey(scriptPubKey: ByteVector): Boolean = { Try(Script.parse(scriptPubKey)) match { @@ -444,7 +444,7 @@ object Helpers { import commitments._ // this is just to estimate the weight, it depends on size of the pubkey scripts val dummyClosingTx = Transactions.makeClosingTx(commitInput, localScriptPubkey, remoteScriptPubkey, localParams.isFunder, Satoshi(0), Satoshi(0), localCommit.spec) - val closingWeight = Transaction.weight(Transactions.addSigs(dummyClosingTx, dummyPublicKey, remoteParams.fundingPubKey, ByteVector.fill(71)(0xaa), ByteVector.fill(71)(0xbb)).tx) + val closingWeight = Transaction.weight(Transactions.addSigs(dummyClosingTx, dummyPublicKey, remoteParams.fundingPubKey, Transactions.PlaceHolderSig, Transactions.PlaceHolderSig).tx) // no need to use a very high fee here, so we target 6 blocks; also, we "MUST set fee_satoshis less than or equal to the base fee of the final commitment transaction" val feeratePerKw = Math.min(Globals.feeratesPerKw.get.blocks_6, commitments.localCommit.spec.feeratePerKw) log.info(s"using feeratePerKw=$feeratePerKw for initial closing tx") @@ -473,7 +473,7 @@ object Helpers { (closingTx, closingSigned) } - def checkClosingSignature(keyManager: KeyManager, commitments: Commitments, localScriptPubkey: ByteVector, remoteScriptPubkey: ByteVector, remoteClosingFee: Satoshi, remoteClosingSig: ByteVector)(implicit log: LoggingAdapter): Try[Transaction] = { + def checkClosingSignature(keyManager: KeyManager, commitments: Commitments, localScriptPubkey: ByteVector, remoteScriptPubkey: ByteVector, remoteClosingFee: Satoshi, remoteClosingSig: ByteVector64)(implicit log: LoggingAdapter): Try[Transaction] = { import commitments._ val lastCommitFeeSatoshi = commitments.commitInput.txOut.amount.amount - commitments.localCommit.publishableTxs.commitTx.tx.txOut.map(_.amount.amount).sum if (remoteClosingFee.amount > lastCommitFeeSatoshi) { @@ -636,7 +636,7 @@ object Helpers { * @param tx the remote commitment transaction that has just been published * @return a list of transactions (one per HTLC that we can claim) */ - def claimRemoteCommitMainOutput(keyManager: KeyManager, commitments: Commitments, remotePerCommitmentPoint: Point, tx: Transaction)(implicit log: LoggingAdapter): RemoteCommitPublished = { + def claimRemoteCommitMainOutput(keyManager: KeyManager, commitments: Commitments, remotePerCommitmentPoint: PublicKey, tx: Transaction)(implicit log: LoggingAdapter): RemoteCommitPublished = { val localPubkey = Generators.derivePubKey(keyManager.paymentPoint(commitments.localParams.channelKeyPath).publicKey, remotePerCommitmentPoint) // no need to use a high fee rate for our main output (we are the only one who can spend it) @@ -677,9 +677,9 @@ object Helpers { log.warning(s"a revoked commit has been published with txnumber=$txnumber") // now we know what commit number this tx is referring to, we can derive the commitment point from the shachain remotePerCommitmentSecrets.getHash(0xFFFFFFFFFFFFL - txnumber) - .map(d => Scalar(d)) + .map(d => PrivateKey(d)) .map { remotePerCommitmentSecret => - val remotePerCommitmentPoint = remotePerCommitmentSecret.toPoint + val remotePerCommitmentPoint = remotePerCommitmentSecret.publicKey val remoteDelayedPaymentPubkey = Generators.derivePubKey(remoteParams.delayedPaymentBasepoint, remotePerCommitmentPoint) val remoteRevocationPubkey = Generators.revocationPubKey(keyManager.revocationPoint(localParams.channelKeyPath).publicKey, remotePerCommitmentPoint) val localPubkey = Generators.derivePubKey(keyManager.paymentPoint(localParams.channelKeyPath).publicKey, remotePerCommitmentPoint) @@ -766,9 +766,9 @@ object Helpers { val txnumber = Transactions.obscuredCommitTxNumber(obscuredTxNumber, !localParams.isFunder, remoteParams.paymentBasepoint, keyManager.paymentPoint(localParams.channelKeyPath).publicKey) // now we know what commit number this tx is referring to, we can derive the commitment point from the shachain remotePerCommitmentSecrets.getHash(0xFFFFFFFFFFFFL - txnumber) - .map(d => Scalar(d)) + .map(d => PrivateKey(d)) .flatMap { remotePerCommitmentSecret => - val remotePerCommitmentPoint = remotePerCommitmentSecret.toPoint + val remotePerCommitmentPoint = remotePerCommitmentSecret.publicKey val remoteDelayedPaymentPubkey = Generators.derivePubKey(remoteParams.delayedPaymentBasepoint, remotePerCommitmentPoint) val remoteRevocationPubkey = Generators.revocationPubKey(keyManager.revocationPoint(localParams.channelKeyPath).publicKey, remotePerCommitmentPoint) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/crypto/Generators.scala b/eclair-core/src/main/scala/fr/acinq/eclair/crypto/Generators.scala index 89afcb546..ae14fbdab 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/crypto/Generators.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/crypto/Generators.scala @@ -16,7 +16,7 @@ package fr.acinq.eclair.crypto -import fr.acinq.bitcoin.Crypto.{Point, PrivateKey, PublicKey, Scalar} +import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey} import fr.acinq.bitcoin.{ByteVector32, Crypto} import scodec.bits.ByteVector @@ -30,31 +30,31 @@ object Generators { case length if length < 32 => ByteVector32(data.padLeft(32)) } - def perCommitSecret(seed: ByteVector32, index: Long): Scalar = Scalar(ShaChain.shaChainFromSeed(seed, 0xFFFFFFFFFFFFL - index)) + def perCommitSecret(seed: ByteVector32, index: Long): PrivateKey = PrivateKey(ShaChain.shaChainFromSeed(seed, 0xFFFFFFFFFFFFL - index)) - def perCommitPoint(seed: ByteVector32, index: Long): Point = perCommitSecret(seed, index).toPoint + def perCommitPoint(seed: ByteVector32, index: Long): PublicKey = perCommitSecret(seed, index).publicKey - def derivePrivKey(secret: Scalar, perCommitPoint: Point): PrivateKey = { + def derivePrivKey(secret: PrivateKey, perCommitPoint: PublicKey): PrivateKey = { // secretkey = basepoint-secret + SHA256(per-commitment-point || basepoint) - PrivateKey(secret.add(Scalar(Crypto.sha256(perCommitPoint.toBin(true) ++ secret.toPoint.toBin(true)))), true) + secret.add(PrivateKey(Crypto.sha256(perCommitPoint.value ++ secret.publicKey.value))) } - def derivePubKey(basePoint: Point, perCommitPoint: Point): PublicKey = { + def derivePubKey(basePoint: PublicKey, perCommitPoint: PublicKey): PublicKey = { //pubkey = basepoint + SHA256(per-commitment-point || basepoint)*G - val a = Scalar(Crypto.sha256(perCommitPoint.toBin(true) ++ basePoint.toBin(true))) - PublicKey(basePoint.add(a.toPoint)) + val a = PrivateKey(Crypto.sha256(perCommitPoint.value ++ basePoint.value)) + basePoint.add(a.publicKey) } - def revocationPubKey(basePoint: Point, perCommitPoint: Point): PublicKey = { - val a = Scalar(Crypto.sha256(basePoint.toBin(true) ++ perCommitPoint.toBin(true))) - val b = Scalar(Crypto.sha256(perCommitPoint.toBin(true) ++ basePoint.toBin(true))) - PublicKey(basePoint.multiply(a).add(perCommitPoint.multiply(b))) + def revocationPubKey(basePoint: PublicKey, perCommitPoint: PublicKey): PublicKey = { + val a = PrivateKey(Crypto.sha256(basePoint.value ++ perCommitPoint.value)) + val b = PrivateKey(Crypto.sha256(perCommitPoint.value ++ basePoint.value)) + basePoint.multiply(a).add(perCommitPoint.multiply(b)) } - def revocationPrivKey(secret: Scalar, perCommitSecret: Scalar): PrivateKey = { - val a = Scalar(Crypto.sha256(secret.toPoint.toBin(true) ++ perCommitSecret.toPoint.toBin(true))) - val b = Scalar(Crypto.sha256(perCommitSecret.toPoint.toBin(true) ++ secret.toPoint.toBin(true))) - PrivateKey(secret.multiply(a).add(perCommitSecret.multiply(b)), true) + def revocationPrivKey(secret: PrivateKey, perCommitSecret: PrivateKey): PrivateKey = { + val a = PrivateKey(Crypto.sha256(secret.publicKey.value ++ perCommitSecret.publicKey.value)) + val b = PrivateKey(Crypto.sha256(perCommitSecret.publicKey.value ++ secret.publicKey.value)) + secret.multiply(a).add(perCommitSecret.multiply(b)) } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/crypto/KeyManager.scala b/eclair-core/src/main/scala/fr/acinq/eclair/crypto/KeyManager.scala index 34f73f615..f96cb9d62 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/crypto/KeyManager.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/crypto/KeyManager.scala @@ -16,9 +16,9 @@ package fr.acinq.eclair.crypto -import fr.acinq.bitcoin.Crypto.{Point, PublicKey, Scalar} +import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey} import fr.acinq.bitcoin.DeterministicWallet.ExtendedPublicKey -import fr.acinq.bitcoin.{ByteVector32, Crypto, DeterministicWallet} +import fr.acinq.bitcoin.{ByteVector32, ByteVector64, Crypto, DeterministicWallet} import fr.acinq.eclair.ShortChannelId import fr.acinq.eclair.transactions.Transactions.TransactionWithInputInfo import scodec.bits.ByteVector @@ -38,9 +38,9 @@ trait KeyManager { def htlcPoint(channelKeyPath: DeterministicWallet.KeyPath): ExtendedPublicKey - def commitmentSecret(channelKeyPath: DeterministicWallet.KeyPath, index: Long): Crypto.Scalar + def commitmentSecret(channelKeyPath: DeterministicWallet.KeyPath, index: Long): Crypto.PrivateKey - def commitmentPoint(channelKeyPath: DeterministicWallet.KeyPath, index: Long): Crypto.Point + def commitmentPoint(channelKeyPath: DeterministicWallet.KeyPath, index: Long): Crypto.PublicKey /** * @@ -49,7 +49,7 @@ trait KeyManager { * @return a signature generated with the private key that matches the input * extended public key */ - def sign(tx: TransactionWithInputInfo, publicKey: ExtendedPublicKey): ByteVector + def sign(tx: TransactionWithInputInfo, publicKey: ExtendedPublicKey): ByteVector64 /** * This method is used to spend funds send to htlc keys/delayed keys @@ -60,7 +60,7 @@ trait KeyManager { * @return a signature generated with a private key generated from the input keys's matching * private key and the remote point. */ - def sign(tx: TransactionWithInputInfo, publicKey: ExtendedPublicKey, remotePoint: Point): ByteVector + def sign(tx: TransactionWithInputInfo, publicKey: ExtendedPublicKey, remotePoint: PublicKey): ByteVector64 /** * Ths method is used to spend revoked transactions, with the corresponding revocation key @@ -71,7 +71,7 @@ trait KeyManager { * @return a signature generated with a private key generated from the input keys's matching * private key and the remote secret. */ - def sign(tx: TransactionWithInputInfo, publicKey: ExtendedPublicKey, remoteSecret: Scalar): ByteVector + def sign(tx: TransactionWithInputInfo, publicKey: ExtendedPublicKey, remoteSecret: PrivateKey): ByteVector64 - def signChannelAnnouncement(channelKeyPath: DeterministicWallet.KeyPath, chainHash: ByteVector32, shortChannelId: ShortChannelId, remoteNodeId: PublicKey, remoteFundingKey: PublicKey, features: ByteVector): (ByteVector, ByteVector) + def signChannelAnnouncement(channelKeyPath: DeterministicWallet.KeyPath, chainHash: ByteVector32, shortChannelId: ShortChannelId, remoteNodeId: PublicKey, remoteFundingKey: PublicKey, features: ByteVector): (ByteVector64, ByteVector64) } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/crypto/LocalKeyManager.scala b/eclair-core/src/main/scala/fr/acinq/eclair/crypto/LocalKeyManager.scala index 558340192..330869834 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/crypto/LocalKeyManager.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/crypto/LocalKeyManager.scala @@ -17,9 +17,9 @@ package fr.acinq.eclair.crypto import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} -import fr.acinq.bitcoin.Crypto.{Point, PublicKey, Scalar} +import fr.acinq.bitcoin.Crypto.{PublicKey, PrivateKey} import fr.acinq.bitcoin.DeterministicWallet.{derivePrivateKey, _} -import fr.acinq.bitcoin.{Block, ByteVector32, Crypto, DeterministicWallet} +import fr.acinq.bitcoin.{Block, ByteVector32, ByteVector64, Crypto, DeterministicWallet} import fr.acinq.eclair.ShortChannelId import fr.acinq.eclair.router.Announcements import fr.acinq.eclair.transactions.Transactions @@ -78,7 +78,7 @@ class LocalKeyManager(seed: ByteVector, chainHash: ByteVector32) extends KeyMana private def htlcSecret(channelKeyPath: DeterministicWallet.KeyPath) = privateKeys.get(internalKeyPath(channelKeyPath, hardened(4))) - private def shaSeed(channelKeyPath: DeterministicWallet.KeyPath) = Crypto.sha256(privateKeys.get(internalKeyPath(channelKeyPath, hardened(5))).privateKey.toBin) + private def shaSeed(channelKeyPath: DeterministicWallet.KeyPath) = Crypto.sha256(privateKeys.get(internalKeyPath(channelKeyPath, hardened(5))).privateKey.value :+ 1.toByte) override def fundingPublicKey(channelKeyPath: DeterministicWallet.KeyPath) = publicKeys.get(internalKeyPath(channelKeyPath, hardened(0))) @@ -101,7 +101,7 @@ class LocalKeyManager(seed: ByteVector, chainHash: ByteVector32) extends KeyMana * @return a signature generated with the private key that matches the input * extended public key */ - def sign(tx: TransactionWithInputInfo, publicKey: ExtendedPublicKey): ByteVector = { + def sign(tx: TransactionWithInputInfo, publicKey: ExtendedPublicKey): ByteVector64 = { val privateKey = privateKeys.get(publicKey.path) Transactions.sign(tx, privateKey.privateKey) } @@ -115,7 +115,7 @@ class LocalKeyManager(seed: ByteVector, chainHash: ByteVector32) extends KeyMana * @return a signature generated with a private key generated from the input keys's matching * private key and the remote point. */ - def sign(tx: TransactionWithInputInfo, publicKey: ExtendedPublicKey, remotePoint: Point): ByteVector = { + def sign(tx: TransactionWithInputInfo, publicKey: ExtendedPublicKey, remotePoint: PublicKey): ByteVector64 = { val privateKey = privateKeys.get(publicKey.path) val currentKey = Generators.derivePrivKey(privateKey.privateKey, remotePoint) Transactions.sign(tx, currentKey) @@ -130,20 +130,20 @@ class LocalKeyManager(seed: ByteVector, chainHash: ByteVector32) extends KeyMana * @return a signature generated with a private key generated from the input keys's matching * private key and the remote secret. */ - def sign(tx: TransactionWithInputInfo, publicKey: ExtendedPublicKey, remoteSecret: Scalar): ByteVector = { + def sign(tx: TransactionWithInputInfo, publicKey: ExtendedPublicKey, remoteSecret: PrivateKey): ByteVector64 = { val privateKey = privateKeys.get(publicKey.path) val currentKey = Generators.revocationPrivKey(privateKey.privateKey, remoteSecret) Transactions.sign(tx, currentKey) } - override def signChannelAnnouncement(channelKeyPath: DeterministicWallet.KeyPath, chainHash: ByteVector32, shortChannelId: ShortChannelId, remoteNodeId: PublicKey, remoteFundingKey: PublicKey, features: ByteVector): (ByteVector, ByteVector) = { + override def signChannelAnnouncement(channelKeyPath: DeterministicWallet.KeyPath, chainHash: ByteVector32, shortChannelId: ShortChannelId, remoteNodeId: PublicKey, remoteFundingKey: PublicKey, features: ByteVector): (ByteVector64, ByteVector64) = { val witness = if (Announcements.isNode1(nodeId, remoteNodeId)) { Announcements.channelAnnouncementWitnessEncode(chainHash, shortChannelId, nodeId, remoteNodeId, fundingPublicKey(channelKeyPath).publicKey, remoteFundingKey, features) } else { Announcements.channelAnnouncementWitnessEncode(chainHash, shortChannelId, remoteNodeId, nodeId, remoteFundingKey, fundingPublicKey(channelKeyPath).publicKey, features) } - val nodeSig = Crypto.encodeSignature(Crypto.sign(witness, nodeKey.privateKey)) :+ 1.toByte - val bitcoinSig = Crypto.encodeSignature(Crypto.sign(witness, fundingPrivateKey(channelKeyPath).privateKey)) :+ 1.toByte + val nodeSig = Crypto.sign(witness, nodeKey.privateKey) + val bitcoinSig = Crypto.sign(witness, fundingPrivateKey(channelKeyPath).privateKey) (nodeSig, bitcoinSig) } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/crypto/Noise.scala b/eclair-core/src/main/scala/fr/acinq/eclair/crypto/Noise.scala index 1030cc8e8..ceed60483 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/crypto/Noise.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/crypto/Noise.scala @@ -19,6 +19,7 @@ package fr.acinq.eclair.crypto import java.math.BigInteger import java.nio.ByteOrder +import fr.acinq.bitcoin.Crypto.PrivateKey import fr.acinq.bitcoin.{Crypto, Protocol} import fr.acinq.eclair.randomBytes import grizzled.slf4j.Logging @@ -54,7 +55,7 @@ object Noise { override def generateKeyPair(priv: ByteVector): KeyPair = { require(priv.length == 32) - KeyPair(Crypto.publicKeyFromPrivateKey(priv :+ 1.toByte), priv) + KeyPair(PrivateKey(priv).publicKey.value, priv) } /** diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/crypto/Sphinx.scala b/eclair-core/src/main/scala/fr/acinq/eclair/crypto/Sphinx.scala index d2f1b8361..861746a8b 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/crypto/Sphinx.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/crypto/Sphinx.scala @@ -19,7 +19,7 @@ package fr.acinq.eclair.crypto import java.io.{ByteArrayInputStream, ByteArrayOutputStream, InputStream, OutputStream} import java.nio.ByteOrder -import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey, Scalar} +import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey} import fr.acinq.bitcoin.{ByteVector32, Crypto, Protocol} import fr.acinq.eclair.wire.{FailureMessage, FailureMessageCodecs} import grizzled.slf4j.Logging @@ -72,11 +72,11 @@ object Sphinx extends Logging { def generateStream(key: ByteVector, length: Int): ByteVector = ChaCha20.encrypt(zeroes(length), key, zeroes(12)) - def computeSharedSecret(pub: PublicKey, secret: PrivateKey): ByteVector32 = Crypto.sha256(ByteVector.view(pub.multiply(secret).normalize().getEncoded(true))) + def computeSharedSecret(pub: PublicKey, secret: PrivateKey): ByteVector32 = Crypto.sha256(pub.multiply(secret).value) - def computeblindingFactor(pub: PublicKey, secret: ByteVector): ByteVector32 = Crypto.sha256(pub.toBin ++ secret) + def computeblindingFactor(pub: PublicKey, secret: ByteVector): ByteVector32 = Crypto.sha256(pub.value ++ secret) - def blind(pub: PublicKey, blindingFactor: ByteVector32): PublicKey = PublicKey(pub.multiply(Scalar(blindingFactor)).normalize(), compressed = true) + def blind(pub: PublicKey, blindingFactor: ByteVector32): PublicKey = pub.multiply(PrivateKey(blindingFactor)) def blind(pub: PublicKey, blindingFactors: Seq[ByteVector32]): PublicKey = blindingFactors.foldLeft(pub)(blind) @@ -88,7 +88,7 @@ object Sphinx extends Logging { * @return a tuple (ephemeral public keys, shared secrets) */ def computeEphemeralPublicKeysAndSharedSecrets(sessionKey: PrivateKey, publicKeys: Seq[PublicKey]): (Seq[PublicKey], Seq[ByteVector32]) = { - val ephemeralPublicKey0 = blind(PublicKey(Crypto.curve.getG, compressed = true), sessionKey.value.toBin) + val ephemeralPublicKey0 = blind(PublicKey(Crypto.curve.getG), sessionKey.value) val secret0 = computeSharedSecret(publicKeys.head, sessionKey) val blindingFactor0 = computeblindingFactor(ephemeralPublicKey0, secret0) computeEphemeralPublicKeysAndSharedSecrets(sessionKey, publicKeys.tail, Seq(ephemeralPublicKey0), Seq(blindingFactor0), Seq(secret0)) @@ -191,7 +191,7 @@ object Sphinx extends Logging { val nextPubKey = blind(PublicKey(packet.publicKey), computeblindingFactor(PublicKey(packet.publicKey), sharedSecret)) - ParsedPacket(payload, Packet(Version, nextPubKey, hmac, nextRouteInfo), sharedSecret) + ParsedPacket(payload, Packet(Version, nextPubKey.value, hmac, nextRouteInfo), sharedSecret) } @tailrec @@ -263,12 +263,12 @@ object Sphinx extends Logging { val (ephemeralPublicKeys, sharedsecrets) = computeEphemeralPublicKeysAndSharedSecrets(sessionKey, publicKeys) val filler = generateFiller("rho", sharedsecrets.dropRight(1), PayloadLength + MacLength, MaxHops) - val lastPacket = makeNextPacket(payloads.last, associatedData, ephemeralPublicKeys.last, sharedsecrets.last, LAST_PACKET, filler) + val lastPacket = makeNextPacket(payloads.last, associatedData, ephemeralPublicKeys.last.value, sharedsecrets.last, LAST_PACKET, filler) @tailrec def loop(hoppayloads: Seq[ByteVector], ephkeys: Seq[PublicKey], sharedSecrets: Seq[ByteVector32], packet: Packet): Packet = { if (hoppayloads.isEmpty) packet else { - val nextPacket = makeNextPacket(hoppayloads.last, associatedData, ephkeys.last, sharedSecrets.last, packet) + val nextPacket = makeNextPacket(hoppayloads.last, associatedData, ephkeys.last.value, sharedSecrets.last, packet) loop(hoppayloads.dropRight(1), ephkeys.dropRight(1), sharedSecrets.dropRight(1), nextPacket) } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteAuditDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteAuditDb.scala index 3ede1c092..519ed96b8 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteAuditDb.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteAuditDb.scala @@ -82,7 +82,7 @@ class SqliteAuditDb(sqlite: Connection) extends AuditDb with Logging { override def add(e: AvailableBalanceChanged): Unit = using(sqlite.prepareStatement("INSERT INTO balance_updated VALUES (?, ?, ?, ?, ?, ?)")) { statement => statement.setBytes(1, e.channelId.toArray) - statement.setBytes(2, e.commitments.remoteParams.nodeId.toBin.toArray) + statement.setBytes(2, e.commitments.remoteParams.nodeId.value.toArray) statement.setLong(3, e.localBalanceMsat) statement.setLong(4, e.commitments.commitInput.txOut.amount.toLong) statement.setLong(5, e.commitments.remoteParams.channelReserveSatoshis) // remote decides what our reserve should be @@ -93,7 +93,7 @@ class SqliteAuditDb(sqlite: Connection) extends AuditDb with Logging { override def add(e: ChannelLifecycleEvent): Unit = using(sqlite.prepareStatement("INSERT INTO channel_events VALUES (?, ?, ?, ?, ?, ?, ?)")) { statement => statement.setBytes(1, e.channelId.toArray) - statement.setBytes(2, e.remoteNodeId.toBin.toArray) + statement.setBytes(2, e.remoteNodeId.value.toArray) statement.setLong(3, e.capacitySat) statement.setBoolean(4, e.isFunder) statement.setBoolean(5, e.isPrivate) @@ -138,7 +138,7 @@ class SqliteAuditDb(sqlite: Connection) extends AuditDb with Logging { override def add(e: NetworkFeePaid): Unit = using(sqlite.prepareStatement("INSERT INTO network_fees VALUES (?, ?, ?, ?, ?, ?)")) { statement => statement.setBytes(1, e.channelId.toArray) - statement.setBytes(2, e.remoteNodeId.toBin.toArray) + statement.setBytes(2, e.remoteNodeId.value.toArray) statement.setBytes(3, e.tx.txid.toArray) statement.setLong(4, e.fee.toLong) statement.setString(5, e.txType) @@ -153,7 +153,7 @@ class SqliteAuditDb(sqlite: Connection) extends AuditDb with Logging { case Channel.RemoteError(error) => ("remote", error.toAscii) } statement.setBytes(1, e.channelId.toArray) - statement.setBytes(2, e.remoteNodeId.toBin.toArray) + statement.setBytes(2, e.remoteNodeId.value.toArray) statement.setString(3, errorName) statement.setString(4, errorMessage) statement.setBoolean(5, e.isFatal) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteNetworkDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteNetworkDb.scala index 4bca51ee2..e5a1a8363 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteNetworkDb.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteNetworkDb.scala @@ -45,7 +45,7 @@ class SqliteNetworkDb(sqlite: Connection) extends NetworkDb { override def addNode(n: NodeAnnouncement): Unit = { using(sqlite.prepareStatement("INSERT OR IGNORE INTO nodes VALUES (?, ?)")) { statement => - statement.setBytes(1, n.nodeId.toBin.toArray) + statement.setBytes(1, n.nodeId.value.toArray) statement.setBytes(2, nodeAnnouncementCodec.encode(n).require.toByteArray) statement.executeUpdate() } @@ -54,14 +54,14 @@ class SqliteNetworkDb(sqlite: Connection) extends NetworkDb { override def updateNode(n: NodeAnnouncement): Unit = { using(sqlite.prepareStatement("UPDATE nodes SET data=? WHERE node_id=?")) { statement => statement.setBytes(1, nodeAnnouncementCodec.encode(n).require.toByteArray) - statement.setBytes(2, n.nodeId.toBin.toArray) + statement.setBytes(2, n.nodeId.value.toArray) statement.executeUpdate() } } override def getNode(nodeId: Crypto.PublicKey): Option[NodeAnnouncement] = { using(sqlite.prepareStatement("SELECT data FROM nodes WHERE node_id=?")) { statement => - statement.setBytes(1, nodeId.toBin.toArray) + statement.setBytes(1, nodeId.value.toArray) val rs = statement.executeQuery() codecSequence(rs, nodeAnnouncementCodec).headOption } @@ -69,7 +69,7 @@ class SqliteNetworkDb(sqlite: Connection) extends NetworkDb { override def removeNode(nodeId: Crypto.PublicKey): Unit = { using(sqlite.prepareStatement("DELETE FROM nodes WHERE node_id=?")) { statement => - statement.setBytes(1, nodeId.toBin.toArray) + statement.setBytes(1, nodeId.value.toArray) statement.executeUpdate() } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqlitePeersDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqlitePeersDb.scala index 1a63a26d2..f34d98d50 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqlitePeersDb.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqlitePeersDb.scala @@ -41,10 +41,10 @@ import SqliteUtils.ExtendedResultSet._ val data = LightningMessageCodecs.nodeaddress.encode(nodeaddress).require.toByteArray using(sqlite.prepareStatement("UPDATE peers SET data=? WHERE node_id=?")) { update => update.setBytes(1, data) - update.setBytes(2, nodeId.toBin.toArray) + update.setBytes(2, nodeId.value.toArray) if (update.executeUpdate() == 0) { using(sqlite.prepareStatement("INSERT INTO peers VALUES (?, ?)")) { statement => - statement.setBytes(1, nodeId.toBin.toArray) + statement.setBytes(1, nodeId.value.toArray) statement.setBytes(2, data) statement.executeUpdate() } @@ -54,7 +54,7 @@ import SqliteUtils.ExtendedResultSet._ override def removePeer(nodeId: Crypto.PublicKey): Unit = { using(sqlite.prepareStatement("DELETE FROM peers WHERE node_id=?")) { statement => - statement.setBytes(1, nodeId.toBin.toArray) + statement.setBytes(1, nodeId.value.toArray) statement.executeUpdate() } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/io/Authenticator.scala b/eclair-core/src/main/scala/fr/acinq/eclair/io/Authenticator.scala index 9c2d89f68..1aebf9c0f 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/io/Authenticator.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/io/Authenticator.scala @@ -44,8 +44,8 @@ class Authenticator(nodeParams: NodeParams) extends Actor with DiagnosticActorLo case pending@PendingAuth(connection, remoteNodeId_opt, address, _) => log.debug(s"authenticating connection to ${address.getHostString}:${address.getPort} (pending=${authenticating.size} handlers=${context.children.size})") val transport = context.actorOf(TransportHandler.props( - KeyPair(nodeParams.nodeId.toBin, nodeParams.privateKey.toBin), - remoteNodeId_opt.map(_.toBin), + KeyPair(nodeParams.nodeId.value, nodeParams.privateKey.value), + remoteNodeId_opt.map(_.value), connection = connection, codec = LightningMessageCodecs.cachedLightningMessageCodec)) context watch transport diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/package.scala b/eclair-core/src/main/scala/fr/acinq/eclair/package.scala index 4e354c743..44adabf9e 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/package.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/package.scala @@ -42,7 +42,9 @@ package object eclair { def randomBytes32: ByteVector32 = ByteVector32(randomBytes(32)) - def randomKey: PrivateKey = PrivateKey(randomBytes32, compressed = true) + def randomBytes64: ByteVector64 = ByteVector64(randomBytes(64)) + + def randomKey: PrivateKey = PrivateKey(randomBytes32) def toLongId(fundingTxHash: ByteVector32, fundingOutputIndex: Int): ByteVector32 = { require(fundingOutputIndex < 65536, "fundingOutputIndex must not be greater than FFFF") diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentLifecycle.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentLifecycle.scala index 1847ebe6b..cf878ed62 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentLifecycle.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentLifecycle.scala @@ -17,9 +17,10 @@ package fr.acinq.eclair.payment import java.util.UUID + import akka.actor.{ActorRef, FSM, Props, Status} import fr.acinq.bitcoin.Crypto.PublicKey -import fr.acinq.bitcoin.{ByteVector32, MilliSatoshi} +import fr.acinq.bitcoin.{ByteVector32, ByteVector64, MilliSatoshi} import fr.acinq.eclair._ import fr.acinq.eclair.channel.{AddHtlcFailed, CMD_ADD_HTLC, Channel, Register} import fr.acinq.eclair.crypto.Sphinx.{ErrorPacket, Packet} @@ -31,6 +32,7 @@ import fr.acinq.eclair.router._ import fr.acinq.eclair.wire._ import scodec.Attempt import scodec.bits.ByteVector + import concurrent.duration._ import scala.compat.Platform import scala.util.{Failure, Success} @@ -124,7 +126,7 @@ class PaymentLifecycle(nodeParams: NodeParams, id: UUID, router: ActorRef, regis case Some(u) if u.shortChannelId != failureMessage.update.shortChannelId => // it is possible that nodes in the route prefer using a different channel (to the same N+1 node) than the one we requested, that's fine log.info(s"received an update for a different channel than the one we asked: requested=${u.shortChannelId} actual=${failureMessage.update.shortChannelId} update=${failureMessage.update}") - case Some(u) if areSame(u, failureMessage.update) => + case Some(u) if Announcements.areSame(u, failureMessage.update) => // node returned the exact same update we used, this can happen e.g. if the channel is imbalanced // in that case, let's temporarily exclude the channel from future routes, giving it time to recover log.info(s"received exact same update from nodeId=$nodeId, excluding the channel from futures routes") @@ -300,16 +302,6 @@ object PaymentLifecycle { */ def getChannelUpdateForNode(nodeId: PublicKey, hops: Seq[Hop]): Option[ChannelUpdate] = hops.find(_.nodeId == nodeId).map(_.lastUpdate) - /** - * This method compares channel updates, ignoring fields that don't matter, like signature or timestamp - * - * @param u1 - * @param u2 - * @return true if channel updates are "equal" - */ - def areSame(u1: ChannelUpdate, u2: ChannelUpdate): Boolean = - u1.copy(signature = ByteVector.empty, timestamp = 0) == u2.copy(signature = ByteVector.empty, timestamp = 0) - /** * This allows us to detect if a bad node always answers with a new update (e.g. with a slightly different expiry or fee) * in order to mess with us. diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentRequest.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentRequest.scala index c1b589412..e34628560 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentRequest.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentRequest.scala @@ -96,10 +96,10 @@ case class PaymentRequest(prefix: String, amount: Option[MilliSatoshi], timestam * @return a signed payment request */ def sign(priv: PrivateKey): PaymentRequest = { - val (r, s) = Crypto.sign(hash, priv) - val (pub1, pub2) = Crypto.recoverPublicKey((r, s), hash) + val sig64 = Crypto.sign(hash, priv) + val (pub1, _) = Crypto.recoverPublicKey(sig64, hash) val recid = if (nodeId == pub1) 0.toByte else 1.toByte - val signature = Crypto.fixSize(ByteVector.view(r.toByteArray.dropWhile(_ == 0.toByte))) ++ Crypto.fixSize(ByteVector.view(s.toByteArray.dropWhile(_ == 0.toByte))) :+ recid + val signature = sig64 :+ recid this.copy(signature = signature) } } @@ -434,15 +434,13 @@ object PaymentRequest { val prefix: String = prefixes.values.find(prefix => hrp.startsWith(prefix)).getOrElse(throw new RuntimeException("unknown prefix")) val data = string2Bits(lowercaseInput.slice(separatorIndex + 1, lowercaseInput.size - 6)) // 6 == checksum size val bolt11Data = Codecs.bolt11DataCodec.decode(data).require.value - val signature = bolt11Data.signature - val r = new BigInteger(1, signature.take(32).toArray) - val s = new BigInteger(1, signature.drop(32).take(32).toArray) + val signature = ByteVector64(bolt11Data.signature.take(64)) val message: ByteVector = ByteVector.view(hrp.getBytes) ++ data.dropRight(520).toByteVector // we drop the sig bytes - val (pub1, pub2) = Crypto.recoverPublicKey((r, s), Crypto.sha256(message)) - val recid = signature.last + val (pub1, pub2) = Crypto.recoverPublicKey(signature, Crypto.sha256(message)) + val recid = bolt11Data.signature.last val pub = if (recid % 2 != 0) pub2 else pub1 val amount_opt = Amount.decode(hrp.drop(prefix.length)) - val validSig = Crypto.verifySignature(Crypto.sha256(message), (r, s), pub) + val validSig = Crypto.verifySignature(Crypto.sha256(message), signature, pub) require(validSig, "invalid signature") PaymentRequest( prefix = prefix, @@ -450,7 +448,7 @@ object PaymentRequest { timestamp = bolt11Data.timestamp, nodeId = pub, tags = bolt11Data.taggedFields, - signature = signature + signature = bolt11Data.signature ) } 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 d6cf79ad5..4a4611664 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 @@ -17,11 +17,12 @@ package fr.acinq.eclair.router import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey, sha256, verifySignature} -import fr.acinq.bitcoin.{ByteVector32, Crypto, LexicographicalOrdering} +import fr.acinq.bitcoin.{ByteVector32, ByteVector64, Crypto, LexicographicalOrdering} import fr.acinq.eclair.wire._ import fr.acinq.eclair.{ShortChannelId, serializationResult} import scodec.bits.{BitVector, ByteVector} import shapeless.HNil + import scala.concurrent.duration._ import scala.compat.Platform import scala.concurrent.duration._ @@ -40,18 +41,18 @@ object Announcements { def channelUpdateWitnessEncode(chainHash: ByteVector32, shortChannelId: ShortChannelId, timestamp: Long, messageFlags: Byte, channelFlags: Byte, cltvExpiryDelta: Int, htlcMinimumMsat: Long, feeBaseMsat: Long, feeProportionalMillionths: Long, htlcMaximumMsat: Option[Long]): ByteVector = sha256(sha256(serializationResult(LightningMessageCodecs.channelUpdateWitnessCodec.encode(chainHash :: shortChannelId :: timestamp :: messageFlags :: channelFlags :: cltvExpiryDelta :: htlcMinimumMsat :: feeBaseMsat :: feeProportionalMillionths :: htlcMaximumMsat :: HNil)))) - def signChannelAnnouncement(chainHash: ByteVector32, shortChannelId: ShortChannelId, localNodeSecret: PrivateKey, remoteNodeId: PublicKey, localFundingPrivKey: PrivateKey, remoteFundingKey: PublicKey, features: ByteVector): (ByteVector, ByteVector) = { + def signChannelAnnouncement(chainHash: ByteVector32, shortChannelId: ShortChannelId, localNodeSecret: PrivateKey, remoteNodeId: PublicKey, localFundingPrivKey: PrivateKey, remoteFundingKey: PublicKey, features: ByteVector): (ByteVector64, ByteVector64) = { val witness = if (isNode1(localNodeSecret.publicKey, remoteNodeId)) { channelAnnouncementWitnessEncode(chainHash, shortChannelId, localNodeSecret.publicKey, remoteNodeId, localFundingPrivKey.publicKey, remoteFundingKey, features) } else { channelAnnouncementWitnessEncode(chainHash, shortChannelId, remoteNodeId, localNodeSecret.publicKey, remoteFundingKey, localFundingPrivKey.publicKey, features) } - val nodeSig = Crypto.encodeSignature(Crypto.sign(witness, localNodeSecret)) :+ 1.toByte - val bitcoinSig = Crypto.encodeSignature(Crypto.sign(witness, localFundingPrivKey)) :+ 1.toByte + val nodeSig = Crypto.sign(witness, localNodeSecret) + val bitcoinSig = Crypto.sign(witness, localFundingPrivKey) (nodeSig, bitcoinSig) } - def makeChannelAnnouncement(chainHash: ByteVector32, shortChannelId: ShortChannelId, localNodeId: PublicKey, remoteNodeId: PublicKey, localFundingKey: PublicKey, remoteFundingKey: PublicKey, localNodeSignature: ByteVector, remoteNodeSignature: ByteVector, localBitcoinSignature: ByteVector, remoteBitcoinSignature: ByteVector): ChannelAnnouncement = { + def makeChannelAnnouncement(chainHash: ByteVector32, shortChannelId: ShortChannelId, localNodeId: PublicKey, remoteNodeId: PublicKey, localFundingKey: PublicKey, remoteFundingKey: PublicKey, localNodeSignature: ByteVector64, remoteNodeSignature: ByteVector64, localBitcoinSignature: ByteVector64, remoteBitcoinSignature: ByteVector64): ChannelAnnouncement = { val (nodeId1, nodeId2, bitcoinKey1, bitcoinKey2, nodeSignature1, nodeSignature2, bitcoinSignature1, bitcoinSignature2) = if (isNode1(localNodeId, remoteNodeId)) { (localNodeId, remoteNodeId, localFundingKey, remoteFundingKey, localNodeSignature, remoteNodeSignature, localBitcoinSignature, remoteBitcoinSignature) @@ -76,7 +77,7 @@ object Announcements { def makeNodeAnnouncement(nodeSecret: PrivateKey, alias: String, color: Color, nodeAddresses: List[NodeAddress], timestamp: Long = Platform.currentTime.milliseconds.toSeconds): NodeAnnouncement = { require(alias.size <= 32) val witness = nodeAnnouncementWitnessEncode(timestamp, nodeSecret.publicKey, color, alias, ByteVector.empty, nodeAddresses) - val sig = Crypto.encodeSignature(Crypto.sign(witness, nodeSecret)) :+ 1.toByte + val sig = Crypto.sign(witness, nodeSecret) NodeAnnouncement( signature = sig, timestamp = timestamp, @@ -96,7 +97,7 @@ object Announcements { * * @return true if localNodeId is node1 */ - def isNode1(localNodeId: PublicKey, remoteNodeId: PublicKey) = LexicographicalOrdering.isLessThan(localNodeId, remoteNodeId) + def isNode1(localNodeId: PublicKey, remoteNodeId: PublicKey) = LexicographicalOrdering.isLessThan(localNodeId.value, remoteNodeId.value) /** * BOLT 7: @@ -115,6 +116,16 @@ object Announcements { */ def isEnabled(channelFlags: Byte): Boolean = (channelFlags & 2) == 0 + /** + * This method compares channel updates, ignoring fields that don't matter, like signature or timestamp + * + * @param u1 + * @param u2 + * @return true if channel updates are "equal" + */ + def areSame(u1: ChannelUpdate, u2: ChannelUpdate): Boolean = + u1.copy(signature = ByteVector64.Zeroes, timestamp = 0) == u2.copy(signature = ByteVector64.Zeroes, timestamp = 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() @@ -125,7 +136,7 @@ object Announcements { val htlcMaximumMsatOpt = Some(htlcMaximumMsat) val witness = channelUpdateWitnessEncode(chainHash, shortChannelId, timestamp, messageFlags, channelFlags, cltvExpiryDelta, htlcMinimumMsat, feeBaseMsat, feeProportionalMillionths, htlcMaximumMsatOpt) - val sig = Crypto.encodeSignature(Crypto.sign(witness, nodeSecret)) :+ 1.toByte + val sig = Crypto.sign(witness, nodeSecret) ChannelUpdate( signature = sig, chainHash = chainHash, diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/router/Router.scala b/eclair-core/src/main/scala/fr/acinq/eclair/router/Router.scala index 779919502..77b70124e 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/router/Router.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/router/Router.scala @@ -19,7 +19,7 @@ package fr.acinq.eclair.router import akka.Done import akka.actor.{ActorRef, Props, Status} import akka.event.Logging.MDC -import fr.acinq.bitcoin.ByteVector32 +import fr.acinq.bitcoin.{ByteVector32, ByteVector64} import fr.acinq.bitcoin.Crypto.PublicKey import fr.acinq.bitcoin.Script.{pay2wsh, write} import fr.acinq.eclair._ @@ -149,7 +149,7 @@ class Router(nodeParams: NodeParams, watcher: ActorRef, initialized: Option[Prom initChannels.values.foreach { c => val txid = channels(c)._1 val TxCoordinates(_, _, outputIndex) = ShortChannelId.coordinates(c.shortChannelId) - val fundingOutputScript = write(pay2wsh(Scripts.multiSig2of2(PublicKey(c.bitcoinKey1), PublicKey(c.bitcoinKey2)))) + val fundingOutputScript = write(pay2wsh(Scripts.multiSig2of2(c.bitcoinKey1, c.bitcoinKey2))) watcher ! WatchSpentBasic(self, txid, outputIndex, fundingOutputScript, BITCOIN_FUNDING_EXTERNAL_CHANNEL_SPENT(c.shortChannelId)) } @@ -231,7 +231,7 @@ class Router(nodeParams: NodeParams, watcher: ActorRef, initialized: Option[Prom case ValidateResult(c, Right((tx, UtxoStatus.Unspent))) => val TxCoordinates(_, _, outputIndex) = ShortChannelId.coordinates(c.shortChannelId) // let's check that the output is indeed a P2WSH multisig 2-of-2 of nodeid1 and nodeid2) - val fundingOutputScript = write(pay2wsh(Scripts.multiSig2of2(PublicKey(c.bitcoinKey1), PublicKey(c.bitcoinKey2)))) + val fundingOutputScript = write(pay2wsh(Scripts.multiSig2of2(c.bitcoinKey1, c.bitcoinKey2))) if (tx.txOut.size < outputIndex + 1 || fundingOutputScript != tx.txOut(outputIndex).publicKeyScript) { log.error(s"invalid script for shortChannelId={}: txid={} does not have script=$fundingOutputScript at outputIndex=$outputIndex ann={}", c.shortChannelId, tx.txid, c) d0.awaiting.get(c) match { @@ -415,7 +415,7 @@ class Router(nodeParams: NodeParams, watcher: ActorRef, initialized: Option[Prom val params = params_opt.getOrElse(defaultRouteParams) val routesToFind = if (params.randomize) DEFAULT_ROUTES_COUNT else 1 - log.info(s"finding a route $start->$end with assistedChannels={} ignoreNodes={} ignoreChannels={} excludedChannels={}", assistedUpdates.keys.mkString(","), ignoreNodes.map(_.toBin).mkString(","), ignoreChannels.mkString(","), d.excludedChannels.mkString(",")) + log.info(s"finding a route $start->$end with assistedChannels={} ignoreNodes={} ignoreChannels={} excludedChannels={}", assistedUpdates.keys.mkString(","), ignoreNodes.map(_.value).mkString(","), ignoreChannels.mkString(","), d.excludedChannels.mkString(",")) log.info(s"finding a route with randomize={} params={}", routesToFind > 1, params) findRoute(d.graph, start, end, amount, numRoutes = routesToFind, extraEdges = extraEdges, ignoredEdges = ignoredUpdates.toSet, routeParams = params) .map(r => sender ! RouteResponse(r, ignoreNodes, ignoreChannels)) @@ -732,7 +732,7 @@ object Router { def toFakeUpdate(extraHop: ExtraHop): ChannelUpdate = // the `direction` bit in flags will not be accurate but it doesn't matter because it is not used // what matters is that the `disable` bit is 0 so that this update doesn't get filtered out - ChannelUpdate(signature = ByteVector.empty, chainHash = ByteVector32.Zeroes, extraHop.shortChannelId, Platform.currentTime.milliseconds.toSeconds, messageFlags = 0, channelFlags = 0, extraHop.cltvExpiryDelta, htlcMinimumMsat = 0L, extraHop.feeBaseMsat, extraHop.feeProportionalMillionths, None) + ChannelUpdate(signature = ByteVector64.Zeroes, chainHash = ByteVector32.Zeroes, extraHop.shortChannelId, Platform.currentTime.milliseconds.toSeconds, messageFlags = 0, channelFlags = 0, extraHop.cltvExpiryDelta, htlcMinimumMsat = 0L, extraHop.feeBaseMsat, extraHop.feeProportionalMillionths, None) def toFakeUpdates(extraRoute: Seq[ExtraHop], targetNodeId: PublicKey): Map[ChannelDesc, ChannelUpdate] = { // BOLT 11: "For each entry, the pubkey is the node ID of the start of the channel", and the last node is the destination diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/transactions/Scripts.scala b/eclair-core/src/main/scala/fr/acinq/eclair/transactions/Scripts.scala index 1dbd13395..41b5ca3a4 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/transactions/Scripts.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/transactions/Scripts.scala @@ -18,7 +18,7 @@ package fr.acinq.eclair.transactions import fr.acinq.bitcoin.Crypto.{PublicKey, ripemd160} import fr.acinq.bitcoin.Script._ -import fr.acinq.bitcoin.{ByteVector32, LexicographicalOrdering, LockTimeThreshold, OP_0, OP_1, OP_1NEGATE, OP_2, OP_2DROP, OP_ADD, OP_CHECKLOCKTIMEVERIFY, OP_CHECKMULTISIG, OP_CHECKSEQUENCEVERIFY, OP_CHECKSIG, OP_DROP, OP_DUP, OP_ELSE, OP_ENDIF, OP_EQUAL, OP_EQUALVERIFY, OP_HASH160, OP_IF, OP_NOTIF, OP_PUSHDATA, OP_SIZE, OP_SWAP, Satoshi, Script, ScriptElt, ScriptWitness, Transaction, TxIn} +import fr.acinq.bitcoin.{ByteVector32, ByteVector64, Crypto, LexicographicalOrdering, LockTimeThreshold, OP_0, OP_1, OP_1NEGATE, OP_2, OP_2DROP, OP_ADD, OP_CHECKLOCKTIMEVERIFY, OP_CHECKMULTISIG, OP_CHECKSEQUENCEVERIFY, OP_CHECKSIG, OP_DROP, OP_DUP, OP_ELSE, OP_ENDIF, OP_EQUAL, OP_EQUALVERIFY, OP_HASH160, OP_IF, OP_NOTIF, OP_PUSHDATA, OP_SIZE, OP_SWAP, Satoshi, Script, ScriptElt, ScriptWitness, Transaction, TxIn} import scodec.bits.ByteVector /** @@ -26,21 +26,9 @@ import scodec.bits.ByteVector */ object Scripts { - def toSelfDelay2csv(in: Int): Long = ??? + def der(sig: ByteVector64): ByteVector = Crypto.compact2der(sig) :+ 1 - /*in match { - case locktime(Blocks(blocks)) => blocks - case locktime(Seconds(seconds)) => TxIn.SEQUENCE_LOCKTIME_TYPE_FLAG | (seconds >> TxIn.SEQUENCE_LOCKTIME_GRANULARITY) - }*/ - - def expiry2cltv(in: Long): Long = ??? - - /*in match { - case locktime(Blocks(blocks)) => blocks - case locktime(Seconds(seconds)) => seconds - }*/ - - def multiSig2of2(pubkey1: PublicKey, pubkey2: PublicKey): Seq[ScriptElt] = if (LexicographicalOrdering.isLessThan(pubkey1.toBin, pubkey2.toBin)) + def multiSig2of2(pubkey1: PublicKey, pubkey2: PublicKey): Seq[ScriptElt] = if (LexicographicalOrdering.isLessThan(pubkey1.value, pubkey2.value)) Script.createMultiSigMofN(2, Seq(pubkey1, pubkey2)) else Script.createMultiSigMofN(2, Seq(pubkey2, pubkey1)) @@ -53,11 +41,11 @@ object Scripts { * @param pubkey2 * @return a script witness that matches the msig 2-of-2 pubkey script for pubkey1 and pubkey2 */ - def witness2of2(sig1: ByteVector, sig2: ByteVector, pubkey1: PublicKey, pubkey2: PublicKey): ScriptWitness = { - if (LexicographicalOrdering.isLessThan(pubkey1.toBin, pubkey2.toBin)) - ScriptWitness(Seq(ByteVector.empty, sig1, sig2, write(multiSig2of2(pubkey1, pubkey2)))) + def witness2of2(sig1: ByteVector64, sig2: ByteVector64, pubkey1: PublicKey, pubkey2: PublicKey): ScriptWitness = { + if (LexicographicalOrdering.isLessThan(pubkey1.value, pubkey2.value)) + ScriptWitness(Seq(ByteVector.empty, der(sig1), der(sig2), write(multiSig2of2(pubkey1, pubkey2)))) else - ScriptWitness(Seq(ByteVector.empty, sig2, sig1, write(multiSig2of2(pubkey1, pubkey2)))) + ScriptWitness(Seq(ByteVector.empty, der(sig2), der(sig1), write(multiSig2of2(pubkey1, pubkey2)))) } @@ -76,55 +64,6 @@ object Scripts { case _ => OP_PUSHDATA(Script.encodeNumber(n)) } - def redeemSecretOrDelay(delayedKey: ByteVector, reltimeout: Long, keyIfSecretKnown: ByteVector, hashOfSecret: ByteVector32): Seq[ScriptElt] = { - // @formatter:off - OP_HASH160 :: OP_PUSHDATA(ripemd160(hashOfSecret)) :: OP_EQUAL :: - OP_IF :: - OP_PUSHDATA(keyIfSecretKnown) :: - OP_ELSE :: - encodeNumber(reltimeout):: OP_CHECKSEQUENCEVERIFY :: OP_DROP :: OP_PUSHDATA(delayedKey) :: - OP_ENDIF :: - OP_CHECKSIG :: Nil - // @formatter:on - } - - def scriptPubKeyHtlcSend(ourkey: ByteVector, theirkey: ByteVector, abstimeout: Long, reltimeout: Long, rhash: ByteVector32, commit_revoke: ByteVector): Seq[ScriptElt] = { - // values lesser than 16 should be encoded using OP_0..OP_16 instead of OP_PUSHDATA - require(abstimeout > 16, s"abstimeout=$abstimeout must be greater than 16") - // @formatter:off - OP_SIZE :: encodeNumber(32) :: OP_EQUALVERIFY :: - OP_HASH160 :: OP_DUP :: - OP_PUSHDATA(ripemd160(rhash)) :: OP_EQUAL :: - OP_SWAP :: OP_PUSHDATA(ripemd160(commit_revoke)) :: OP_EQUAL :: OP_ADD :: - OP_IF :: - OP_PUSHDATA(theirkey) :: - OP_ELSE :: - encodeNumber(abstimeout) :: OP_CHECKLOCKTIMEVERIFY :: encodeNumber(reltimeout) :: OP_CHECKSEQUENCEVERIFY :: OP_2DROP :: OP_PUSHDATA(ourkey) :: - OP_ENDIF :: - OP_CHECKSIG :: Nil - // @formatter:on - } - - def scriptPubKeyHtlcReceive(ourkey: ByteVector, theirkey: ByteVector, abstimeout: Long, reltimeout: Long, rhash: ByteVector32, commit_revoke: ByteVector): Seq[ScriptElt] = { - // values lesser than 16 should be encoded using OP_0..OP_16 instead of OP_PUSHDATA - require(abstimeout > 16, s"abstimeout=$abstimeout must be greater than 16") - // @formatter:off - OP_SIZE :: encodeNumber(32) :: OP_EQUALVERIFY :: - OP_HASH160 :: OP_DUP :: - OP_PUSHDATA(ripemd160(rhash)) :: OP_EQUAL :: - OP_IF :: - encodeNumber(reltimeout) :: OP_CHECKSEQUENCEVERIFY :: OP_2DROP :: OP_PUSHDATA(ourkey) :: - OP_ELSE :: - OP_PUSHDATA(ripemd160(commit_revoke)) :: OP_EQUAL :: - OP_NOTIF :: - encodeNumber(abstimeout) :: OP_CHECKLOCKTIMEVERIFY :: OP_DROP :: - OP_ENDIF :: - OP_PUSHDATA(theirkey) :: - OP_ENDIF :: - OP_CHECKSIG :: Nil - // @formatter:on - } - def applyFees(amount_us: Satoshi, amount_them: Satoshi, fee: Satoshi) = { val (amount_us1: Satoshi, amount_them1: Satoshi) = (amount_us, amount_them) match { case (Satoshi(us), Satoshi(them)) if us >= fee.toLong / 2 && them >= fee.toLong / 2 => (Satoshi(us - fee.toLong / 2), Satoshi(them - fee.toLong / 2)) @@ -188,15 +127,15 @@ object Scripts { /** * This witness script spends a [[toLocalDelayed]] output using a local sig after a delay */ - def witnessToLocalDelayedAfterDelay(localSig: ByteVector, toLocalDelayedScript: ByteVector) = - ScriptWitness(localSig :: ByteVector.empty :: toLocalDelayedScript :: Nil) + def witnessToLocalDelayedAfterDelay(localSig: ByteVector64, toLocalDelayedScript: ByteVector) = + ScriptWitness(der(localSig) :: ByteVector.empty :: toLocalDelayedScript :: Nil) /** * This witness script spends (steals) a [[toLocalDelayed]] output using a revocation key as a punishment * for having published a revoked transaction */ - def witnessToLocalDelayedWithRevocationSig(revocationSig: ByteVector, toLocalScript: ByteVector) = - ScriptWitness(revocationSig :: ByteVector(1) :: toLocalScript :: Nil) + def witnessToLocalDelayedWithRevocationSig(revocationSig: ByteVector64, toLocalScript: ByteVector) = + ScriptWitness(der(revocationSig) :: ByteVector(1) :: toLocalScript :: Nil) def htlcOffered(localHtlcPubkey: PublicKey, remoteHtlcPubkey: PublicKey, revocationPubKey: PublicKey, paymentHash: ByteVector): Seq[ScriptElt] = { // @formatter:off @@ -220,15 +159,15 @@ object Scripts { /** * This is the witness script of the 2nd-stage HTLC Success transaction (consumes htlcOffered script from commit tx) */ - def witnessHtlcSuccess(localSig: ByteVector, remoteSig: ByteVector, paymentPreimage: ByteVector32, htlcOfferedScript: ByteVector) = - ScriptWitness(ByteVector.empty :: remoteSig :: localSig :: paymentPreimage.bytes :: htlcOfferedScript :: Nil) + def witnessHtlcSuccess(localSig: ByteVector64, remoteSig: ByteVector64, paymentPreimage: ByteVector32, htlcOfferedScript: ByteVector) = + ScriptWitness(ByteVector.empty :: der(remoteSig) :: der(localSig) :: paymentPreimage.bytes :: htlcOfferedScript :: Nil) /** * If local publishes its commit tx where there was a local->remote htlc, then remote uses this script to * claim its funds using a payment preimage (consumes htlcOffered script from commit tx) */ - def witnessClaimHtlcSuccessFromCommitTx(localSig: ByteVector, paymentPreimage: ByteVector32, htlcOfferedScript: ByteVector) = - ScriptWitness(localSig :: paymentPreimage.bytes :: htlcOfferedScript :: Nil) + def witnessClaimHtlcSuccessFromCommitTx(localSig: ByteVector64, paymentPreimage: ByteVector32, htlcOfferedScript: ByteVector) = + ScriptWitness(der(localSig) :: paymentPreimage.bytes :: htlcOfferedScript :: Nil) def htlcReceived(localHtlcPubkey: PublicKey, remoteHtlcPubkey: PublicKey, revocationPubKey: PublicKey, paymentHash: ByteVector, lockTime: Long) = { // @formatter:off @@ -254,21 +193,21 @@ object Scripts { /** * This is the witness script of the 2nd-stage HTLC Timeout transaction (consumes htlcReceived script from commit tx) */ - def witnessHtlcTimeout(localSig: ByteVector, remoteSig: ByteVector, htlcReceivedScript: ByteVector) = - ScriptWitness(ByteVector.empty :: remoteSig :: localSig :: ByteVector.empty :: htlcReceivedScript :: Nil) + def witnessHtlcTimeout(localSig: ByteVector64, remoteSig: ByteVector64, htlcReceivedScript: ByteVector) = + ScriptWitness(ByteVector.empty :: der(remoteSig) :: der(localSig) :: ByteVector.empty :: htlcReceivedScript :: Nil) /** * If local publishes its commit tx where there was a remote->local htlc, then remote uses this script to * claim its funds after timeout (consumes htlcReceived script from commit tx) */ - def witnessClaimHtlcTimeoutFromCommitTx(localSig: ByteVector, htlcReceivedScript: ByteVector) = - ScriptWitness(localSig :: ByteVector.empty :: htlcReceivedScript :: Nil) + def witnessClaimHtlcTimeoutFromCommitTx(localSig: ByteVector64, htlcReceivedScript: ByteVector) = + ScriptWitness(der(localSig) :: ByteVector.empty :: htlcReceivedScript :: Nil) /** * This witness script spends (steals) a [[htlcOffered]] or [[htlcReceived]] output using a revocation key as a punishment * for having published a revoked transaction */ - def witnessHtlcWithRevocationSig(revocationSig: ByteVector, revocationPubkey: PublicKey, htlcScript: ByteVector) = - ScriptWitness(revocationSig :: revocationPubkey.toBin :: htlcScript :: Nil) + def witnessHtlcWithRevocationSig(revocationSig: ByteVector64, revocationPubkey: PublicKey, htlcScript: ByteVector) = + ScriptWitness(der(revocationSig) :: revocationPubkey.value :: htlcScript :: Nil) } \ No newline at end of file diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/transactions/Transactions.scala b/eclair-core/src/main/scala/fr/acinq/eclair/transactions/Transactions.scala index 119f03a8d..0febeaab9 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/transactions/Transactions.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/transactions/Transactions.scala @@ -18,10 +18,10 @@ package fr.acinq.eclair.transactions import java.nio.ByteOrder -import fr.acinq.bitcoin.Crypto.{Point, PrivateKey, PublicKey, ripemd160} +import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey, ripemd160} import fr.acinq.bitcoin.Script._ import fr.acinq.bitcoin.SigVersion._ -import fr.acinq.bitcoin.{ByteVector32, Crypto, LexicographicalOrdering, MilliSatoshi, OutPoint, Protocol, SIGHASH_ALL, Satoshi, Script, ScriptElt, ScriptFlags, ScriptWitness, Transaction, TxIn, TxOut, millisatoshi2satoshi} +import fr.acinq.bitcoin.{ByteVector32, ByteVector64, Crypto, LexicographicalOrdering, MilliSatoshi, OutPoint, Protocol, SIGHASH_ALL, Satoshi, Script, ScriptElt, ScriptFlags, ScriptWitness, Transaction, TxIn, TxOut, millisatoshi2satoshi} import fr.acinq.eclair.transactions.Scripts._ import fr.acinq.eclair.wire.UpdateAddHtlc import scodec.bits.ByteVector @@ -100,12 +100,12 @@ object Transactions { /** * these values specific to us and used to estimate fees */ - val claimP2WPKHOutputWeight = 439 - val claimHtlcDelayedWeight = 484 - val claimHtlcSuccessWeight = 572 - val claimHtlcTimeoutWeight = 546 - val mainPenaltyWeight = 485 - val htlcPenaltyWeight = 579 // based on spending an HTLC-Success output (would be 571 with HTLC-Timeout) + val claimP2WPKHOutputWeight = 438 + val claimHtlcDelayedWeight = 483 + val claimHtlcSuccessWeight = 571 + val claimHtlcTimeoutWeight = 545 + val mainPenaltyWeight = 484 + val htlcPenaltyWeight = 578 // based on spending an HTLC-Success output (would be 571 with HTLC-Timeout) def weight2fee(feeratePerKw: Long, weight: Int) = Satoshi((feeratePerKw * weight) / 1000) @@ -148,12 +148,12 @@ object Transactions { * @param remotePaymentBasePoint remote payment base point * @return the obscured tx number as defined in BOLT #3 (a 48 bits integer) */ - def obscuredCommitTxNumber(commitTxNumber: Long, isFunder: Boolean, localPaymentBasePoint: Point, remotePaymentBasePoint: Point): Long = { + def obscuredCommitTxNumber(commitTxNumber: Long, isFunder: Boolean, localPaymentBasePoint: PublicKey, remotePaymentBasePoint: PublicKey): Long = { // from BOLT 3: SHA256(payment-basepoint from open_channel || payment-basepoint from accept_channel) val h = if (isFunder) - Crypto.sha256(localPaymentBasePoint.toBin(true) ++ remotePaymentBasePoint.toBin(true)) + Crypto.sha256(localPaymentBasePoint.value ++ remotePaymentBasePoint.value) else - Crypto.sha256(remotePaymentBasePoint.toBin(true) ++ localPaymentBasePoint.toBin(true)) + Crypto.sha256(remotePaymentBasePoint.value ++ localPaymentBasePoint.value) val blind = Protocol.uint64((h.takeRight(6).reverse ++ ByteVector.fromValidHex("0000")).toArray, ByteOrder.LITTLE_ENDIAN) commitTxNumber ^ blind @@ -167,7 +167,7 @@ object Transactions { * @param remotePaymentBasePoint remote payment base point * @return the actual commit tx number that was blinded and stored in locktime and sequence fields */ - def getCommitTxNumber(commitTx: Transaction, isFunder: Boolean, localPaymentBasePoint: Point, remotePaymentBasePoint: Point): Long = { + def getCommitTxNumber(commitTx: Transaction, isFunder: Boolean, localPaymentBasePoint: PublicKey, remotePaymentBasePoint: PublicKey): Long = { val blind = obscuredCommitTxNumber(0, isFunder, localPaymentBasePoint, remotePaymentBasePoint) val obscured = decodeTxNumber(commitTx.txIn.head.sequence, commitTx.lockTime) obscured ^ blind @@ -186,7 +186,7 @@ object Transactions { def decodeTxNumber(sequence: Long, locktime: Long): Long = ((sequence & 0xffffffL) << 24) + (locktime & 0xffffffL) - def makeCommitTx(commitTxInput: InputInfo, commitTxNumber: Long, localPaymentBasePoint: Point, remotePaymentBasePoint: Point, localIsFunder: Boolean, localDustLimit: Satoshi, localRevocationPubkey: PublicKey, toLocalDelay: Int, localDelayedPaymentPubkey: PublicKey, remotePaymentPubkey: PublicKey, localHtlcPubkey: PublicKey, remoteHtlcPubkey: PublicKey, spec: CommitmentSpec): CommitTx = { + def makeCommitTx(commitTxInput: InputInfo, commitTxNumber: Long, localPaymentBasePoint: PublicKey, remotePaymentBasePoint: PublicKey, localIsFunder: Boolean, localDustLimit: Satoshi, localRevocationPubkey: PublicKey, toLocalDelay: Int, localDelayedPaymentPubkey: PublicKey, remotePaymentPubkey: PublicKey, localHtlcPubkey: PublicKey, remoteHtlcPubkey: PublicKey, spec: CommitmentSpec): CommitTx = { val commitFee = commitTxFee(localDustLimit, spec) val (toLocalAmount: Satoshi, toRemoteAmount: Satoshi) = if (localIsFunder) { @@ -275,7 +275,7 @@ object Transactions { txOut = TxOut(Satoshi(0), localFinalScriptPubKey) :: Nil, lockTime = 0) - val weight = Transactions.addSigs(ClaimHtlcSuccessTx(input, tx), ByteVector.fill(73)(0), ByteVector32.Zeroes).tx.weight() + val weight = addSigs(ClaimHtlcSuccessTx(input, tx), PlaceHolderSig, ByteVector32.Zeroes).tx.weight() val fee = weight2fee(feeratePerKw, weight) val amount = input.txOut.amount - fee if (amount < localDustLimit) { @@ -299,7 +299,7 @@ object Transactions { txOut = TxOut(Satoshi(0), localFinalScriptPubKey) :: Nil, lockTime = htlc.cltvExpiry) - val weight = Transactions.addSigs(ClaimHtlcTimeoutTx(input, tx), ByteVector.fill(73)(0)).tx.weight() + val weight = addSigs(ClaimHtlcTimeoutTx(input, tx), PlaceHolderSig).tx.weight() val fee = weight2fee(feeratePerKw, weight) val amount = input.txOut.amount - fee @@ -325,7 +325,7 @@ object Transactions { lockTime = 0) // compute weight with a dummy 73 bytes signature (the largest you can get) and a dummy 33 bytes pubkey - val weight = Transactions.addSigs(ClaimP2WPKHOutputTx(input, tx), ByteVector.fill(33)(0), ByteVector.fill(73)(0)).tx.weight() + val weight = addSigs(ClaimP2WPKHOutputTx(input, tx), PlaceHolderPubKey, PlaceHolderSig).tx.weight() val fee = weight2fee(feeratePerKw, weight) val amount = input.txOut.amount - fee @@ -351,7 +351,7 @@ object Transactions { lockTime = 0) // compute weight with a dummy 73 bytes signature (the largest you can get) - val weight = Transactions.addSigs(ClaimDelayedOutputTx(input, tx), ByteVector.fill(73)(0)).tx.weight() + val weight = addSigs(ClaimDelayedOutputTx(input, tx), PlaceHolderSig).tx.weight() val fee = weight2fee(feeratePerKw, weight) val amount = input.txOut.amount - fee @@ -377,7 +377,7 @@ object Transactions { lockTime = 0) // compute weight with a dummy 73 bytes signature (the largest you can get) - val weight = Transactions.addSigs(ClaimDelayedOutputPenaltyTx(input, tx), ByteVector.fill(73)(0)).tx.weight() + val weight = addSigs(ClaimDelayedOutputPenaltyTx(input, tx), PlaceHolderSig).tx.weight() val fee = weight2fee(feeratePerKw, weight) val amount = input.txOut.amount - fee @@ -403,7 +403,7 @@ object Transactions { lockTime = 0) // compute weight with a dummy 73 bytes signature (the largest you can get) - val weight = Transactions.addSigs(MainPenaltyTx(input, tx), ByteVector.fill(73)(0)).tx.weight() + val weight = addSigs(MainPenaltyTx(input, tx), PlaceHolderSig).tx.weight() val fee = weight2fee(feeratePerKw, weight) val amount = input.txOut.amount - fee @@ -431,7 +431,7 @@ object Transactions { lockTime = 0) // compute weight with a dummy 73 bytes signature (the largest you can get) - val weight = Transactions.addSigs(MainPenaltyTx(input, tx), ByteVector.fill(73)(0)).tx.weight() + val weight = addSigs(MainPenaltyTx(input, tx), PlaceHolderSig).tx.weight() val fee = weight2fee(feeratePerKw, weight) val amount = input.txOut.amount - fee @@ -474,67 +474,80 @@ object Transactions { } } + /** + * Default public key used for fee estimation + */ + val PlaceHolderPubKey = PrivateKey(ByteVector32.One).publicKey - def sign(tx: Transaction, inputIndex: Int, redeemScript: ByteVector, amount: Satoshi, key: PrivateKey): ByteVector = { - Transaction.signInput(tx, inputIndex, redeemScript, SIGHASH_ALL, amount, SIGVERSION_WITNESS_V0, key) + /** + * This default sig takes 72B when encoded in DER (incl. 1B for the trailing sig hash), it is used for fee estimation + * It is 72 bytes because our signatures are normalized (low-s) and will take up 72 bytes at most in DER format + */ + val PlaceHolderSig = ByteVector64(ByteVector.fill(64)(0xaa)) + assert(der(PlaceHolderSig).size == 72) + + def sign(tx: Transaction, inputIndex: Int, redeemScript: ByteVector, amount: Satoshi, key: PrivateKey): ByteVector64 = { + val sigDER = Transaction.signInput(tx, inputIndex, redeemScript, SIGHASH_ALL, amount, SIGVERSION_WITNESS_V0, key) + val sig64 = Crypto.der2compact(sigDER) + sig64 } - def sign(txinfo: TransactionWithInputInfo, key: PrivateKey): ByteVector = { + def sign(txinfo: TransactionWithInputInfo, key: PrivateKey): ByteVector64 = { require(txinfo.tx.txIn.lengthCompare(1) == 0, "only one input allowed") sign(txinfo.tx, inputIndex = 0, txinfo.input.redeemScript, txinfo.input.txOut.amount, key) } - def addSigs(commitTx: CommitTx, localFundingPubkey: PublicKey, remoteFundingPubkey: PublicKey, localSig: ByteVector, remoteSig: ByteVector): CommitTx = { + def addSigs(commitTx: CommitTx, localFundingPubkey: PublicKey, remoteFundingPubkey: PublicKey, localSig: ByteVector64, remoteSig: ByteVector64): CommitTx = { val witness = Scripts.witness2of2(localSig, remoteSig, localFundingPubkey, remoteFundingPubkey) commitTx.copy(tx = commitTx.tx.updateWitness(0, witness)) } - def addSigs(mainPenaltyTx: MainPenaltyTx, revocationSig: ByteVector): MainPenaltyTx = { + def addSigs(mainPenaltyTx: MainPenaltyTx, revocationSig: ByteVector64): MainPenaltyTx = { val witness = Scripts.witnessToLocalDelayedWithRevocationSig(revocationSig, mainPenaltyTx.input.redeemScript) mainPenaltyTx.copy(tx = mainPenaltyTx.tx.updateWitness(0, witness)) } - def addSigs(htlcPenaltyTx: HtlcPenaltyTx, revocationSig: ByteVector, revocationPubkey: PublicKey): HtlcPenaltyTx = { + def addSigs(htlcPenaltyTx: HtlcPenaltyTx, revocationSig: ByteVector64, revocationPubkey: PublicKey): HtlcPenaltyTx = { val witness = Scripts.witnessHtlcWithRevocationSig(revocationSig, revocationPubkey, htlcPenaltyTx.input.redeemScript) htlcPenaltyTx.copy(tx = htlcPenaltyTx.tx.updateWitness(0, witness)) } - def addSigs(htlcSuccessTx: HtlcSuccessTx, localSig: ByteVector, remoteSig: ByteVector, paymentPreimage: ByteVector32): HtlcSuccessTx = { + def addSigs(htlcSuccessTx: HtlcSuccessTx, localSig: ByteVector64, remoteSig: ByteVector64, paymentPreimage: ByteVector32): HtlcSuccessTx = { val witness = witnessHtlcSuccess(localSig, remoteSig, paymentPreimage, htlcSuccessTx.input.redeemScript) htlcSuccessTx.copy(tx = htlcSuccessTx.tx.updateWitness(0, witness)) } - def addSigs(htlcTimeoutTx: HtlcTimeoutTx, localSig: ByteVector, remoteSig: ByteVector): HtlcTimeoutTx = { + def addSigs(htlcTimeoutTx: HtlcTimeoutTx, localSig: ByteVector64, remoteSig: ByteVector64): HtlcTimeoutTx = { val witness = witnessHtlcTimeout(localSig, remoteSig, htlcTimeoutTx.input.redeemScript) htlcTimeoutTx.copy(tx = htlcTimeoutTx.tx.updateWitness(0, witness)) } - def addSigs(claimHtlcSuccessTx: ClaimHtlcSuccessTx, localSig: ByteVector, paymentPreimage: ByteVector32): ClaimHtlcSuccessTx = { + def addSigs(claimHtlcSuccessTx: ClaimHtlcSuccessTx, localSig: ByteVector64, paymentPreimage: ByteVector32): ClaimHtlcSuccessTx = { val witness = witnessClaimHtlcSuccessFromCommitTx(localSig, paymentPreimage, claimHtlcSuccessTx.input.redeemScript) claimHtlcSuccessTx.copy(tx = claimHtlcSuccessTx.tx.updateWitness(0, witness)) } - def addSigs(claimHtlcTimeoutTx: ClaimHtlcTimeoutTx, localSig: ByteVector): ClaimHtlcTimeoutTx = { + def addSigs(claimHtlcTimeoutTx: ClaimHtlcTimeoutTx, localSig: ByteVector64): ClaimHtlcTimeoutTx = { val witness = witnessClaimHtlcTimeoutFromCommitTx(localSig, claimHtlcTimeoutTx.input.redeemScript) claimHtlcTimeoutTx.copy(tx = claimHtlcTimeoutTx.tx.updateWitness(0, witness)) } - def addSigs(claimP2WPKHOutputTx: ClaimP2WPKHOutputTx, localPaymentPubkey: ByteVector, localSig: ByteVector): ClaimP2WPKHOutputTx = { - val witness = ScriptWitness(Seq(localSig, localPaymentPubkey)) + def addSigs(claimP2WPKHOutputTx: ClaimP2WPKHOutputTx, localPaymentPubkey: PublicKey, localSig: ByteVector64): ClaimP2WPKHOutputTx = { + val witness = ScriptWitness(Seq(der(localSig), localPaymentPubkey.value)) claimP2WPKHOutputTx.copy(tx = claimP2WPKHOutputTx.tx.updateWitness(0, witness)) } - def addSigs(claimHtlcDelayed: ClaimDelayedOutputTx, localSig: ByteVector): ClaimDelayedOutputTx = { + def addSigs(claimHtlcDelayed: ClaimDelayedOutputTx, localSig: ByteVector64): ClaimDelayedOutputTx = { val witness = witnessToLocalDelayedAfterDelay(localSig, claimHtlcDelayed.input.redeemScript) claimHtlcDelayed.copy(tx = claimHtlcDelayed.tx.updateWitness(0, witness)) } - def addSigs(claimHtlcDelayedPenalty: ClaimDelayedOutputPenaltyTx, revocationSig: ByteVector): ClaimDelayedOutputPenaltyTx = { + def addSigs(claimHtlcDelayedPenalty: ClaimDelayedOutputPenaltyTx, revocationSig: ByteVector64): ClaimDelayedOutputPenaltyTx = { val witness = Scripts.witnessToLocalDelayedWithRevocationSig(revocationSig, claimHtlcDelayedPenalty.input.redeemScript) claimHtlcDelayedPenalty.copy(tx = claimHtlcDelayedPenalty.tx.updateWitness(0, witness)) } - def addSigs(closingTx: ClosingTx, localFundingPubkey: PublicKey, remoteFundingPubkey: PublicKey, localSig: ByteVector, remoteSig: ByteVector): ClosingTx = { + def addSigs(closingTx: ClosingTx, localFundingPubkey: PublicKey, remoteFundingPubkey: PublicKey, localSig: ByteVector64, remoteSig: ByteVector64): ClosingTx = { val witness = Scripts.witness2of2(localSig, remoteSig, localFundingPubkey, remoteFundingPubkey) closingTx.copy(tx = closingTx.tx.updateWitness(0, witness)) } @@ -542,7 +555,7 @@ object Transactions { def checkSpendable(txinfo: TransactionWithInputInfo): Try[Unit] = Try(Transaction.correctlySpends(txinfo.tx, Map(txinfo.tx.txIn.head.outPoint -> txinfo.input.txOut), ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)) - def checkSig(txinfo: TransactionWithInputInfo, sig: ByteVector, pubKey: PublicKey): Boolean = { + def checkSig(txinfo: TransactionWithInputInfo, sig: ByteVector64, pubKey: PublicKey): Boolean = { val data = Transaction.hashForSigning(txinfo.tx, inputIndex = 0, txinfo.input.redeemScript, SIGHASH_ALL, txinfo.input.txOut.amount, SIGVERSION_WITNESS_V0) Crypto.verifySignature(data, sig, pubKey) } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/ChannelCodecs.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/ChannelCodecs.scala index 9f5c12b2c..66569bc30 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/ChannelCodecs.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/ChannelCodecs.scala @@ -20,7 +20,7 @@ import java.util.UUID import akka.actor.ActorRef import fr.acinq.bitcoin.DeterministicWallet.{ExtendedPrivateKey, KeyPath} -import fr.acinq.bitcoin.{ByteVector32, OutPoint, Transaction, TxOut} +import fr.acinq.bitcoin.{ByteVector32, ByteVector64, Crypto, OutPoint, Transaction, TxOut} import fr.acinq.eclair.channel._ import fr.acinq.eclair.crypto.ShaChain import fr.acinq.eclair.payment.{Local, Origin, Relayed} @@ -31,9 +31,8 @@ import grizzled.slf4j.Logging import scodec.bits.BitVector import scodec.codecs._ import scodec.{Attempt, Codec} -import scala.concurrent.duration._ -import scala.compat.Platform +import scala.compat.Platform import scala.concurrent.duration._ @@ -74,10 +73,10 @@ object ChannelCodecs extends Logging { ("toSelfDelay" | uint16) :: ("maxAcceptedHtlcs" | uint16) :: ("fundingPubKey" | publicKey) :: - ("revocationBasepoint" | point) :: - ("paymentBasepoint" | point) :: - ("delayedPaymentBasepoint" | point) :: - ("htlcBasepoint" | point) :: + ("revocationBasepoint" | publicKey) :: + ("paymentBasepoint" | publicKey) :: + ("delayedPaymentBasepoint" | publicKey) :: + ("htlcBasepoint" | publicKey) :: ("globalFeatures" | varsizebinarydata) :: ("localFeatures" | varsizebinarydata)).as[RemoteParams] @@ -124,10 +123,19 @@ object ChannelCodecs extends Logging { .typecase(0x09, (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec)).as[HtlcPenaltyTx]) .typecase(0x10, (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec)).as[ClosingTx]) + // this is a backward compatible codec (we used to store the sig as DER encoded), now we store it as 64-bytes + val sig64OrDERCodec: Codec[ByteVector64] = Codec[ByteVector64]( + (value: ByteVector64) => bytes(64).encode(value), + (wire: BitVector) => bytes.decode(wire).map(_.map { + case bin64 if bin64.size == 64 => ByteVector64(bin64) + case der => Crypto.der2compact(der) + }) + ) + val htlcTxAndSigsCodec: Codec[HtlcTxAndSigs] = ( ("txinfo" | txWithInputInfoCodec) :: - ("localSig" | varsizebinarydata) :: - ("remoteSig" | varsizebinarydata)).as[HtlcTxAndSigs] + ("localSig" | variableSizeBytes(uint16, sig64OrDERCodec)) :: // we store as variable length for historical purposes (we used to store as DER encoded) + ("remoteSig" | variableSizeBytes(uint16, sig64OrDERCodec))).as[HtlcTxAndSigs] val publishableTxsCodec: Codec[PublishableTxs] = ( ("commitTx" | (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec)).as[CommitTx]) :: @@ -142,7 +150,7 @@ object ChannelCodecs extends Logging { ("index" | uint64) :: ("spec" | commitmentSpecCodec) :: ("txid" | bytes32) :: - ("remotePerCommitmentPoint" | point)).as[RemoteCommit] + ("remotePerCommitmentPoint" | publicKey)).as[RemoteCommit] val updateMessageCodec: Codec[UpdateMessage] = lightningMessageCodec.narrow(f => Attempt.successful(f.asInstanceOf[UpdateMessage]), g => g) @@ -206,7 +214,7 @@ object ChannelCodecs extends Logging { ("localNextHtlcId" | uint64) :: ("remoteNextHtlcId" | uint64) :: ("originChannels" | originsMapCodec) :: - ("remoteNextCommitInfo" | either(bool, waitingForRevocationCodec, point)) :: + ("remoteNextCommitInfo" | either(bool, waitingForRevocationCodec, publicKey)) :: ("commitInput" | inputInfoCodec) :: ("remotePerCommitmentSecrets" | ShaChain.shaChainCodec) :: ("channelId" | bytes32)).as[Commitments] diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/LightningMessageCodecs.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/LightningMessageCodecs.scala index e6becf2fc..6b6e89ff9 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/LightningMessageCodecs.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/LightningMessageCodecs.scala @@ -16,13 +16,12 @@ package fr.acinq.eclair.wire -import java.math.BigInteger import java.net.{Inet4Address, Inet6Address, InetAddress} import com.google.common.cache.{CacheBuilder, CacheLoader} -import fr.acinq.bitcoin.Crypto.{Point, PrivateKey, PublicKey, Scalar} -import fr.acinq.bitcoin.{ByteVector32, Crypto} -import fr.acinq.eclair.crypto.{Generators, Sphinx} +import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey} +import fr.acinq.bitcoin.{ByteVector32, ByteVector64} +import fr.acinq.eclair.crypto.Sphinx import fr.acinq.eclair.wire.FixedSizeStrictCodec.bytesStrict import fr.acinq.eclair.{ShortChannelId, UInt64, wire} import org.apache.commons.codec.binary.Base32 @@ -51,9 +50,11 @@ object LightningMessageCodecs { def bytes32: Codec[ByteVector32] = limitedSizeBytes(32, bytesStrict(32).xmap(d => ByteVector32(d), d => d.bytes)) + def bytes64: Codec[ByteVector64] = limitedSizeBytes(64, bytesStrict(64).xmap(d => ByteVector64(d), d => d.bytes)) + def varsizebinarydata: Codec[ByteVector] = variableSizeBytes(uint16, bytes) - def listofsignatures: Codec[List[ByteVector]] = listOfN(uint16, signature) + def listofsignatures: Codec[List[ByteVector64]] = listOfN(uint16, bytes64) def ipv4address: Codec[Inet4Address] = bytes(4).xmap(b => InetAddress.getByAddress(b.toArray).asInstanceOf[Inet4Address], a => ByteVector(a.getAddress)) @@ -75,58 +76,20 @@ object LightningMessageCodecs { def shortchannelid: Codec[ShortChannelId] = int64.xmap(l => ShortChannelId(l), s => s.toLong) - def signature: Codec[ByteVector] = Codec[ByteVector]( - (der: ByteVector) => bytes(64).encode(der2wire(der)), - (wire: BitVector) => bytes(64).decode(wire).map(_.map(b => wire2der(b))) - ) - - def scalar: Codec[Scalar] = Codec[Scalar]( - (value: Scalar) => bytes(32).encode(ByteVector(value.toBin.toArray)), - (wire: BitVector) => bytes(32).decode(wire).map(_.map(b => Scalar(b))) - ) - - def point: Codec[Point] = Codec[Point]( - (point: Point) => bytes(33).encode(point.toBin(compressed = true)), - (wire: BitVector) => bytes(33).decode(wire).map(_.map(b => Point(b))) - ) - def privateKey: Codec[PrivateKey] = Codec[PrivateKey]( - (priv: PrivateKey) => bytes(32).encode(priv.value.toBin), - (wire: BitVector) => bytes(32).decode(wire).map(_.map(b => PrivateKey(b, compressed = true))) + (priv: PrivateKey) => bytes(32).encode(priv.value), + (wire: BitVector) => bytes(32).decode(wire).map(_.map(b => PrivateKey(b))) ) def publicKey: Codec[PublicKey] = Codec[PublicKey]( - (pub: PublicKey) => bytes(33).encode(pub.value.toBin(compressed = true)), + (pub: PublicKey) => bytes(33).encode(pub.value), (wire: BitVector) => bytes(33).decode(wire).map(_.map(b => PublicKey(b))) ) - def optionalSignature: Codec[Option[ByteVector]] = Codec[Option[ByteVector]]( - (der: Option[ByteVector]) => der match { - case Some(sig) => bytes(64).encode(der2wire(sig)) - case None => bytes(64).encode(ByteVector.fill[Byte](64)(0)) - }, - (wire: BitVector) => bytes(64).decode(wire).map(_.map(b => { - if (b.toArray.exists(_ != 0)) Some(wire2der(b)) else None - })) - ) - def rgb: Codec[Color] = bytes(3).xmap(buf => Color(buf(0), buf(1), buf(2)), t => ByteVector(t.r, t.g, t.b)) def zeropaddedstring(size: Int): Codec[String] = fixedSizeBytes(32, utf8).xmap(s => s.takeWhile(_ != '\u0000'), s => s) - def der2wire(signature: ByteVector): ByteVector = { - require(Crypto.isDERSignature(signature), s"invalid DER signature $signature") - val (r, s) = Crypto.decodeSignature(signature) - Generators.fixSize(ByteVector.view(r.toByteArray.dropWhile(_ == 0))) ++ Generators.fixSize(ByteVector.view(s.toByteArray.dropWhile(_ == 0))) - } - - def wire2der(sig: ByteVector): ByteVector = { - require(sig.length == 64, "wire signature length must be 64") - val r = new BigInteger(1, sig.take(32).toArray) - val s = new BigInteger(1, sig.takeRight(32).toArray) - Crypto.encodeSignature(r, s) :+ fr.acinq.bitcoin.SIGHASH_ALL.toByte // wtf ?? - } - val initCodec: Codec[Init] = ( ("globalFeatures" | varsizebinarydata) :: ("localFeatures" | varsizebinarydata)).as[Init] @@ -146,8 +109,8 @@ object LightningMessageCodecs { ("channelId" | bytes32) :: ("nextLocalCommitmentNumber" | uint64) :: ("nextRemoteRevocationNumber" | uint64) :: - ("yourLastPerCommitmentSecret" | optional(bitsRemaining, scalar)) :: - ("myCurrentPerCommitmentPoint" | optional(bitsRemaining, point))).as[ChannelReestablish] + ("yourLastPerCommitmentSecret" | optional(bitsRemaining, privateKey)) :: + ("myCurrentPerCommitmentPoint" | optional(bitsRemaining, publicKey))).as[ChannelReestablish] val openChannelCodec: Codec[OpenChannel] = ( ("chainHash" | bytes32) :: @@ -162,11 +125,11 @@ object LightningMessageCodecs { ("toSelfDelay" | uint16) :: ("maxAcceptedHtlcs" | uint16) :: ("fundingPubkey" | publicKey) :: - ("revocationBasepoint" | point) :: - ("paymentBasepoint" | point) :: - ("delayedPaymentBasepoint" | point) :: - ("htlcBasepoint" | point) :: - ("firstPerCommitmentPoint" | point) :: + ("revocationBasepoint" | publicKey) :: + ("paymentBasepoint" | publicKey) :: + ("delayedPaymentBasepoint" | publicKey) :: + ("htlcBasepoint" | publicKey) :: + ("firstPerCommitmentPoint" | publicKey) :: ("channelFlags" | byte)).as[OpenChannel] val acceptChannelCodec: Codec[AcceptChannel] = ( @@ -179,25 +142,25 @@ object LightningMessageCodecs { ("toSelfDelay" | uint16) :: ("maxAcceptedHtlcs" | uint16) :: ("fundingPubkey" | publicKey) :: - ("revocationBasepoint" | point) :: - ("paymentBasepoint" | point) :: - ("delayedPaymentBasepoint" | point) :: - ("htlcBasepoint" | point) :: - ("firstPerCommitmentPoint" | point)).as[AcceptChannel] + ("revocationBasepoint" | publicKey) :: + ("paymentBasepoint" | publicKey) :: + ("delayedPaymentBasepoint" | publicKey) :: + ("htlcBasepoint" | publicKey) :: + ("firstPerCommitmentPoint" | publicKey)).as[AcceptChannel] val fundingCreatedCodec: Codec[FundingCreated] = ( ("temporaryChannelId" | bytes32) :: ("fundingTxid" | bytes32) :: ("fundingOutputIndex" | uint16) :: - ("signature" | signature)).as[FundingCreated] + ("signature" | bytes64)).as[FundingCreated] val fundingSignedCodec: Codec[FundingSigned] = ( ("channelId" | bytes32) :: - ("signature" | signature)).as[FundingSigned] + ("signature" | bytes64)).as[FundingSigned] val fundingLockedCodec: Codec[FundingLocked] = ( ("channelId" | bytes32) :: - ("nextPerCommitmentPoint" | point)).as[FundingLocked] + ("nextPerCommitmentPoint" | publicKey)).as[FundingLocked] val shutdownCodec: Codec[wire.Shutdown] = ( ("channelId" | bytes32) :: @@ -206,7 +169,7 @@ object LightningMessageCodecs { val closingSignedCodec: Codec[ClosingSigned] = ( ("channelId" | bytes32) :: ("feeSatoshis" | uint64) :: - ("signature" | signature)).as[ClosingSigned] + ("signature" | bytes64)).as[ClosingSigned] val updateAddHtlcCodec: Codec[UpdateAddHtlc] = ( ("channelId" | bytes32) :: @@ -234,13 +197,13 @@ object LightningMessageCodecs { val commitSigCodec: Codec[CommitSig] = ( ("channelId" | bytes32) :: - ("signature" | signature) :: + ("signature" | bytes64) :: ("htlcSignatures" | listofsignatures)).as[CommitSig] val revokeAndAckCodec: Codec[RevokeAndAck] = ( ("channelId" | bytes32) :: - ("perCommitmentSecret" | scalar) :: - ("nextPerCommitmentPoint" | point) + ("perCommitmentSecret" | privateKey) :: + ("nextPerCommitmentPoint" | publicKey) ).as[RevokeAndAck] val updateFeeCodec: Codec[UpdateFee] = ( @@ -250,8 +213,8 @@ object LightningMessageCodecs { val announcementSignaturesCodec: Codec[AnnouncementSignatures] = ( ("channelId" | bytes32) :: ("shortChannelId" | shortchannelid) :: - ("nodeSignature" | signature) :: - ("bitcoinSignature" | signature)).as[AnnouncementSignatures] + ("nodeSignature" | bytes64) :: + ("bitcoinSignature" | bytes64)).as[AnnouncementSignatures] val channelAnnouncementWitnessCodec = ( ("features" | varsizebinarydata) :: @@ -263,10 +226,10 @@ object LightningMessageCodecs { ("bitcoinKey2" | publicKey)) val channelAnnouncementCodec: Codec[ChannelAnnouncement] = ( - ("nodeSignature1" | signature) :: - ("nodeSignature2" | signature) :: - ("bitcoinSignature1" | signature) :: - ("bitcoinSignature2" | signature) :: + ("nodeSignature1" | bytes64) :: + ("nodeSignature2" | bytes64) :: + ("bitcoinSignature1" | bytes64) :: + ("bitcoinSignature2" | bytes64) :: channelAnnouncementWitnessCodec).as[ChannelAnnouncement] val nodeAnnouncementWitnessCodec = ( @@ -278,7 +241,7 @@ object LightningMessageCodecs { ("addresses" | listofnodeaddresses)) val nodeAnnouncementCodec: Codec[NodeAnnouncement] = ( - ("signature" | signature) :: + ("signature" | bytes64) :: nodeAnnouncementWitnessCodec).as[NodeAnnouncement] val channelUpdateWitnessCodec = @@ -295,7 +258,7 @@ object LightningMessageCodecs { }) val channelUpdateCodec: Codec[ChannelUpdate] = ( - ("signature" | signature) :: + ("signature" | bytes64) :: channelUpdateWitnessCodec).as[ChannelUpdate] val queryShortChannelIdsCodec: Codec[QueryShortChannelIds] = ( diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/LightningMessageTypes.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/LightningMessageTypes.scala index 0a629ac93..e6019d7d9 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/LightningMessageTypes.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/LightningMessageTypes.scala @@ -20,8 +20,8 @@ import java.net.{Inet4Address, Inet6Address, InetAddress, InetSocketAddress} import java.nio.charset.StandardCharsets import com.google.common.base.Charsets -import fr.acinq.bitcoin.ByteVector32 -import fr.acinq.bitcoin.Crypto.{Point, PublicKey, Scalar} +import fr.acinq.bitcoin.{ByteVector32, ByteVector64} +import fr.acinq.bitcoin.Crypto.{PublicKey, PrivateKey} import fr.acinq.eclair.{ShortChannelId, UInt64} import scodec.bits.ByteVector @@ -63,8 +63,8 @@ case class Pong(data: ByteVector) extends SetupMessage case class ChannelReestablish(channelId: ByteVector32, nextLocalCommitmentNumber: Long, nextRemoteRevocationNumber: Long, - yourLastPerCommitmentSecret: Option[Scalar] = None, - myCurrentPerCommitmentPoint: Option[Point] = None) extends ChannelMessage with HasChannelId + yourLastPerCommitmentSecret: Option[PrivateKey] = None, + myCurrentPerCommitmentPoint: Option[PublicKey] = None) extends ChannelMessage with HasChannelId case class OpenChannel(chainHash: ByteVector32, temporaryChannelId: ByteVector32, @@ -78,11 +78,11 @@ case class OpenChannel(chainHash: ByteVector32, toSelfDelay: Int, maxAcceptedHtlcs: Int, fundingPubkey: PublicKey, - revocationBasepoint: Point, - paymentBasepoint: Point, - delayedPaymentBasepoint: Point, - htlcBasepoint: Point, - firstPerCommitmentPoint: Point, + revocationBasepoint: PublicKey, + paymentBasepoint: PublicKey, + delayedPaymentBasepoint: PublicKey, + htlcBasepoint: PublicKey, + firstPerCommitmentPoint: PublicKey, channelFlags: Byte) extends ChannelMessage with HasTemporaryChannelId with HasChainHash case class AcceptChannel(temporaryChannelId: ByteVector32, @@ -94,29 +94,29 @@ case class AcceptChannel(temporaryChannelId: ByteVector32, toSelfDelay: Int, maxAcceptedHtlcs: Int, fundingPubkey: PublicKey, - revocationBasepoint: Point, - paymentBasepoint: Point, - delayedPaymentBasepoint: Point, - htlcBasepoint: Point, - firstPerCommitmentPoint: Point) extends ChannelMessage with HasTemporaryChannelId + revocationBasepoint: PublicKey, + paymentBasepoint: PublicKey, + delayedPaymentBasepoint: PublicKey, + htlcBasepoint: PublicKey, + firstPerCommitmentPoint: PublicKey) extends ChannelMessage with HasTemporaryChannelId case class FundingCreated(temporaryChannelId: ByteVector32, fundingTxid: ByteVector32, fundingOutputIndex: Int, - signature: ByteVector) extends ChannelMessage with HasTemporaryChannelId + signature: ByteVector64) extends ChannelMessage with HasTemporaryChannelId case class FundingSigned(channelId: ByteVector32, - signature: ByteVector) extends ChannelMessage with HasChannelId + signature: ByteVector64) extends ChannelMessage with HasChannelId case class FundingLocked(channelId: ByteVector32, - nextPerCommitmentPoint: Point) extends ChannelMessage with HasChannelId + nextPerCommitmentPoint: PublicKey) extends ChannelMessage with HasChannelId case class Shutdown(channelId: ByteVector32, scriptPubKey: ByteVector) extends ChannelMessage with HasChannelId case class ClosingSigned(channelId: ByteVector32, feeSatoshis: Long, - signature: ByteVector) extends ChannelMessage with HasChannelId + signature: ByteVector64) extends ChannelMessage with HasChannelId case class UpdateAddHtlc(channelId: ByteVector32, id: Long, @@ -139,25 +139,25 @@ case class UpdateFailMalformedHtlc(channelId: ByteVector32, failureCode: Int) extends HtlcMessage with UpdateMessage with HasChannelId case class CommitSig(channelId: ByteVector32, - signature: ByteVector, - htlcSignatures: List[ByteVector]) extends HtlcMessage with HasChannelId + signature: ByteVector64, + htlcSignatures: List[ByteVector64]) extends HtlcMessage with HasChannelId case class RevokeAndAck(channelId: ByteVector32, - perCommitmentSecret: Scalar, - nextPerCommitmentPoint: Point) extends HtlcMessage with HasChannelId + perCommitmentSecret: PrivateKey, + nextPerCommitmentPoint: PublicKey) extends HtlcMessage with HasChannelId case class UpdateFee(channelId: ByteVector32, feeratePerKw: Long) extends ChannelMessage with UpdateMessage with HasChannelId case class AnnouncementSignatures(channelId: ByteVector32, shortChannelId: ShortChannelId, - nodeSignature: ByteVector, - bitcoinSignature: ByteVector) extends RoutingMessage with HasChannelId + nodeSignature: ByteVector64, + bitcoinSignature: ByteVector64) extends RoutingMessage with HasChannelId -case class ChannelAnnouncement(nodeSignature1: ByteVector, - nodeSignature2: ByteVector, - bitcoinSignature1: ByteVector, - bitcoinSignature2: ByteVector, +case class ChannelAnnouncement(nodeSignature1: ByteVector64, + nodeSignature2: ByteVector64, + bitcoinSignature1: ByteVector64, + bitcoinSignature2: ByteVector64, features: ByteVector, chainHash: ByteVector32, shortChannelId: ShortChannelId, @@ -204,7 +204,7 @@ case class Tor3(tor3: String, port: Int) extends OnionAddress { override def soc // @formatter:on -case class NodeAnnouncement(signature: ByteVector, +case class NodeAnnouncement(signature: ByteVector64, features: ByteVector, timestamp: Long, nodeId: PublicKey, @@ -212,7 +212,7 @@ case class NodeAnnouncement(signature: ByteVector, alias: String, addresses: List[NodeAddress]) extends RoutingMessage with HasTimestamp -case class ChannelUpdate(signature: ByteVector, +case class ChannelUpdate(signature: ByteVector64, chainHash: ByteVector32, shortChannelId: ShortChannelId, timestamp: Long, diff --git a/eclair-core/src/test/resources/normal_data_htlcs.bin b/eclair-core/src/test/resources/normal_data_htlcs.bin new file mode 100644 index 000000000..8ab3e80b5 --- /dev/null +++ b/eclair-core/src/test/resources/normal_data_htlcs.bin @@ -0,0 +1 @@ +00000603933884aaf1d6b108397e5efe5c86bcf2d8ca8d2f700eda99db9214fc2712b13400048aa09cd9ebe0abd1b3472fb124ded55e0000000000000222000000003b9aca0000000000000186a000000000000000010090001e000bd48a29d3e72e2e43e86aa964fb38fdcbfd753abfaaf64380000000c50163e0a31eb5c22f1f7a87b7e009d16d7733942e0bfde612a091c9ba61c6e8d9a9800000000000011e80000001270b0180000000000000c35000000000000001f4025880f181fe8a7089484cb64fb4c5e0a7a406c63ca1c1614ab50f12489a946ff10b94684801c30e81fb2bdd88687c71ac373915d4fca7cd616cdaf48536dbd449415af09d038193d0fca9dfebfe58a4c5a8512d7cc2c2d31232a40b32d06d1feaa476dd4785f8817107618584653b6af9ea03563fb79ea987938acbb77f943b052f314a7d3251c381335c6682ec604acff2dd205159fdf3cd2012661d51328f7d98dca7f607aadaed800000000080000000000000e30006be6ee8f639d4fd51a95550ec539e61188e14ee5e4d9ba5f7a61e2e8ba81b2498800000000000002440000000001ab5e7b0766d6ad4d8ad12d78a91a9a7f6fb0a5897047c36ab4b7f5f7f2d5ca63d600f0004f3f08000c64d81277f8df797362e9d9893d9eaa680403958c9dd2120a4542dfe16a708425adfa7d7dfbbed150b45ca1c3ddbaf7bb2a85b58fb72ab07b72d802bedc8020f5a7b3748a99de4d2303409c34517e46da1e6931164497334de0922a17869bd6ac6540494337e16515ad7459beeda293c989315018e214bdd3703f89b3c820ba133e3ce1dc34353cd06e608ebf22a3ec7b8d87a7a69f64a3d7de29c4a12d063150f5f257f382f4faa92fa4ff0b01b03734f51f056731f60ae807c8ff29a2584c25cc60c7c18b65f466a0f2894a52e8666809121081e8626a397dab76ea7e1f29bd338d9b64276c396ed23b0da0f6264067b4a71b523e268128916b7fd099b3f19acd92c07e9308674f24496f2a45f1ef514da9aeeaffd47a0542a3ce407859a9db98c24b759775baa93ab8cd6d2f5eaeec741caa83c4d19ad81812a3df8011b0efc48f17be31ee77f0bf15b868e1165904c993c96106dcc418c0cee4ac51bbdd65bdb216c306d07e1f1a0b3e0803fe92812c761a1bfcd5c413141e496893302f42cd3ffe20ea01b22d7e9f36f1f7ca7ebb70a86e1f0652b5b574923c4d2ff713c24bfabd1ebf10c8d0452850041169b5542b2e089e150a21a1001622273cd58a08baea60eba63f352245b55e765299e206ddb7a74c23b2047ec5dc9936bb7e3e3cb804d7b4e69538a658d958fbacd38ae13bc4694f3033c5f18c19d97fa62a0bf79c995163ae59e71ef24d1808d844e7b2014fb4934d42b42f2e17b5424bb67be72b1c563263acdf6dbfe9f37f059bb7845ef0bd356297954f94b4b0846b166f230daf1505239c68a2e45a8e4e3fbed89d08026b88215601b321f61760537f89521a1bc3f06ae43291f4c4c86fe19b4b23342db31f97cab5af67afdbb4f0ce43497087de281b8c4c7a49d9a2d7b64e5922391e4de309136eaa36c7fcc3987c27786f6a6c7d43ed0d2be08d517c5f85b876bb85929d7931d228125171b14ba49b0d05121461261b85a2213e6abf4e14ce1428784cf41478cf76adb6998bbe7acb8d49ab106925a54582fa733ac3fcb8aefd816146b47077013de0588e3fdf69a0e5c86bc99517bdf91ed5271ad0309ef5853114578eb1ba6cdf74f7fa93ecfdfd9d14801a07a4cf405980307602bd05a34be0e0a69e0005feada3875b312dd98ac793813e495f6d2338149b358b0c0ac2feb26b205c1fffd21e0e1d6926709c10ca320fb56779a3c5800253a1072c3c088eed2d24b87f17f5616786dfa64291d4714f5489ef61b76a50bc6b51b2a98913524c60d219f1f59551ec09b01ddb833a2fd0d53f36244d4dfa223721eda19b12e208eb171b050b89cacee3e477912060a8ce31a82076480988c0fd192d564553002569ba614a64b24d232a8d640f18cc7a0d98659ba763cb67d545f13a670736e4a8310a023da4a16ab739f00d97ca8e1c41c67a9270a0686b1cae858aeee4f7b5ccf6dada9d3147268d7a399d8b6117209f6e69294bdbd608fc9d3c8aeab8f317a8c39701fad2e7b2777a24f62de774d6ab516b9f0fd94f4c103fa78e58ae060f3fdf8813ab0c6e0bd87acdddd513f3a312ebb1acb62ca8cdcce4a05c766ce6fc0f62e1ccbb491ffa00b4eb0a32fe9616f2729d88f3ed2250ede0e8031d4ef58f092131d6cb450b6107d42ed925d65f810e2cac310c1d74395eee91a5fff1134160d80cca040fb8e1f5b86711512e741b50eba5a264fae0fabcffb6fa9bf7b18058d3f00c5e06c88b7b735224f7c09a5bf403c158b50d9db3c573a5255779b580248a86d27942797cb4d420e303807c32a3e6c518ea338e88b9116f32442c8788212758df45e67055700a204b4cad69730d7f16ee9e2a496c8c836da5852d46b037ba4c040ecc4b08927510d8df267d39be7338c764d5c4ef349e2218b4baa74512f01d59f37747b1cea7ea8d4aaa87629cf308c470a772f26cdd2fbd30f1745d40d924c400000000000000da000000000280ee74ce4aa3fe4a682ec1331be709941742aa9f4a3ed1c9ccfd5e24779f4b6d441d9800279e82000798b8e9f9b3ca9322cc5c1d5067f12fca6590c035eeda4bd28b1ba6e0ab711d32fa015120d0c247f45353ff276038218eda413db2852a3b1c60ec17e848ca0865aa58556e0ecfcb0c18f0cd19e5163846d4e83ee0379ae59ddfcb2a6b5450818fe62d5db8acd98fb06d1e0f83aebd838dd105b0ac5fe11d655532ab08e2078b5341fa4b0b9d4bef171ceb512f9f08119abab772210ca4aaf2006582149fe87395ae3ee48b95f7171d2752581a2509f9ab8f0bdfe3255441ff49d9ad00e0f321c44561163303396fb45ba9a519044a628518e2fc760cfdf940e62304c8af508d84ab785667c3c97d51807d9a1db54a11a2ef5fd90c7d72be57fa10b90ff7ea8060f3ffcaf5b566eeda360e1b09d8ed4f5ec48e6a6b2ff896216df219e134acfd19fef3cceb2008ccc144cd1261aaa5ee8feac3080d5365ae26dfd4ffcf8785e369d680cf10a4c526250552744c218917c9817d96537a7c00805b062fc328c00aa3646a19eac97be19309a2b5b04e9937d39ad5cffba726cd41401b9fa0b81acf961ac68abcc4c886bcdc2cb2bbbc9e0582274fea9c8c66fbe195a589495c6ef9b6951cb175b1e492cc7c9df43a3ea744443e18d7066276a2280d89ed42fdc4cf129baff50d68b143cd634f6becfb5c588353b2a59b29bd3af9070974a47e93577f25ee35b4d7248bea0935946b8f65c9508f776e10e92ce6b9d21f57b6c6bb6b798c57bc3621b233bd407c8320208dbe7a5ffbb774bed5a565cba57f5ac9b1878cfc71042fe91ffd4f42a9d6a33bb1a1a91ef8687e71e25227453079b2577cdf12d46b3ceb191b635e1cc0e0e716be1d46f377c52ba80079113ff17ad33b7193e0bf6cd9b69b570f3657d9ca8506df824c0e4cc938bfc47749bb7f5b7bbbf137d4ab601489f87ef765e6d33f70c9f290bdc43dd27f408db30a374b7649baf5cf6c77bb5aa0cfc6d702eddb34f153773b9750b38a95672f85b9089831d625db42eddf267daf4fa3668c83da59c99b98e0a51eff24def543be12287a3d1cbd100f575c7dab877204ae726955a296a96b58baaeead06b43310205fa93681bfbe2245646fac5a3b72c369f2e4a0ef4fc7c383c27833807d22e1b232839e304cc15cdede671bf5cfbf50a4183bfd4d005410f56c730c1fc7408b6f6497120e9f0a914397bc1e8b50a642d54368f776da20762889e84075e04ac50fc05da0bb7bacd8aed3b4f8c37d7da95f7aa8b06ee990942a8c30d6ae6651ce622f50d1ddca4d03201ccf2122cd685c0eaadf585a4012e4f2e6e3d9676a097a7b0dcf286e1668715e9057a0552fd3e55f5ad71e113e0fceb5539bfc7de7fd8fac8ef21a34dba6db00dbf44da0e634005b4962cdb972847d8d4f17ae6409d5a5e1c835312320ffb93b1124dcf9f0a3bb57e495f7d8a06d547cebc731309c64dcbc56882f58d60590bcf122e061e179259f83b75e2da6811653fe4dcc42801335f6935de22b2475bc5d0b90465d629236afd366ecbb989e27ea8c72a5633166d2b44a61935c65fd89c5d0730ba6fbe1ba915a0218ecd6ffbb9877ba140544953094f832c2d1defa0d9291f341aa34a6fbbe8852474de0297bc15203ba023e3efede2973d3ccccbd05ea3bd981db974c3f80da002ac1ea33898f677d69b3dcd6f272feb40b74ddd70a0186092d59c206fb0258fe333f6af94ced4786ce8def768694d20ceee9a3a6dde00b76134323ca7b077ede196ef2a607c9c93bb0ff20b4e7a4e0db9a56dd4d612883b0d84bc43525379695c7c8d2e7c6e4638ac64418819fa3871b90251c73b76b0392146e65cd834d76207c620767d7deda968d9752aef756d0a554b03716a5f06035e5e8fff8b95be5d74d68c5bd33d3d34bd4acffd5aad3f4cd722984f3f5734f8d6f972e2b8e512afc21ea2a8b7489df8f8ef9ef9bba3d8e753f546a55543b14e7984623853b979366e97de9878ba2ea06c9262000000000000009200000000007a19e0b13b1f490cde2990c6d41d4605978af7fd41e091619d668955d49de55bc267dc0013cfc20002b85ee7d3b4817d8944e79810fca7e1f783a5f4d8b10058423c5c2f798b5bdd4607c085fdf8385921ba9c3b29c0de8314382660ee48efa208f4aef129dfb448f0a55620f3edddcad9ff3a6193715a4bfb3d9d5632262de12ab44b1528233bbb1cf226467b1a7c628642ee998020986196660764ea0da1ad8b169a69ad5694cdddea817de990acd7a1141e677fe1ecd5dbb0bfa7b77adfacc7d84c22786baf9269adddc221f3f280fe1d9056c52798656ab3dad47d756109f7042dd58052c96b14e8faf55134bf4d70ede0f6c995052ad614c632eb33e33c4a91ef9d9e4dde418d94abb00189601117ac59dc41c97a364e73f75650cb55730ee48566ca0e18d055e56ec9bed3c036bff80b886d04a53670bc1d753a27921a03a7f1b6369b7251fe1f9a0a1d6786264b971eb87bef1805955bb658219846e9e66ff8fcdd5a873934631a96c311881fc65fc80e3deadcec01cac59b4ba8c79510632418bad169f558b130e1888e38b7c131790b3120f13ffb69524d9dcf228b1ac8e60540034d62ddd38691ba09a7f2d8338f11857eac1a3111a0f6ef9c1084142daafb3ce1b722fee3b6e1763d77648755a371ac2f56a0a1ca76ca7ffd6e4a5e4d202a1fca36d78fc3490b47a25d372a423478e9bafff1dda80d88500353a7c652702d6a4cad02de64120754a206d383930f6d164b3af9b680775897d39cc49743f9067737c217ef5974450166391f7f67b83e91c61205b78f0da7278f9c34d7aeb2217bec7244dac595d69529acdeaf6a0b04a26c19a1612bc26935f54f8266a0cef930ccd7c89e202ac4cf614ca6cafc11f31aff0ef93791c4d635a4836e52ca57cacbe1b34f13454dbada065e404a06a2210a0da9f06e8294077ca75159a4f0de67580387808b31f84dce7bccf1f42972929a1a028f4d91bc6565a54c56d53d5f6d9874c86234ad8000f16402fe2d3ac5cb210d3681ddace4cd97a37fcde992977b353cc83aa9ce18099b7e681c0251f100146594312e37c808b3bd58c5f057774fd375468f3db2c9ebca8d1612bbcf90125a500e8d2e12f92386f0eb454aeccc85436a2f7ec23a893029286a5abb8d91b9b843414fef8134edafd08594dd35b26ccc17270cf30f451dc0a90b2c9c3eba85f9cb326608089ceb5fdb2e04ed610617f17b6dd2a08da5dbe7bec0a431a4547c88702e428a6c2e7625d93cd9ab3a0c9a03d75499ac13be519709f94eceb94cdf97d0a3e8aa633a29de78d809a337e0aaf3f81c231fa8d88333ddcfbdaeb2e3a85a67a2cb16302a2293f33b422f3b8b28679805052a4a4425696987d4da4d15592230fe829381d820425c21e08ae3b6d124839a8d669d382f7cb6ea09a405998ab5eee86be1642c99e014e498cd99688ac473f9cdd4476827b5d1b0e3b29f6d4e4de9b2795283f18ca90e53e71f5ef0802c07d126309c77fc0faa6f8fe7b627c23c241df8c5ec4b632a220aa9a9b00f0336524568546b7aa65b46e8c471c7f90911397d1e546a2553c8f3f2ef8a0d6e8d912b2e37dfa9b86e15ccb9e0cddfb6d4ace2ce686a13e5fce2f2959c595bbe189aad01ea2a3e8fe2c993594bb3730a93a7d9ad6569045617ae8676bc7f858a40a9f90e97997393c56adbc8860a8efa57e32109e0b550918c58483cc5d55c5247145ddd22a93f684e3a0b26ac468a5cb56132467aa2e606b4c232ede142ebdc5a1ad97cc417172aec13ecea83cf37542b4c18897120f95efaf194df28e9ee0c31986ce4125b65051e91aee8d62f07a653bb0d82e9be6c60421e356952324ed99cd80f0eadf922684f8b1742eab47372750913c7dc82be79c35b242f7face416a3dd59c232166dd2d9769be98b3c30a574443b9391ac2e3e971430af4330b97464ac55dcf26a9d10314a9bb8aa750c8ed239a0ab04890c6dd2427cddd1ec73a9faa352aaa1d8a73cc2311c29dcbc9b374bef4c3c5d175036493100000000000000500000000000bd974048ab479e0520986ecb9fc97c2f3f77066d58371fd8e17031648dc56a6ce18d3f0009e80b800191a5a55b34c015edc52c63d02dc0ebd36e55f232742af4b0dadda203a85be55006a6fd342f33e1e0127dcad83b96e78be0aaed69a5e9746612bd4b777f4a88698e1c4e7e26c8b1ace76dfcb252703530f4adbeab5ac9533cda1f24a867f711004be0483e5f7b9576bf8e37ea5c987a16757d59246241c414ef456b2ab103e0d70e7199fa61b3ee7291c26263a7d9c60df8cad562c4657645042f90c7e21ec0578e44ed4135ca892015cf27b533539e5aa2eecf33eff86ef199f906e8bc05c8ae9bbdb276e62b9448d3d061c2e0b47bfce53b80b4db1d043bd8c5f2ee2efb0beae32e605ceea19d90fcde0c447f74e38e862ee0e3a11ef62f996dce3fa8aae6ee1db43af7596c20985a0e347d34067a28cc1cfab85b15a675eca00f94c084ccdda3c4aa5d1d2421b52510a4c663abf0e468db83cccd471284a5c72d2bd521dbb499270a6bcdc2613f1a38f65e0c95e475bc3f7cba32ddc925316a935a09f3c1422e6ac6811c1d032bb15216fb8db378f90c5e51f52654de7f49264370224fb96f43ce98dc89e193c8a684716b42d0bdd5d35789feb0cef0caa86b6eddfc7ddf7736dd8c5f2b1b1cb0941f189c83de691198684940b2e7bc827143bfaecd87d01430a0a780611ded71dbd6dd4059181858579d00603843e5ce45b678c603a5735a207883705a317a97c4c0dfb6b1adcd5f33b5282bc0f6a5d29b20c4cc65ee102078efefd3b4f15db0693fcbddfd79c0b4c61cb97191f7de6c8b1baea722fe708842046cf80d173c7111bfa72cd9d3104bb7594b276217dcae594c55d338fd31da4faa7416c4c9d4de44be4e74b2c085daf077a78a7c6787208a108808d1e58381daaa818af52c0184e133baf6d5d51b42fd4fce67c7b13d03ef1435cb4b13759b67b6de37fa6610629fbc1480060cf345822a1d4717e93adb1bf8b0b89e9276ae3883900e34f637c23f8376600d905163bccb1643ee6caf6c4885402c7372bfad85ea8ef05d1aaeeecf5e8278238dc229667b9746a74f4a41627e7fec630212737bdf40c58326adb0949779319ebf40882cff7e966dc1e3ff2261323f152c2b0d755628ad91fabe3d7413bb8e9c13f3b18c593a1c105d26a2101bfbf65f08656ba505babbf9f441a113295c50459af91c689bc1c809eea5ada4743b5e4892af2fae8c3eb767b8448abe750479b10b87b85d5b67f1cb51d42e907284852c6133beb3d2556aa73bfe52cc735d356884b7a46bfbcb7a9fe9ffc065768a9df1db952b875cf7c90a0d63380fc904ffaf9621fe454ef127917d48da1abbfcf2c4f55230e9d4f1f89da4fa062588e08ca399c8b95f1e84b76f0d33c65a1cef8a155496b37560ba7b854762a365a38343afb85003fda7fefe6ce0dcd41cdbfa820a54b558cec733913e29475fd6f090b5ce0efe786ded9feb4798c68263ca86cd995834da4e2c202064a510d330d6abde864ebda2fe59a7e31e9ff2f35959bed85eaa744ef064600b56a9317b010eeb410786583aae8cca25d2f7d961f5d238977cbfd17187e56da017e0771448a174b1206a7f7d6052f4d36fcc2d1b08e276e1a3dd233a38da6a5db42f692351086df44f586911264899a37090f026e8f33cb1fcc53d17d2d1a022ab321102bc392baa62f3cf4fdd7d53be8a37b3dbc437e4c733a01ff29af69847be751ba738e71221b931f0520c191b3ffedd2dbd281dd95d16d22ffe26e9683f94709ced4393ef01e120dc0fe3f8645c763b8a01d2c6d2a52b315dd45550ef87e16db208b5ee19554131c0880ab6bd052bd49bca577e28c8d0bba3096f2d5173882984f39c81d97a569dba83f0c3d8ea9d320df067b34e018f3c3175bf38f2971f7154bbeed87e3a99973da1e7b78187ab51e3c7f89ca0324a3b4d3fcd1e11eaa4232deee406e8d9947f22e71b9f7ad6be6ee8f639d4fd51a95550ec539e61188e14ee5e4d9ba5f7a61e2e8ba81b2498800000000000001a4000000000501dce99c9547fc94d05d826637ce13282e85553e947da39399fabc48ef3e96da883b30004f3d04000a7444db37894e47df91479b60b1335825b871dd010f18655e54e9d143fdfd54062ebe981e582732d66931823a26fb77bf875abad628c50cdc9080a706ec16fefb7ad637ef20efa227d6225ce0938b95294203dc63bc84dc2b07af57abba39da2084b30c79438efbb9959998982c566deea90ad09f22bb43c779ec8ee347c06c7120c5825089812f62003549d1b911c9edc603eb07d18c8cf51c1176f7c2a27ea3a6616b15f7c2b22fe18d35bf1d168deb061e56fc385145aa5d63a039fe22d6618df1326056300b703b897893bc0b94cc03351d1a6a25c55f01d59bd43fdb2a487d7ed2ccd1695e5ad2e59b68a14345aa75ba96308b0af522cb54a7ca7040b841052a485e3679b6ead102c6b471a1f8456e652bfddde9eb661897cba26515c6b82b4f55cd4d799d277c50e2b7cd4c4e1a6f63097168800872dc5dec2bc44fe369940e26b59aebb66e425dbffbdcb9a6ef33164de5a9d81d43ea5c3103e2757afaff41d53a76ae84ad0d8f9dbd3c76878aaae533709a570e96d2d0a85184fdea638ea7e4ac6ca16de92953bebd42c5f6b3e8d4ffb43500158f0478c2a936dc0c112caba2358429b63aee58aa47936985bf3b3d916b676650b6c5c29d838faf8929aa845b3e3a83153c53cd8a05d2f535ac61a578666ed8cde4022941aca7f5930565b53ce143159010c2dfcc0270d805e731d3bac83667f9b9ae98469f9ae4e9775ae6d015e324fff71b462293fe1cd3d588be2c2ad34a23d85378e92d3058189155a80d749d85a88c17e261ec6935744528b8918771102103902071352b4307be33f21edea53b430de31df27efdc35b5837f2b7c47e6662d2b24780381d23e0ec680aaf327e777515de29b023f6029b20ff1b5b72dee915902276567adc53e0e76f493d3aaaafa8b83d40139b136d63308edafb995580d953ae59c99b4ea885fa0f108e7bf38dfcf1ecff68897fc5c92e62aa45f65e15acca5459757bf9e54766c7cda7a7b73f2acc5d6758c83c5e51cf8c3839eb06190389192282f9e23ad5e18bf9fa1de9f8599c84e14a4bc6dd499629d32028e03f898c348f5dab7af5d43414eccc5bccf38badb28456e4c16be8942a541c25219893df05a083822181bdf50cd98d262a5347c6a1184af717d0a8f136b5ce874475b4ddd2e233351c5b46c0453ae63fc7f6351532cf2db54b95790185653e403f708f1ead51401488da8e5971d0d7afd0ab01376b88c64e39395d41fa845bb5f520519d68a5eac7fda78f7fbc1a05923c9a219091dd0bdb3f4724b1fd2ff832f575b56bb68d650eba6be238fd78ac0bfc6f7aaf3a336789224445cbd102d1cb6367f02ef1108c527eb9f1c714ede960049503d1f6b4092dabfd4f8f8c623b708192abb857397bbbe089ad674476f20922a8a3e2e770966b3b00389079218e256cf549a9bb8cac8decc990ae74a3f84d95995b545ddb4c38084a2414662d21eebc6032d71e7ed9e30365baf259dd9af42a1f08ed1d722d38eb0267e586f87d7e1f236668e8fac8f0eab4503eb3c0ed7bc29e456961f2105759548ab71850a7680fbb3fd8e8e1c20f3cfd56ec8d019f27b187eec771b6ae7991b67d3f064a173e327b9a13c095c09e62f3fc459643719681308fb2f77b9f2da322f80d6b11f8eaf54e1915326e69bd285d792f98731f77824812ada1e929c18e28c27666af55e7697ae0ccea495a0c5a3297caaefdb1e59846c2a81fa44c4cf31768c187151c0ce63503f0371d8f32c64abf943a73e69bc12bc0aa9544a909a3ae81bab385b9df0a09434c9e6d6d54d6d49b70878b0af159e2228712370cde5d2c1ad24d0c820de2c8a44af93d8c6dd8c531969d68d2920a3ae9b974d5969bc5099912cfc819dd7a42ef28e894487a7b86969bf860364104bdf83a405605b2d2e1f35b3f0a4201f37747b1cea7ea8d4aaa87629cf308c470a772f26cdd2fbd30f1745d40d924c4000000000000013c000000000280ee759b8c631811bd02dadf40a7609229fc09a3b056021a62bb889b6a1455470c5e940027a02e000652abbe571d6bdfd1b13588cf2fa82a16b5fd21fd9606a86075391d80da708cb523eeca53d3f44cc8bd1e4538447e4318b865bddf5ec2407ada7d8cb160804b021864bf939e9556cc346d17467369575bf088effc647cf1899901ed6993ad5777261fc6228ed56761345f6829408bfd7d815cab16245ff615d95d4fc3c1db690ae6b28453e3377058b614ff5057f002438f7e1ee5173a089bceb09370d869fe6ab95075877040512f1c177b51d2dfb596effb2271c07d500536efc8c784658627fc633a760926d06a238a6e6f4e1cb59b5ce4097b4fa7192c0d9b6f54d83b599c2c463039cb2c8bdd214b01c6f691ab92f771e8b8323cb76130f2bb7b499af6c0c9b4b0674441c72b692b0ddbf7fbf0175f833a1d6b13923fb0559c9476cb9c60d66a3d5cf2d8871da51349295be01bf264e5f71af5476b9dbf8d29eb7922699df33e258bfc3e64e231bdfc8d7200c3d17b935e7e272e0d19785a0705f4b452430a244cb24d7b996e04f21708353d2e438693161aba76b5e7088387864f2217b3d66022d9bb675610f4f275f1048da2ca112853a1e2a6de2712052f18b80797d1d474206eb9065b818c687bba60b2912c17d446311742342a110f517489ba19867643375598ea59f66de717536c23efe09a347821498704b2a80ecd83218ddd1ddf18283c47cdf9f8779465964b71d89fa4f5335c18fcd8b81a23eca0f7c12a401a4e8de44e92e404d8e02232e74324f9d079a3f60a01216c945ac834a2e9fc5c0f7c9fbbd06f772d6eeb9424d648e8ce868b6a90fd118730bafd263d07291f75bf9842619c87ff70b3b7cb490e3d81acda23a0634312b4f364aa20436efa8bd9c2bfe6458d28004efc505dbea0bf1f1db204d9dad0c02c6632e38dcadae197e2d2b93a302e1a97050938ecd13670caafee8325de541e050f85092fd5f7cee7dee83edb693368ff1035b3b13f542c536ecec31d83cb4888de8392cbab380ee3e42a06a22063329cf5ffc1360eee0679419746215d98ea805b38bccfd750ed1d8da0d0b247e98fd45c740d7c775881a6afc62a2aa397e8c826b0cb27fcba5b84c72abead101c2150f96d986c2720518d12881ea11eee207ed9147f313b8b0449d98db61a1b03facb0ea660285e1916eb2330403806c1b84db0a2438fefe36217dc282d3273a8d719cea1b2e2c0c85548ded1b60842e49da4889b0079f5bcee761778271d6e5bf38dd09a23d5b068cd1ddd29fe0030defbc3789c5626357ee719aba91976004cc93c1f7d40cc3e49ca6afc9d7d13e8b9dec8964e9840cd94950cf71d6c08881d2d159772188a2d27dd4181a245bd31d80ee41e80dd8563f9fb9b7afc3db96270e28a62887dd55f1814466821d26733bafdb0447497cf4858caff36d7888cbd6a55711547b20d5caab356fa66a5a7335bc2d29d306fe22f510968634bd1c3dc4795408b30b1e265f9c572e6d3175c2653625250e03278bbc2df8768f3107953dd5829c4ba8c1e922199aaac5a9cba3c7b0a68ffa022b595bfdd53397d0b500c85b914d13d82b0fe4dc8e967494ea42715e6a2c28b9542eb4eda54d77b8bc707932bb7250796c7e535f74c3253d8b20c3bbc0995c94c1ce8470620d375e9b41a2cef35cb2ffac60b6bb9e186401f784dd05e4a3afec617eca89d3d6eed83425fda61570a985b03566206b088c66859445b73810818f623cd6eda25a766a8dbaf7041634c52559cb50305d700b5936776d6619d915e0225e9846b9fcfb907430bd28cbd4b896fe45b91a123db8fd7fcc68a5c6f2282d991642fa12e274eff3c4be15151486c34359d93f87c8f35dbc5233e0fbb36862abd2929b5614b25404299952e0d2a675e907f450c2a32d0acd81dde7ef9bd593c51c359c491bc72d39403026e84a7de58b79303b9f9d11d098800f9bba3d8e753f546a55543b14e7984623853b979366e97de9878ba2ea06c9262000000000000007c000000000140773aaa10a771aadf2066f73a0df6d0729503b0ac829872077cd12a6556d458d75fa50013cf580003d64703dc8d1ddb44911b32d8f21964a1a4f63a782996e0e949d559b5c18e0dba34cbb19c7f1dfcadeee9bcf8fbdaab20e6d3480f2ddd253beccc0f3c4419342af6027df59e3710064fa6ec1cebb92619faf9e087bab4334391d9897e5897effc9a39440116fc81c2c5e66ba8758b092493c0b07067124bf3784382138f6891033c4f21186d03efba5eaf8905179c168e0ab5eba571e7049fada61217f704a3ee67cc988375d7235a3fda5db1cbd5ea602e5ed8b8feb1fa5a311ad0517bd446e4fd6a2140a513e2c8349f1616288e28a4fc48e689fb176c5fd2fac4ca5d67fc9c121774c16bba802c6a3210e097b4d56c103a382d4201380faf93e3445a7fbff1b63611b2daa00a9e8ce48f7f94c59aef7db6966dc0af68369d8b6c92b770e870e6f77394045b15aa80fc411a5d126b5a813024fbf7ae11cb242ccca168eb18dd324b22e9157a712ed0c273ecae30219c8e51a8106835ab7b4a52550361c85ed627916ff0eba09c2dc2415aeca81e1f2a8cfa9e5b3cf9d88680ff78539e1275daeb78605edde4f7a882f3f2b161002ccb183efe944b5bc3677c98091559505842b2df6f35164e4dfa50852a45972805625b1e91c8541540c65391cbadfe9c999b49b38a053b6e85506518ef53d7144c5eef1dfc86edb7868e3f25dbbe7242e6ee7899c4e423561a3b3eed47fa25b3f53957675d70aafd263bbc5aa06fb89f2b41ff97f5ecea14afb0cbe057b424ed5a3d0a55c9d03e60eacee19867a5231a5abc602369dbf3fee5c3681293e6efd05c15a6021d33e4b68ddb6b589fe176c806d0f1610c401cce0ae023ad4f6fe98fd3c70960ddddc30d60cdfaa0e4fe72321c87079e9b25e84604e96487da945e60f362e5c65edbf7bb19f5994a35f2f384cfee8c568a90bb012656db0d0b2df8803cfbb92bb738d9120533b041cf2a32e36404e5e5b10b5d0bce90f4ab57310e65231a4bfc3094b4716e04bb9713cb68163bb917ed38dd01ae0cf719c3fe6eac7a0ca9d8acb5091a6ffb5fda00da01835dba7cc0efab5269645c8f7602a5c05961d085332f5f3a9dc762517d3cb03961499a714c913be317ba17fac21ef3da0bf0f210a846e2ce244f639e5b389a968d4c3451dc55261a7f33ff62f5f79d4c01950316ffa9e21b49d00f14ca87a06bbd7a80208ae4a1673ea8bbe368cac2040b2debf55cb23831d44870e28abe6f8c94e9848ef586d601a9eab80331ff8169d86f0f37ac6ae1d14cee679a9f8a882aac5efa071bdd210bbc38f02c258b001fe9c9d488ea3e9bdcf205142298c1ca94ed3706d1ddcff3ac0ec8b52469bd1ddac9e32b0010623c8954d4a1dbbb44eca149eb800b4d467e67586f0d411bf6f009e9b49ee2301cc8d693e4a1e21505377e7669c7ded48b4e26798c213f9c31648893da866ff4b7f6880d2c343d571c3057f011fc350d8ea837f1faf0359e78d9811e1bf281dc3cd34260f44a583950cac5714f8584f852a54ef418f052c8431f16eeefaef6522df3d25c5877e26d1a724858f9fec051bbdb381bda7ceb30ed5bf8f11ed997da37ab888759746187fdba71835f2d3b7ac6b9baf504f36fc6c910ec2549ca0ba089f6654a7d0f3ce9edde08055821ffd618d48000e7aafa55fbe38fd5328e6296cc7a25789f1b5465295965e9d1608f3e8474d386eead3446ad824d2ab129b6489e456d59020a4cb301d339594af793dabeca6d8209c3113afaf93f09b58180d016f9aa50a45a4a597cd7983f10cbff7650d908575bb536adcdb589fdd5fb69a09e6b743e676ed66cd1b97c2e7a07e51e1b618ec50d8c1415812cd4e17340020c801c0921c92210611cea72d66f9a06ecfb1a8e9e15f4cb6002b3359a36103c8f6859852b86804fd06469bfd1b1cb173a10e914d88cd3645e2895c7f203680ae6c7dcca7cddd1ec73a9faa352aaa1d8a73cc2311c29dcbc9b374bef4c3c5d1750364931000000000000004f8000000000a03b9d49e105cd093d5699af8c26cf290fa60991fc6e0a1f23c05c31c3ee4b1a8336238009e80b80019daa26bedcd5b93f3d102dc7f1904e97447be9fb60cb29b8489b0e50d3d476961a102343ccf574993b2a6969c913464519b69aa763b9d8c8ba403352d3c7d7080c68d53f1d3d089c61641e23053179f15ff7fe96d7bbb38991b1a2a6bd1ad053b194182eb79efdad8fdf3328eaed06f751c896e56f2daeac2700b823b7f3fbecd7914d1db2211a5372e229571ead19affad618fabb7f9d218a127736cd9beb8f9082dff25c3431ceb95b59611190ca30f9b771c19b6a3f1c750ee00cfa7d2d2d431a29b57637765b380b60d44eb8d5c5c87d53fe503114fe26f70440c7c2861b79bc2b002c4ccd99d6e82818391953cb4976fe7e99e8472b945dedb4ad705bb6f278405a726ab5a521af27a78ddc6f8456c657184e84474d8f654201bced3b6730f89881fc43fda4c5f50f836fa0261f9dc9c87dbb677018ae0a588064262d1b8df64de7b20614261dfefb5efd6e4d475016bd87b68c1b22797d617fc1eafff288345fc26a408e69bf7bbfb370f2f333a839cdd54da5bbb47fe28e2e4ef0812bc2e293ed527949813fe6b9368f277c259d5b065e67da9c81f23fde10e870b5f9e71fad26baf761bc5c5f72547fa6012513e01ff3cb0d553ade4c8d8a3c0ef36c106375d331b389a79d356ef638617e50e599cb544017b515851b76d00094dfa90551c9f5d3ab7251c3dbf1a8d235c198932ffd646689bf2f3f7be416c2bf03d608905160493423ace0743cc65885709094b5a04da75b79c7b2f34f3355bd41a27e33646e99a8550cd70717c0ee8ef1ccd07dbb322618fda7e2a7f26f14c85bc92def74747684b1faabfdc36fefb72f3be2ad5e999f39846a1e88166487ca1417401613d13b1f95b9f164c458c6ad0d4867128fe4b04ffcc6ab917f4e65ef73064f7368cf4b1c05f1970888ff36e694707a43c7d189d42bc97d0cec3b9174cc806518b2c09d8ca936ad81968917515fda9cb95383dc6f0af061934cb96577c38dcd1bfbdb1e1f14358e63c4492da9abc28b706c4efa843664fd9074454fd00b55187d1a32f7ecda988b00d2632bc68d1ab757d9c918d4c1676ebd84de41866cc4ff4e59add7ea347909d430297a47b664a3f391828730794a7c575bcd8a5164e07ce646e7c36313ad734870d78070fcd4351da25a1850d1036dc2d46081bbadb64afbb7b722b199000343ec74868aad53e27e42feb6d853127cb2e7b2284e7ff6a402fe1c0059e7ec0556422ce2e56925f48fdb00905936e07881e13bb14d39e5c7d7612a02775b19b24682e9665289b805ad11b4d798449fee047d9dd6bacb4567868b868c5812f31ab643dd350b6b9e7193836cde1a07eb1d18647b9b9db59edd88f875650e6ebea49e9db7a4c7579d8ab743988760a0d6a72848fff075e14c815d4bc2719fce421c886d13075edb02be09b6b2e6b88f916c074351c4ca41606e4adce0a8872e2b851e00c479a2d33ed772eb83a84b10c16953c06fc0597f2724831a2d177765cedf472213fcd239e935f0306895aede388c167062a43c6400932544a4fe9c27cfa7b1f774fd77b999c3193e4d9b5b840b68437106f401875c6595247a659308f17134e0377c60f08856db8fc2b63057b6bfac503dbc0bec85f7f76349b9d36e0e0e68dbfbd0e3d3f64ad8e0df42c55f8f05545d621e6673d60bd332e0358f761e4dc18af14fc61b139573ef58561c1870d130681188be8ecbdbbb592dcdffc74a1f3266b1bae4ee174897fb1b2fb2cb9e213ac66edc107337e75066da65f63044ff93c52334c2135407ffcfe3edfe088e81c2c53c78f1f08df1c17f0868e58287604f7a6d979caebd55f9883c5f4ac2346c4bf75c8f4b806b4fa4ffa9075cb4c93cea4225bf9efc7194e33817581297579c4162c0ba3967edb96e1eb3d55715cbc76744b91b619d797769e6a83e6ee8f639d4fd51a95550ec539e61188e14ee5e4d9ba5f7a61e2e8ba81b2498800000000000001c4000000000501dce99c9547fc94d05d826637ce13282e85553e947da39399fabc48ef3e96da883b30004f3d04000cebc96537d70ebe8773f6281ca3d2b3c95882a903b9edf29dc8cd9b4535016372b189186de39516217086ab4a0696472eb9eedbd0c8bee334b984a62a73df7fadbade0e9e3345452eab64661aea8a816f60167473beceecc6999ad42538dc47a1b33f32bc305a19e578a294512853a9644497d65cec270dfc823e9aea3a751d7d83685544097bbb2d292ef64b19f54f1655a62994bc86f3356a7cab052176047435e30e70839713a231f127d2fca6510ba9e13f43974656452a2340f4f0d64a02d1ab231decd6282a30608983d7c20629de2372a40b70687b03142c5cc65fedaf879de34fd69b6cf193a74d4cdbc2679b695434147b9602559c660ca29534c511dd196102bbc3459344872906dc5483e05032b4bb21c9227c50e0be2871f2b0da39ef3f008850068532bb3919e53ce67f990cb4282b76dc3aff2c33ba747e24ea8023d7dfab1fbba44c1c5f98c7c975732309e5e56733f77a8bcca88012410245ce46d80584c30fcc888a3f184ac5eaae3a5a7a86fce0b459f9c930df52b0de7be82b146339d41e12ebdea1188aa1051f968670ef63b0a2d81c8cfa8f6492adbdb6807ab2b55c5d7e2654e5055d3ae4e93607e4fc6be0a4189fd39b2109bf2d40404865bc0c04ba16486c50e50da9dd5041870dce712f76e3d71e8631cdcb838783683c4fa6c0d4d8e39ba919e2cd282de1cef085fe51abe550c6a2d190776de18e9f1fcb2e08dcc61a59d9927b352b95549d960e437122746b8e29931397cdfafad8b073c7f2ac2a276089a4236037d61016cd774a31268ac1e6107738df937dcd216159571d6855228c78bc0f32f80c91ef0d883f609ee0b8ff251cf89833139e18f64688195e14d5a5df42fb567fd6dabefaa6cf5acb098b2a306e9002870b4d39a0d896067b50ff375ab512c1d32cf6537567389b9b51ac3d3100eca38042d8d8130663504562c87a5165d0b4d63d01429ca3e30303f2f08a1191f8ccfc093e7d2cb9467e78c38070ef858e48b609b3a179d1ba11972386d18ea6fa5861042b6d46998f02d18ca5d53346ab05f7c3afd111b42debbc824ae9bc4d672e8f018ccfc6b084d53881c3cb172d8ce198c8d79999ed7e16a904126a33ef7476725ae2dcd3a09390ef11196b51f6cb42518fad1db7663747c4d35da708033eed7644740dfe4d88d348cdeec002f043913e49c5e40869de81aa2e8d78ac10dcc1e41c51c917562301cc2fa4c71e47e0b6b2680dce21a6139b9f61b28f45677166be17394ee7d4d10ec159e8b37005beb86467857c7f0e9d88bd35828a8a826dda516befd341bd670d7ade3c8e071cdaa2979f4908b86778270b30125244b7804bac821daa1f77f557f63541db3f5a330062b4cfcd020640605cee4e9d009de0679360853bd537c39987bfcd6cf13a1393835b0ac15745a26485d35f8bd5735263d7c3588a6caf63de01a74798530c42c6cf2f29da32c13970b985e84f5ac5d30b550814cb197277edb677a39abfb410cc83fec101f4735e0314d58fd43bf20fe40de7ef9b4b8e5b9cc6a0865ec754869f8d9b7492dce7dc348b538565458ce57d8ee337641945bc28107d7939b8c891c2ec31da5368e360aacd84396c68fa9b5fd28c369b5fa18f00a9dc0035b52a980272a6c588b4eb86d806593be05d3f1200c9238c95bd2c06acc5ee9b9e3aa195ae91fb97738a794f202c5f2107535928a91aabfa5fbc1d48a79b317b3b84bd1f43b343c0d07c2d89ae3bc96bbb738f3008bc4e5182481e2d764e9900c1f39b697138432917a692cf9102d8b91ba14c2d3ea86b73be0e4bb76c0e01a188417aee64288cac69f77122cf97159945f2db30ecc5cf3646d7f677ee9842734cee1af4159ed4f36bd060e998586abc70ab484708cb88022309da3f76abaea8f705dcb6fdd90af8423765f37747b1cea7ea8d4aaa87629cf308c470a772f26cdd2fbd30f1745d40d924c400000000000000ca000000000e9cf969e031b61563b6521a775dbfd1e64071d335c38540313d9cfa09ff4ecb29e740f200279e5600070c277cfc5b9c08175d4495a9eb45de062dd35aad58c2143f69b3737f8a60f3914f7616229837ccb66781a1b5b42fae130ddc6c087891e11a1eacee23dea9a3e5e597b9d4dcf8d7aa00ea167703bb25ecea5f99f1ee77a8fcd4a7ee7631a39247a29a2253c20b57dadc66884aeb53d1daa7ccd6aaa6461bdf706e32dfdcf255b82ed09a155c3c48db7647def0957602a600170cd30d0cd2e3a0fe13e76f12e606fe4992122ad6537d8f48f80632205496791e085dea2d34c2aeca5f9e8769c8383193ec9b8474a1a0a5ae278620914e9f3493b7c2daf908525188469acca56ab92c23c5764184b4deeffd2d8dae1706a8c4bf81d4ec8b9920abf008b51609c682353eac055821be4916a2221711c82e623c86cc0abdaf2760d3b0270946f37c776135250148609f78a1d045827d33502c97e3429c269d6b946150e12e701aa31a203686c3fc875ba74b5d6a33e6624825e0c82a24a2d0878c71627e287dc15d5a99a6e1db54a78dc63aa282857c3f79a95abdfb03a717b8b35f37c3de44b6285174c50c85c5b8641650229aa29f488804eb1f3a65818d34d27de1ae4e6e4418b084091e7765bb19cd069d1f375ed11d7675216ed7863acaeab09c7a626685cc789426f09b6cc646507db6b16845b14b8450313b527e9eba81f146549cb043c9cbf47e9b040a70a3292e7e8d2767d387edc25602582d5198d2264d5d32fa322bb246b9ffbb35ca261d1a7a56bc0d6175745da2452d105ac340cd1997008d1299130629400d328ff8d66d66f28bd78c282047629cfcec7b9c1ce933ca90a5a03c497b5e8e5327a07e2f1abadbab919fb61fcd8911e99a764e770e7927b411792313b2d6679dacb97548d1cdbcfc9cfda64738968aa5762a6d390666a540f8c4d7b070348b3abbc40b6590f03d3db2fb960ed1fc9a6dc88b1feed2ae1c69ca5a08db245d6112c2ba87d411bf0e12968444ed2251b49e75bf74ee1df282d780ee789b47af236914dfbf298072553ae1981ffc42ec7787604a2b06409e077ef69f42c1feed6ac385ae8bef745b15f9cd60b7f90c83fcad8704f4fb4777c0bbc6d935254f849f29723ec9a9bd8a0eb6736ba8095e89961eefcea17d3fb526558b241f8fb2dd0d7673e225d8a0953263270c5c382731784cef3ef0972a9fd95bf01d77641f41ea6aa0a937d5f5c099125cc3dec88a6a4f45c9b2ba9c4b4836221b68061eeedaecc48266cba75f8558e0a58c30bdf262746db236a8a87a0489a6d9de28ab4bd7e399f088551567209ecb370dba04a3616aded1a1a24f1ddc45fe1740ca7eb7af0ecee68a05bce5a3b966913436cf32db6b218f47f205b63129085f8b4489d5170d9f6e0954ba347614fc498066e31a912be70ae0979b0af14d0ab5fc81c7cfb86f2a713c191924397846c4b825bb733f53d753379d35fb2dbe6a429c42dedcf83dce51e2795c19d4bb42bb057167e31b990e259c36eeca8d15f9d592ec21578e60c6f6524ca04c8a1926bc95896612477f78e6c754fe61de69a9ab7d05cbee8cae792b3b370fac65255af1b9152609ade54614aaecb787b151e6c7f8df187acaef1e2bc1b821174e76b91d8ba3655b6a56c193073895e68024247a1e86d0584121c38ef6dcd93524c2fca937859e39a6083e06739b43836bad95c60ad4304c2a6a916681eafbbb5fde1a8242ce30b62b034cbf1b9decd0acec3cd920dfa1372dd915df20d3d32dceaaae52e4a56de738f1c9688d1189577ffc866996bd6d6bc47aa80c1dbabee70c9073bac0a40d15b4d05089568f6fcde543e045498ef1871e51ab0a218d7e08e70f7af95b8e1563bfd3c6f8f4a16a751cc0763c4e241d34d6fc45f394cac32cb92fe394cea50fcd3f83847fd8da2c14bc5b721c0cd6dec29b44cdbf3a74f48f62446ba17cf066c97cd44af9bba3d8e753f546a55543b14e7984623853b979366e97de9878ba2ea06c9262000000000000008f000000000140773aac94c1fda5f03ee19a2a007bd78c0df583aa7185f3243ca6f1b8c6b547ea2aa70013cfa90002f8399d938e50266fe7df1b4b7e5467f857c81661e5f732e3484ebc9340903d3922b124864c27af9d046f9281a0f093c773261f1c1bbaae4f98ea3035888b560267788a6eb7fe64880615c87bfeb9645a780c174083e2733ecea68407be42b8e7316289dbe327900fc0a9e042a3e7d736509da025117d7063fc703e0ff51392315e0ada33f8c9e0709d93bbfd62d90c49e26af6df04bd09d6ac9f6cf918def2dfd75ca952242a75a623470b47013cade1a6feea6b2bc41e0f8d2df485d4fa4f3575b5bcf4df4480b7f37b10333bd3ba876848a775e6139ff9d6a1fe3fecdd0a546fac12309bbab9e78c3613e2877c0008dafa24bbe125615373026ca436d2751424bf1339537d6753abf75a6d252f43dc804e28b01bd8b3854839f13b1926bd009ee55cf6c2cc06677de38b5c88c794c760f46f4de41ec69fe5dde53da18d3c5efd625b8155337e6b5eb797929a39892cf9b726e57bcb45d8c858d62bc641ce27b91bc87f407c35e44c89bf6cb5fc555d779c6ce996cc24292128d9a39e23749c26c018349607749dd8c11cb45bdd8f45e1539325f5b8f90c7a4275148d02fcc8b21c1ce15a2b09e0c02a1780dbd15da16f62b057aacc96c3efa65bff7df9ca6eb9685f3834100ab1eb9b855f82e659453d2b86e8029cd1deccc61d1c7ec9e5af306181b01dfa3017eca4a74a8f5483d5355c1ce4357333c9a90e31d908a5b230e0dbac20f5c39744b604e0204b5ae3f24734b5c1ded8a2aacc88d6e472b697a71a397d3f980f2d2d4b8e4c5d693ca0bf39051e84b07f379026422b4e2ff95eff19ee8d3864e54c0397176ba82b8cfca2eb5ed5bb6dc36c0c779ff3743f550919d6d4cd70049fb3bc2a1d56d5ed3d5d85afdf2b4485160776f92b676821913ae78e15621f98857a3131fcd373102f60473737bb2c8eac54c850c2be3a6ccc2dc3e86b397907cda675fa985004087a90dea83e82f087a3ce84f288502257ecc81fcb6a305a22eb66c8ddf50ffd060634a3dcc40806e24b818d8c190edeeb9c0ab80d25380fb54feea46a5cf6a6b8471953cde524a790a7e637280d39eea275eef072eafc50ac4fd38555b75bd03c36f033cc260fda0cd312d37e41f413751eea22206d99ca6813776bbde26228d4e060eb4eafbba8989bb51015d228ca057aace2887b24f9bea3eb1ed7ada29b592d6693a8bc1411e05a686f22157fa7651cc3fa56247572ba29291950d058171835e9ce70ea52e8fb1ba5feab1b16f72528c526df1d6be5a878f2ad6f3b5bd4717cd372369b1cbc7a6b04c72dc757069ce1ac96e79dad14a46287743d9c1bda077a6a28791d3720cb729d647fb7bbb0a6c5b0652ff189885b2fd82c94a26f4b24fca10a95b047051a686e78a348c6cc9578187cfd7c0898d1dbbe1dc8c82d988211355b3a1e16a2c624467245a267cda21a8a7ac67b84d2e95d0ccd28f8962c2ad7a740c0547c3b48303e4985419781f7a97821bd6a1a841f8f52b7e5a8a8db9b7edb3a871082bf91ea9df971101fa0ea212951c4c9f0c3c2ea9656a3645ee28a9f43e8bebceb5ccb4e5e453504ceb2d47d1a6ac115eff225da9c57b3b74802e238827f22d1f8bfe2ac8b48b861f8fc4b610f41d242f5803d533c12ecb80af376260ac2055fa82d7834a249840d866bd77a44afbf9decc0a865ff1fc9aed5cca494519252d338d0b29cd01a70be45451d3b34268d9414a733d15d75be276df8917bdf467f2f66ebb5a530fb7d621ca6748a79e8f6faf5f25abeb2ec358bd60352b78ea6a3d5af01dccb22f71c02944385f24b308a8d5d68743555ad5f9e6636ba187f3915a63cd1150448bd0e6d66c2ce603bcfad3f653f95ec887ab3046309a02f35b53d35ca235846a106b7373f3adc3893a4c9ab72b0f9360746b24607f1dd6fb5fbaa1fff707cddd1ec73a9faa352aaa1d8a73cc2311c29dcbc9b374bef4c3c5d1750364931000000000000004280000000017d7c5a3e84b3daabc85c03ac75aba13edd6befc42c0442a0386c441ee389b51d8a7cd18009e7cb00014aa6d45dbc8339df2a5b98a55a29bbf1fb46fd98f0da7be23ed5fd09ef91340f6950488fb2cc75c6824e4229d2a5505cc89449664a50ba954d595589c47cf7a1fefd383e357ff9a48c4fe448cb71ec803ce1acf4257825952b3bc320bc263d04acc0d1a44aa9de3ae9c5cab8028d1b5523c750f75aa4984881bccb74fd23579d5e1fa6305709cfb5bc79947ac4b381885ee03a4e0e2dcff74d20ae12090d37b04809ca4127d59d539ee99d42ccde110708f3a14a87c74786ba0266f243dd992ee3d5c2d32f2ebe6dbdaa4fcf815f6c3da11e3b4f5310b438a5f3f2bb46abc55e6f7de9395dde54dec65add8a4757bd910e4707143a9b44288b0929b43b289918d7777f2dfc75c1cb9f3d5f5a83b24a9f2dce109f4490b9ff9d41af87cc2701612838ee432d01c328437d2afe4445d0c37fb3c24e2e4ec30c6034b4891eb83d633b1ffe0eca36f814f3409cfba5d816f9cf80b6322a716f8844125feeadd504d41e7aa2c5c036a50556c7f7d3c4aab14800686c69546fff1585364ee37eae53b0f86aad4cfe2e3d5dcf8c908147e89840d808c9df5d8e73638b4eee8cd3858899a2984dcbea58a770b041d7bf3f0bccba77d06555b6d6b402e1d0f8185fa80c71f78b408abe05cb5ca0703cb1e2e7e42f9a2ecda5a86a8b1a6e6a591a124be303d024f552b7ec082555cff8da1af3937a20039c973216503f203a267a324d891311f8861226c00ccb37c814dca02fa111fa32984f56e57fbae411e9c5e0043f8edf38e26557aaf5ef956ba30175bec1b8a457952289be39f42e4eb42f91b01801006dd8c082bf3ed1372fa58be8df9f3ab5ffb5581255abe1d927f7da384c0704626fd62967fc05b859978a0ceb01d28b7cad29ebc13054736d9d093aac6b7ecce0e7a7d509a91ec9336aa87e9f59621fa046a4d9801d4830e0c58f8072e6985f70fa10674d23534dec52721bf20bc71a1b1c588beef66d6db3b5ee5f5d715da390f2aed612e4f1c8c11ac2c290357206dd49ffa3859b919476d0294ba9a5f96fd75e1467bbc18499f6d30091d8a2cb3eca3cf106468faaec52ec43ce20f026493c9ea0737a07c9f5307050b8b49288eb541129ac7e1b34bb3b2835bb5620fe35226d9ee4f3975f7a49aa3740d1ebfbb4e3ac0942be2a755273955413d80084a2318039e5fdd9ae5b1a98d7da1220a3420d120faeb38f8ea4342ad7d2127f622da86e81161894e1de8daade70528ef4c4ea1fb1d4e63797317f7baac5f61e1fb954bcfdd91bbc572a2c1f3459f4c92fc063058f7df6443cd67f7516b0612eb50e0f3bfa586f19a6c1051df335c53a90a33bdcde16b0b670b80bce75ad698d1cc9850fa3de8ced61210aa31fbda66d54320b323ba5aa4edb04bba74df01749eec4e37dfc01460a176e1d69b6a1ce30f3d9061aeb721139506f15fbf264fd17ab4c9696c2aeb0c310940b07ab5fa779ca787d13071b61ce048f88378054f10de36a80a1d8d3c46646cd23a80f8583ecb3436ff5c80b31a506a332b2eb285daf12e31d81e7ff26ddd314465412ae14af2020b078f2f2070a2531f02d61a70126ba072c8fa5df98bdc975bc5db3190a5ab066ecbf0913ce8393318b6c500e10512d3ac223022ef76e0ddab200799b8fb7b3363d20f05e3f21e3dc5ced218dd52eb591c622d5b8f7c9ef49bce5694d33d49d8d4404d59e217dbeaefe3441d5ec5da1a0f0420ff8ac245dee596acd1fcf9827e1c886dad48afe2c69fa8de66ec7e2add084b428ea0cae2adc8792d7298364d4d623231cf5d93c717c3910dfd347ece0a307ea4f8845c64f818772fe1e0f73c264b663403b7e883a8ea1d04d7aca0ca968038eafcf604b34f31955c6c7371a41897b26eda4d44dbfce0a419cf304cb09a4abd220a2eac63a145a453f53dffe12db1fe212c93e6ee8f639d4fd51a95550ec539e61188e14ee5e4d9ba5f7a61e2e8ba81b2498800000000000001d4000000000501dce99c9547fc94d05d826637ce13282e85553e947da39399fabc48ef3e96da883b30004f3d04000da562f17e6c2940b8bc99f425ee2e5abb326d5b3e034028555a2de4e6a4bfda70d13b3e5c9938f62b9278e620067f8cf30c4ec8b6bdeb13d1beb035ab863692450e2dfa6304c693c04d20a3b7d0fe173c3e4d627697fbdaaffc31b496a685cf6f18ad519c6c0c0092fc0a95473c0bf584970883365fab48cf93205ddfb108d6941d4c6d9c8c99b12d49edd1f76a6121e72b20190ef6dd9def162e977aab38b30604b6519ac9ff2a84a8c36abf514e1c955919601fe74afc0e33d76e1038fc91b7faf491f89a19c51e320cadd892ead42016080f5823a42207eae61db3d8fcfa9bcf0e53f589f3a39fe1bed96512cc82cef105f4a6c525dc4abc44c70e5c5792ef44d662998f0c3dfe0b1b6ec01b68b606900dd046b749197a14511d589b2bd8e36e0717dc70bc3c644378d3383fee9e1e5e4ae878b7afe0e7e9d2e199add8d10fed39094c257125a873aa04e3373851fdc6550736d13b613fff6ec5ca39d556a2523277740f4476e0034d1fc541f7dffa1ea68eff7dd58980af4dc5241fde9e0dc90c326a71aebb8fc5b30e15e13cfe3280712e78c51df0ea183be0e1592c4238329cbfca827283aea794a36968df93f32ee2447f9a4f214c4b867e04434f381f91941bd898ed8f55ed5b3b9d0d1a4fb3bcb34edfbdc1afb47ad0aab7624a530b04b93c5d9f00b57aaa3f46b1e48f9d42d11408d7e760873202b92e8471890fc0d665642bca315bc5a5fd3a332f9eebdd1da01768a76ec6056f54f6aa1fa05d249c06cfe451e16704f495326a5471728244a9bd5ef191c488f27797fd1f1f45df7146ca837f5d5e88426c957eed2f30f27dca9532433fbb5ea85d82bf980253ea7c979165d301dd1f472b33cfc89b9cc04b39d2b07b9d2036b333dd8970319f7188031911e958b1428583b2ea673aa60201f069ee6b196c998f3c8ae390727521a20dbfdd7e4c2d65f9bddd65f7f94f02125aed7a4127ba470f1c24ce433396ab5ebc06bb7bdde3d18c5c6864dae65df66e99a2d0b45147f40293bac6f0b20f99fdc7165e50d4b39ca03b9d305c42446271ca40ce4b55b4221ab0c09e870fb97745cfe8981e5930f0be8f929204b3b6cb7bde277a96ed51acad025c65a9d5f4d015d83b420e64595c44ee819e0e6acf28548ee1d32a04f9d300201708f1e9bf4b3aeb71dc023a042ecb8aa8852150001abea90316163d17dec974f6cdae32792bae26b1112178a3f84bac0e351e47805475a7f09b00d82e033293f68c0d3f6f5fe1533ecfbcf8c308794cfb25642626ba20a3febee3d14dabebfc801af4cf75fdff5b37adae3cee9ec63fb899751f68bd0f82f00e668b9cc410e91cf7476945599aedd2458b6e63f108a4462b53654e0fb2c88636000094c395001d9ab3fb4eebc1d11f6e712639b2e8ed1c4a784ed51ac6cd47ed94f2ae780c4f25366033ee97ff3fe563e224a618abd3e5c9d253a2a0c753326433aa7d91ccf20089b46976aaae4126c4afea92090814713615514dc464d682a25f4229db0f6dbdedc317084c70ceb5fa42fbbe7023c757b7c1a84b21ed37bdf43de253282fefce885add655eecd16cebd222d7f5e45ef4f20e65f84fa2c530aa06a737b2f98b90e6fcf85f0026ad30eb87c74213a49fe53727d8b977aa27303fdec8d0eaae4b8d3b85bea9ee30c9a794cf5c160283243180b91c7c2f821aaa5602bc91aecdd60c91dcadb847790291c727261359366f1d15ec5f255dc699f24dd981bdb44962cbae4425d2953479a8c873d27e507221062505620a48f3ee62b153fead2d4d81d9d49689b9596e5d738ec1ab06bd7551b06808f53dada566bc1dd9e1e65d258815c58af6eca5a77dc34dd396cec00c5779680a0ea1cee7f3379eae8e1dcc4107f48566d61d019318e063d9cdcb861a90428b256239f42a9022b4000003e800000001dd5390d8000000071826c2500093e6ee8f639d4fd51a95550ec539e61188e14ee5e4d9ba5f7a61e2e8ba81b249880000000000ae025a600000000000880083fff8ff99c311947ec9f00142b1827d3472e83790651d05ed77f12a1d7dac2ca8011d48840ba78e54cfd499eef8caa77880ef8e22eaa573a461e00fd3922c57b32561d5bc64840ff453844a4265b27da62f053d203631e50e0b0a55a8789244d4a37f885ca342414ab80e2408000000000407e6ee8f639d4fd51a95550ec539e61188e14ee5e4d9ba5f7a61e2e8ba81b2498800000000018da46a003d686c000000000000880081ee1c8089830727560587a354391f4261ea097e6516caba008d24983962c4d581087c000000000000880082978461e882a9c2ab757c6d17c7c1ed7617125074e8e0f646bd84b3fa92d961a029480000000000008800816a70c0ae4f4af18e997a69e8906ddb2b29c17dd2f59de99e2a251dee5c5a901429480000000000008800826e4d0bc61d076a174c69af89f28603da3a4236d22f8895e096c7696154fd38542948000000000000880082852ea2f13172061dced950f5b9f02b4b7b96c8dbe22530d1fd279646a619424c2948000000000000880082eaf06696437c28bea3e909764cf36893abd7123c5c4fd5e7a6a4eef87e2b3a582948000000000000880083d37df7fc0accd5ec0fbb9a159ab09f23134431dd6b734116fe126d1f212c33302948000000000000880083d37df7fc0accd5ec0fbb9a159ab09f23134431dd6b734116fe126d1f212c33302948000000000000880083d37df7fc0accd5ec0fbb9a159ab09f23134431dd6b734116fe126d1f212c33302948000000000000880083d37df7fc0accd5ec0fbb9a159ab09f23134431dd6b734116fe126d1f212c3330498400000000000088008323c7c6a1929f34e71fc4dbe367af1dda088d33c066bd3477e08af58a3ccb14094b0c000000000000880081cf1ecfa03b4ee4bb58429583e38e8a6aa8662acc12fb56b91e6283babc88eb7f6378040000000000880083db3a286396f5775ade4947e7924cbc2fb425239acccf75213e074918a69b701212307800000000008800830cdee24d4350a68c49c9cfc4b7f8e99442b9d167d87c158052fe10274b157f53bce1d000000000005800522542accc2438ab0ab0cd489c257c3e7487eb570c100120c114088402eeab0f66f5ddb8bfab9ce497f019bc4f5194fe7064a6e7b67e01a173d05f67ac0880925ba807b3d9ed91d47196148b3cb114e9e24690c924e8208784884b5d9b2064051cc11008800792d3fb7b4aefeb7fdb7dd2488d8a6eacdd86a5b5cb828258adb3f536a9c7f80881adc5113e7cbfb41e5a6697aa343fa71f0c411b47d0cfddcc6c60ae139b5f2aa0051d48840ba78e54cfd499eef8caa77880ef8e22eaa573a461e00fd3922c57b32561d5bc64840ff453844a4265b27da62f053d203631e50e0b0a55a8789244d4a37f885ca342414ab875c1fc800034000c00921bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c0000000000ad686c000000000000880081ee1c8089830727560587a354391f4261ea097e6516caba008d24983962c4d5800215daa4535b119a8bdf23c80a985438c60e901f0177e3157a1d8eb19c840f586852f0d080203f185e7c724fc8459f93e1625847c3cb749d59256e316987f5f20804821d91d549f0840a3ae6e11593fe15fe5cefd785cfe00e9fa037a4434ef146a71192e8657b1ad89d4ab99ea453fb7e4ecf0df30f551c97e19f303bd7b09c6d763e22b1a1a0017808000000061bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c00000000000000000006d4680000000000008800830cdee24d4350a68c49c9cfc4b7f8e99442b9d167d87c158052fe10274b157f530b3c4c00011cc1100880412bb6354bf94388c4ea6915981e0e28336b1d7f70debf960710579e8836a5100880d346b1a7193cd15c4946fa331ad4045bc9914e04c99fee6be8f8c629e7d3cc90040120c1140884025ff36902c8a45ad7586eb7b7d84999630ad649f5400b30b0f97e229e2e97d16c088129d2f3e209f62532ac54ff0147ce7638abad98058a9a815e5fdc50161e984ccc04000c00921bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c0400000000ad087c000000000000880082978461e882a9c2ab757c6d17c7c1ed7617125074e8e0f646bd84b3fa92d961a00215daa4535b119a8bdf23c80a985438c60e901f0177e3157a1d8eb19c840f586852f0d080203f185e7c724fc8459f93e1625847c3cb749d59256e316987f5f20804821d91d549f0840a3ae6e11593fe15fe5cefd785cfe00e9fa037a4434ef146a71192e8657b1ad89d4ab99ea45137249d4e1ce371b604c416f4cbce9fbd08ff60c622b1a1a0017808000000061bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c0400000000000000000674780000000000008800830cdee24d4350a68c49c9cfc4b7f8e99442b9d167d87c158052fe10274b157f530b3c4c000120c11408840278395e21ae07f50567d9c9ca22475e6480ec8f0d47a4c4d2ae9116ca2110f4d40881eddfea9089c49adf421c5131d7d39722be76dc16cde01b6f4b4fd0ce599b4c88040120c114088402a900fe88fa08de685ae85ad11b6f7b9693e1e6e9a42b4fd3c43711177ca0cae8088002947f6f6da772f554609d0c0ff3b39835c672db564fa20bf6c064094eeeaaac04000c00921bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c0800000000ac29480000000000008800816a70c0ae4f4af18e997a69e8906ddb2b29c17dd2f59de99e2a251dee5c5a90140215daa4535b119a8bdf23c80a985438c60e901f0177e3157a1d8eb19c840f586852f0d080203f185e7c724fc8459f93e1625847c3cb749d59256e316987f5f20804821d91d549f0840a3ae6e11593fe15fe5cefd785cfe00e9fa037a4434ef146a71192e8657b1ad89d4ab99ea45074905f59500c73ed3ba65f12293dff7f8f2d28ce22b1a1a0017808000000061bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c0800000000000000000595440000000000008800830cdee24d4350a68c49c9cfc4b7f8e99442b9d167d87c158052fe10274b157f505f404c000120c114088403ed1399bf0a5367b38b0cd194e60887b73f615c8234ca8db7b6ae641b402cd5b408802317f3e9ed07e185b55fb4026a1852b298dbb6b851e6d845380d01ee9bf3ba9c040120c114088402a6b5976b6979540225c5cef298404d684c02225739117ef24351035501af585408819135d79729dd3bdad567febc420ea697689430f21d99b65b6bcd6d203bc3195c04000c00921bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c0c00000000ac29480000000000008800826e4d0bc61d076a174c69af89f28603da3a4236d22f8895e096c7696154fd38540215daa4535b119a8bdf23c80a985438c60e901f0177e3157a1d8eb19c840f586852f0d080203f185e7c724fc8459f93e1625847c3cb749d59256e316987f5f20804821d91d549f0840a3ae6e11593fe15fe5cefd785cfe00e9fa037a4434ef146a71192e8657b1ad89d4ab99ea4513462c2d43a4c8de8ad2dbf11712f04bb42408c2222b1a1a0017808000000061bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c0c00000000000000000595440000000000008800830cdee24d4350a68c49c9cfc4b7f8e99442b9d167d87c158052fe10274b157f505f404c00011cc11008802793ce595b721c4a18efff8ab37e9348917d97ce288c3740d2c89b70b97f5f38088127004239130e674686d9eadbd8159bd24761a0901ab1280ea453e688465b5ae804011cc11008818e9a0e700c7f554d35e01ea3ffe3283e69cb62aa75e3b84135bb38ad05eb1738088064f66d4869100474c74aee16f9410c46710a9ebb8f67bda2dce230f270e9e01404000c00921bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c1000000000ac2948000000000000880082852ea2f13172061dced950f5b9f02b4b7b96c8dbe22530d1fd279646a619424c0215daa4535b119a8bdf23c80a985438c60e901f0177e3157a1d8eb19c840f586852f0d080203f185e7c724fc8459f93e1625847c3cb749d59256e316987f5f20804821d91d549f0840a3ae6e11593fe15fe5cefd785cfe00e9fa037a4434ef146a71192e8657b1ad89d4ab99ea45373587d2ac3f83e8a316be72672e1238db7263b7a22b1a1a0017808000000061bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c1000000000000000000595440000000000008800830cdee24d4350a68c49c9cfc4b7f8e99442b9d167d87c158052fe10274b157f51633c4c00011cc1100881fabd0d6522e78132d9547e5fb97616b21e78834feb8fd72682203bd14b4cab240881ba717242630f8efbc0434c5095e208f1fed0ed0ff82800db5207c62805a2c83c04011cc11008817769132762abcde9bd7db265c4f4a8005a0bc964ea58332bd708c167348005ac08802646c26f337f21e46214994292fe9ed2b240a0404d898c6d7b0dd909c2104e3804000c00921bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c1400000000ac2948000000000000880082eaf06696437c28bea3e909764cf36893abd7123c5c4fd5e7a6a4eef87e2b3a580215daa4535b119a8bdf23c80a985438c60e901f0177e3157a1d8eb19c840f586852f0d080203f185e7c724fc8459f93e1625847c3cb749d59256e316987f5f20804821d91d549f0840a3ae6e11593fe15fe5cefd785cfe00e9fa037a4434ef146a71192e8657b1ad89d4ab99ea4501d28627bdbc7f65d297c1e1148515b611a7a77c222b1a1a0017808000000061bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c1400000000000000000595440000000000008800830cdee24d4350a68c49c9cfc4b7f8e99442b9d167d87c158052fe10274b157f52a73c4c00011cc110088008cb6ee751b6af195a7cfd97d3ca92df96b3defa15a1ee625c507a5e132e74dc088021ecb077691135cd465d84d6d5bcbcb288b6300f56a902b2f3a3109076bd5054040120c114088403b76a93dd9ef1c3f59cafba60ec59136ffe51e77bc3bf1f329e7c36a86eb4dc340881320408e25a2ba7a1ae674d21193a6b1e81b90089408ab9080eb8dbeb64a3089004000c00921bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c1800000000ac2948000000000000880083d37df7fc0accd5ec0fbb9a159ab09f23134431dd6b734116fe126d1f212c33300215daa4535b119a8bdf23c80a985438c60e901f0177e3157a1d8eb19c840f586852f0d080203f185e7c724fc8459f93e1625847c3cb749d59256e316987f5f20804821d91d549f0840a3ae6e11593fe15fe5cefd785cfe00e9fa037a4434ef146a71192e8657b1ad89d4ab99ea4504cea013557d5d8d7281736267b01538c13146b0622b1a1a0017808000000061bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c1800000000000000000595440000000000008800830cdee24d4350a68c49c9cfc4b7f8e99442b9d167d87c158052fe10274b157f51073c4c000120c1140884036a98124d36d50aa96a6f2f523ad99beb22b37ba4b33635088a3d48ddf14461280881a7e1551dec9f3b0e6c57ee7999b026db1218322e03344c574926365f673c091804011cc1100880839b50f7b80fd515d360917b7cb8c4b42924b748d42dfe37da0e42e21db79590088155751b35517fc3b30a039af20ba5dbfd252ead7cac4a201d899d4ffee795e5b804000c00921bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c1c00000000ac2948000000000000880083d37df7fc0accd5ec0fbb9a159ab09f23134431dd6b734116fe126d1f212c33300215daa4535b119a8bdf23c80a985438c60e901f0177e3157a1d8eb19c840f586852f0d080203f185e7c724fc8459f93e1625847c3cb749d59256e316987f5f20804821d91d549f0840a3ae6e11593fe15fe5cefd785cfe00e9fa037a4434ef146a71192e8657b1ad89d4ab99ea4504cea013557d5d8d7281736267b01538c13146b0622b1a1a0017808000000061bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c1c00000000000000000595440000000000008800830cdee24d4350a68c49c9cfc4b7f8e99442b9d167d87c158052fe10274b157f51073c4c00011cc11008811b0f3e23e7a18e725531611c9520b75b9900f7b68e0930deba576e6fe674d058088190cbddb1728f58654455706c88ff4d775089fe0f87fe6a4d0ded8d45559f48fc040120c11408840204a8d829b771105e7e0396d094af29d2ed76f8768948c327a2bf34e228bf591c0880b916996048fc098f28455fbe6a6657b78589747b2e94ca55c617c760112a874c04000c00921bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c2000000000ac2948000000000000880083d37df7fc0accd5ec0fbb9a159ab09f23134431dd6b734116fe126d1f212c33300215daa4535b119a8bdf23c80a985438c60e901f0177e3157a1d8eb19c840f586852f0d080203f185e7c724fc8459f93e1625847c3cb749d59256e316987f5f20804821d91d549f0840a3ae6e11593fe15fe5cefd785cfe00e9fa037a4434ef146a71192e8657b1ad89d4ab99ea4504cea013557d5d8d7281736267b01538c13146b0622b1a1a0017808000000061bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c2000000000000000000595440000000000008800830cdee24d4350a68c49c9cfc4b7f8e99442b9d167d87c158052fe10274b157f51073c4c00011cc110088120b7c063946d3da109c5a0af0f20712074a235840029df7c849abf810b0d1c7408811dc9c83fc7a8c8e89c496408a81162b9348a397d84eb798ff28e3ad7510dd5a804011cc1100881da8ce34896e25510486633964daaa990cf687d47766f2f9a7656878258305a600881f6c99d45ff1cf8d63827d3224c4eb85fb5357db610cbebb27b079e128a9f69f004000c00921bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c2400000000ac2948000000000000880083d37df7fc0accd5ec0fbb9a159ab09f23134431dd6b734116fe126d1f212c33300215daa4535b119a8bdf23c80a985438c60e901f0177e3157a1d8eb19c840f586852f0d080203f185e7c724fc8459f93e1625847c3cb749d59256e316987f5f20804821d91d549f0840a3ae6e11593fe15fe5cefd785cfe00e9fa037a4434ef146a71192e8657b1ad89d4ab99ea4504cea013557d5d8d7281736267b01538c13146b0622b1a1a0017808000000061bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c2400000000000000000595440000000000008800830cdee24d4350a68c49c9cfc4b7f8e99442b9d167d87c158052fe10274b157f51073c4c000120c114088402458956d5d6359b351bd94fe2969c6c7215efeb869f3e14656d75e11db104e314088004078fb261d2a89911a4598afe2516860abc48deed871167a76ffd256e425f4004011cc11008807426691929da5ac8aa3962d9714ba6946681a97127a5105575397ef4534fd6e40880f024703b16e102a02f9ce74a6d7f0b1d72e524a397aa0c4cfe13489bca3352dc04000c00921bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c2800000000ac498400000000000088008323c7c6a1929f34e71fc4dbe367af1dda088d33c066bd3477e08af58a3ccb14080215daa4535b119a8bdf23c80a985438c60e901f0177e3157a1d8eb19c840f586852f0d080203f185e7c724fc8459f93e1625847c3cb749d59256e316987f5f20804821d91d549f0840a3ae6e11593fe15fe5cefd785cfe00e9fa037a4434ef146a71192e8657b1ad89d4ab99ea45149d6e9c52d8cff9fd703807f156df476ac33046222b1a1a0017808000000061bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c28000000000000000005b5800000000000008800830cdee24d4350a68c49c9cfc4b7f8e99442b9d167d87c158052fe10274b157f505f404c000120c11408840363aa90dbebb1ace3c55419624fc484b390d857efb126d2bdcbc58465dd2d51c80880cea8c0e64bd2a33f871df9a834aea9ceeab40c16a23685b49848b94da19d0c4404011cc1100880ebdedfa85b30063f6d053d4b34e3b55237095f2ec1b204b27707581402c852140881cb35d2df926ff604f826eb9de36a40ed12dec2b88ae1ebd30d33e2eddf7f7bc804000c00921bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c2c00000000ad4b0c000000000000880081cf1ecfa03b4ee4bb58429583e38e8a6aa8662acc12fb56b91e6283babc88eb7c0215daa4535b119a8bdf23c80a985438c60e901f0177e3157a1d8eb19c840f586852f0d080203f185e7c724fc8459f93e1625847c3cb749d59256e316987f5f20804821d91d549f0840a3ae6e11593fe15fe5cefd785cfe00e9fa037a4434ef146a71192e8657b1ad89d4ab99ea450306a74e9564c77db956b93679a833577c0e0ac2622b1a1a0017808000000061bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c2c000000000000000006b7080000000000008800830cdee24d4350a68c49c9cfc4b7f8e99442b9d167d87c158052fe10274b157f525b3c4c00011cc11008813d77410afed127148804a8b518796a5327a8ef98a702ac246c387d55a7ac4670088085047fe2bfeaeac4b34e7c3aa993c6c23e9cab69bcdf4751b1af79120f61a7ec04011cc11008810d5cfb6cdefa05746e5ee4200fd8077f2470949e7336981ab59457fb9c87f95c0880c2b0a20d022881d691ccdd10fad4e418aa2b1ad514efbecae618f5a5f8d7760804000c00921bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c3000000000af6378040000000000880083db3a286396f5775ade4947e7924cbc2fb425239acccf75213e074918a69b70100215daa4535b119a8bdf23c80a985438c60e901f0177e3157a1d8eb19c840f586852f0d080203f185e7c724fc8459f93e1625847c3cb749d59256e316987f5f20804821d91d549f0840a3ae6e11593fe15fe5cefd785cfe00e9fa037a4434ef146a71192e8657b1ad89d4ab99ea451b717a849e11edcc1f9da13d7a7982697b42cee0222b1a1a0017808000000061bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c30000000000000000004cf780400000000008800830cdee24d4350a68c49c9cfc4b7f8e99442b9d167d87c158052fe10274b157f50af3c4c00011cc1100880c3fcf5350bbc70f5d085928c3fdec084857a207649de94a85c81452911cca25408812c243dad04ba32ab2d6f2fca56eaec1d767552438fdd9e30a24819e03dbc2f24040120c1140884024ff5f7e8fbdb37d6e533eb7e0c08afa6018b54f35c25e9ce51d14ec8429207ac088188b98e886de1fa9eed57c06adbf940ff2e3bd32d96b739e8992b805100d91bf80400000000000007140047f37747b1cea7ea8d4aaa87629cf308c470a772f26cdd2fbd30f1745d40d924c4000000000000011c000000000004a380886e98c357fb5b9e58229b17db9f8ce78a9c0786521a9e5fcf1eb31e19dc37ae00279f4c0007d73c1565e89baa7fb0231fbcdc5cdce7fdbfa7750a2ebcda6041aae5cd32ed424b13b55fb33bf167aaf53358ad333b1afd78b6f37c9ce5d6d68187cc07a6bac42dd855b63acd2f51886673d8ad9a208c4a25237fca0d92c6b5ca6ba6dc2430da2be3c044ba49d33f185a376713ae6e68efdcf7f5026873ef44e7a044602eafcb5954bb6152b6974f97e9be58b77382b9469054acb0cc8fa8ee401cf0fc4b08f08035f1b1d3b3c2ab8134b830feb0a8fbbeab1756b445754461e7c2d0708645e947d09881fc2b7cd0d2b8b7a400f6e950999fd21380942c2b69801bd3abfa36b2f4f5c8af638cb3f610ca898745cc4e37797d0af82355c83d2d632c69a31eac82c1bddc1610e173fb6d03d6aa6d7d825be991035deb0e7eb7a12df77d7008ecd2b83464d0ca7f531ff4decd42e7f9fce217fa37afff7d6edf9b8b38fcf1843e88a21aaa8df49c5c83c1a66d8c3ab3bf945d4bb4d2a1d439affa8b8841b481287b9edce44cf206ade65f4521e5a82b05810fcfb120eac0831a4b65dc1fa7397afc9368e0b62247e98a07a731f0941e48335bf599f8606769c1a360f91747019dccb77cf7760cdc6c7a4bec07559d6af26c5433f822d75a04597858330bf0fff79ebe94c520020ad87d1bc5693a060527232fba57d6106f99d2af26865dbf3f39b826fac2230a6f2bf5e2b076e9aa93c8c3c19681e98c4234ef44df79a117a4ce94b909051e71531a99eaa1d115b4f187c1c5437561e3d872903d93934bbc9e368180d2e3fdb87d684c3b5f5e2c011b0e9e2afaa1cbfca573ae124b9416488e63116e1b6cc396272d0eb07c970de8a6278cc6d9bd9061b8d73c26fa09b1d4c6010010db2c965188b03e36e1951221aa852645b30dccf9a4e22c8bf41fadbddb13fdfd8cfafd9301ee6bea28b614c8d9fd69c146456510ac9edaa70c94ffb5ae9fd1e56547087f1ce962ad655c78926118d3127f86df812f41ea54707f354c9cd5680b805ec10be2cd9faffbd55be3ebcc3805ac77474c47b738855390a453c600dbf6642ec7a487ba88d86ae9970eabb3558805c1974d466e0a1f5aa3567ebe501228fe0689dfb2f76c7fa3bf07a97c9833091a9631144942b25b7e13d7d4b185d9f17d78986e5fd3a1812ad5490246f49bbc6e85a01b08794b9b9e4f8f95db4cb1e27a3cbfda276491785d7289e547f1c5b51b80f36a65c9fc21c7c401d38884fd0a0f45a67954c981e2dc392241946c0eec3025ef2f86738906f9a857611dbfece93135df8c00093910d69a0347593d3152310e297a8468581b097a17d7d8ad70796ab99dfcd4520d3b7b1c23332cd8e201fa7896a33e56bffe88a5d2b76ece8742c135c825d2d65ec91ee4a49e265a3135837541daf55e1e3af7bfbec620daeb3066fea9dd098e9badbb675fa78b83f3662756ef568c32889bf689c16feb60f63180d2722b9891394f5a4ef5444ff0efa230bf8e920266ee60bcb6849285af819f4763d2bd542ac8ca6facb469e3cde5f90a331854f4d209fd7ab1f4f2c363006d019e41910424820503480a3921710364bd444b0eed0f9a13a556239b568c07597ed5d3033729b1534416d7f53eca6e4f6f1df39ea725440b87c428a360402cfc218100decb75c488d77de977c7cbe1bb3adfa66819bcd27b930d508503150ff95a94fbe356d750d06ed7d7b9b785abe7f92e2d632b588d39b764c070c36c69c58caaf106165a0221c66af2c20ef9a11957afa7c66bb63354fee2e008e2e382fb8ee34bc0ccc85778a98482f3104c77a05c15236cfc1f1771b940951a5fbab2ea9152d268cc6ad06605e03787448a1691218a55825842a6bbf0dcfc1761bf6cf66331492a870102cfcffff04ff00e4a08592b3b0997385579dcde4399405af414ea1018b39dff8cdaedf123a69fc16664334123f9bba3d8e753f546a55543b14e7984623853b979366e97de9878ba2ea06c9262000000000000009200000000007a19e0b13b1f490cde2990c6d41d4605978af7fd41e091619d668955d49de55bc267dc0013cfc20002b85ee7d3b4817d8944e79810fca7e1f783a5f4d8b10058423c5c2f798b5bdd4607c085fdf8385921ba9c3b29c0de8314382660ee48efa208f4aef129dfb448f0a55620f3edddcad9ff3a6193715a4bfb3d9d5632262de12ab44b1528233bbb1cf226467b1a7c628642ee998020986196660764ea0da1ad8b169a69ad5694cdddea817de990acd7a1141e677fe1ecd5dbb0bfa7b77adfacc7d84c22786baf9269adddc221f3f280fe1d9056c52798656ab3dad47d756109f7042dd58052c96b14e8faf55134bf4d70ede0f6c995052ad614c632eb33e33c4a91ef9d9e4dde418d94abb00189601117ac59dc41c97a364e73f75650cb55730ee48566ca0e18d055e56ec9bed3c036bff80b886d04a53670bc1d753a27921a03a7f1b6369b7251fe1f9a0a1d6786264b971eb87bef1805955bb658219846e9e66ff8fcdd5a873934631a96c311881fc65fc80e3deadcec01cac59b4ba8c79510632418bad169f558b130e1888e38b7c131790b3120f13ffb69524d9dcf228b1ac8e60540034d62ddd38691ba09a7f2d8338f11857eac1a3111a0f6ef9c1084142daafb3ce1b722fee3b6e1763d77648755a371ac2f56a0a1ca76ca7ffd6e4a5e4d202a1fca36d78fc3490b47a25d372a423478e9bafff1dda80d88500353a7c652702d6a4cad02de64120754a206d383930f6d164b3af9b680775897d39cc49743f9067737c217ef5974450166391f7f67b83e91c61205b78f0da7278f9c34d7aeb2217bec7244dac595d69529acdeaf6a0b04a26c19a1612bc26935f54f8266a0cef930ccd7c89e202ac4cf614ca6cafc11f31aff0ef93791c4d635a4836e52ca57cacbe1b34f13454dbada065e404a06a2210a0da9f06e8294077ca75159a4f0de67580387808b31f84dce7bccf1f42972929a1a028f4d91bc6565a54c56d53d5f6d9874c86234ad8000f16402fe2d3ac5cb210d3681ddace4cd97a37fcde992977b353cc83aa9ce18099b7e681c0251f100146594312e37c808b3bd58c5f057774fd375468f3db2c9ebca8d1612bbcf90125a500e8d2e12f92386f0eb454aeccc85436a2f7ec23a893029286a5abb8d91b9b843414fef8134edafd08594dd35b26ccc17270cf30f451dc0a90b2c9c3eba85f9cb326608089ceb5fdb2e04ed610617f17b6dd2a08da5dbe7bec0a431a4547c88702e428a6c2e7625d93cd9ab3a0c9a03d75499ac13be519709f94eceb94cdf97d0a3e8aa633a29de78d809a337e0aaf3f81c231fa8d88333ddcfbdaeb2e3a85a67a2cb16302a2293f33b422f3b8b28679805052a4a4425696987d4da4d15592230fe829381d820425c21e08ae3b6d124839a8d669d382f7cb6ea09a405998ab5eee86be1642c99e014e498cd99688ac473f9cdd4476827b5d1b0e3b29f6d4e4de9b2795283f18ca90e53e71f5ef0802c07d126309c77fc0faa6f8fe7b627c23c241df8c5ec4b632a220aa9a9b00f0336524568546b7aa65b46e8c471c7f90911397d1e546a2553c8f3f2ef8a0d6e8d912b2e37dfa9b86e15ccb9e0cddfb6d4ace2ce686a13e5fce2f2959c595bbe189aad01ea2a3e8fe2c993594bb3730a93a7d9ad6569045617ae8676bc7f858a40a9f90e97997393c56adbc8860a8efa57e32109e0b550918c58483cc5d55c5247145ddd22a93f684e3a0b26ac468a5cb56132467aa2e606b4c232ede142ebdc5a1ad97cc417172aec13ecea83cf37542b4c18897120f95efaf194df28e9ee0c31986ce4125b65051e91aee8d62f07a653bb0d82e9be6c60421e356952324ed99cd80f0eadf922684f8b1742eab47372750913c7dc82be79c35b242f7face416a3dd59c232166dd2d9769be98b3c30a574443b9391ac2e3e971430af4330b97464ac55dcf26a9d10314a9bb8aa750c8ed239a0ab04890c6dd242fcddd1ec73a9faa352aaa1d8a73cc2311c29dcbc9b374bef4c3c5d175036493100000000000000368000000000a03b9d3392a8ff929a0bb04cc6f9c26505d0aaa7d28fb472733f57891de7d2db5107660009e7a08001e62e3a7e6cf2a4c8b317075419fc4bf29964300d7bb692f4a2c6e9b82adc474cbe805448343091fd14d4ffc9d80e0863b6904f6ca14a8ec7183b05fa123282196a96155b83b3f2c3063c334679458e11b53a0fb80de6b96777f2ca9ad5142063f98b576e2b3663ec1b4783e0ebaf60e374416c2b17f84759554caac23881e2d4d07e92c2e752fbc5c73ad44be7c20466aeaddc8843292abc8019608527fa1ce56b8fb922e57dc5c749d4960689427e6ae3c2f7f8c955107fd2766b40383cc8711158458cc0ce5bed16ea6946411298a14638bf1d833f7e503988c1322bd423612ade1599f0f25f54601f66876d528468bbd7f6431f5caf95fe842e43fdfaa0183cfff2bd6d59bbb68d8386c2763b53d7b1239a9acbfe25885b7c86784d2b3f467fbcf33ac8023330513344986aa97ba3fab0c20354d96b89b7f53ff3e1e178da75a033c42931498941549d13086245f2605f6594de9f002016c18bf0ca3002a8d91a867ab25ef864c268ad6c13a64df4e6b573fee9c9b3505006e7e82e06b3e586b1a2af313221af370b2caeef27816089d3faa72319bef865696252571bbe6da5472c5d6c7924b31f277d0e8fa9d1110f8635c1989da88a03627b50bf7133c4a6ebfd435a2c50f358d3dafb3ed71620d4eca966ca6f4ebe41c25d291fa4d5dfc97b8d6d35c922fa824d651ae3d9725423dddb843a4b39ae7487d5edb1aedade6315ef0d886c8cef501f20c808236f9e97feeddd2fb5695972e95fd6b26c61e33f1c410bfa47ff53d0aa75a8ceec686a47be1a1f9c789489d14c1e6c95df37c4b51acf3ac646d8d787303839c5af8751bcddf14aea001e444ffc5eb4cedc64f82fdb366da6d5c3cd95f672a141b7e0930393324e2ff11dd26edfd6deeefc4df52ad805227e1fbdd979b4cfdc327ca42f710f749fd0236cc28dd2dd926ebd73db1deed6a833f1b5c0bb76cd3c54ddcee5d42ce2a559cbe16e42260c758976d0bb77c99f6bd3e8d9a320f6967266e6382947bfc937bd50ef848a1e8f472f4403d5d71f6ae1dc812b9c9a5568a5aa5ad62eabbab41ad0cc40817ea4da06fef8891591beb168edcb0da7cb9283bd3f1f0e0f09e0ce01f48b86c8ca0e78c13305737b799c6fd73efd429060eff534015043d5b1cc307f1d022dbd925c483a7c2a450e5ef07a2d42990b550da3dddb6881d8a227a101d7812b143f017682edeeb362bb4ed3e30df5f6a57deaa2c1bba64250aa30c35ab999473988bd43477729340c80733c848b35a1703aab7d6169004b93cb9b8f659da825e9ec373ca1b859a1c57a415e8154bf4f957d6b5c7844f83f3ad54e6ff1f79ff63eb23bc868d36e9b6c036fd1368398d0016d258b36e5ca11f6353c5eb9902756978720d4c48c83fee4ec449373e7c28eed5f9257df6281b551f3af1cc4c2719372f15a20bd63581642f3c48b818785e4967e0edd78b69a04594ff937310a004cd7da4d7788ac91d6f1742e4119758a48dabf4d9bb2ee62789faa31ca958cc59b4ad129864d7197f6271741cc2e9bef86ea45680863b35bfeee61dee850151254c253e0cb0b477be8364a47cd06a8d29beefa21491d3780a5ef05480ee808f8fbfb78a5cf4f3332f417a8ef66076e5d30fe036800ab07a8ce263d9df5a6cf735bc9cbfad02dd3775c28061824b567081bec0963f8ccfdabe533b51e1b3a37bdda1a534833bba68e9b77802dd84d0c8f29ec1dfb7865bbca981f2724eec3fc82d39e93836e695b753584a20ec3612f10d494de5a571f234b9f1b918e2b191062067e8e1c6e409471ceddac0e4851b997360d35d881f1881d9f5f7b6a5a365d4abbdd5b429552c0dc5a97c180d797a3ffe2e56f975d35a316f4cf4f4d2f52b3ff56ab4fd335c8a613cfd5cd3e35be5cb8ae3944abf087a8aa2dd2277e3e3be7fe6ee8f639d4fd51a95550ec539e61188e14ee5e4d9ba5f7a61e2e8ba81b2498800000000000002800000000005ecba02455a3cf02904c3765cfe4be179fbb8336ac1b8fec70b818b246e2b53670c69f8004f405c000c8d2d2ad9a600af6e29631e816e075e9b72af9193a157a586d6ed101d42df2a803537e9a1799f0f0093ee56c1dcb73c5f05576b4d2f4ba33095ea5bbbfa54434c70e273f136458d673b6fe5929381a987a56df55ad64a99e6d0f925433fb888025f0241f2fbdcabb5fc71bf52e4c3d0b3abeac923120e20a77a2b5955881f06b8738ccfd30d9f73948e13131d3ece306fc656ab16232bb228217c863f10f602bc72276a09ae544900ae793da99a9cf2d51776799f7fc3778ccfc83745e02e4574dded93b7315ca2469e830e1705a3dfe729dc05a6d8e821dec62f977177d85f57197302e7750cec87e6f06223fba71c743177071d08f7b17ccb6e71fd45573770eda1d7bacb6104c2d071a3e9a033d14660e7d5c2d8ad33af65007ca6042666ed1e2552e8e9210da9288526331d5f872346dc1e666a3894252e39695ea90edda4c938535e6e1309f8d1c7b2f064af23ade1fbe5d196ee49298b549ad04f9e0a1173563408e0e8195d8a90b7dc6d9bc7c862f28fa932a6f3fa49321b81127dcb7a1e74c6e44f0c9e4534238b5a1685eeae9abc4ff586778655435b76efe3eefbb9b6ec62f958d8e584a0f8c4e41ef3488cc3424a05973de4138a1dfd766c3e80a185053c0308ef6b8edeb6ea02c8c0c2c2bce80301c21f2e722db3c6301d2b9ad103c41b82d18bd4be2606fdb58d6e6af99da9415e07b52e94d90626632f708103c77f7e9da78aed8349fe5eefebce05a630e5cb8c8fbef36458dd753917f38442102367c068b9e3888dfd3966ce98825dbaca593b10bee572ca62ae99c7e98ed27d53a0b6264ea6f225f273a596042ed783bd3c53e33c3904508440468f2c1c0ed5540c57a9600c27099dd7b6aea8da17ea7e733e3d89e81f78a1ae5a589bacdb3db6f1bfd3308314fde0a40030679a2c1150ea38bf49d6d8dfc585c4f493b571c41c8071a7b1be11fc1bb3006c828b1de658b21f73657b62442a01639b95fd6c2f547782e8d577767af413c11c6e114b33dcba353a7a520b13f3ff631810939bdefa062c19356d84a4bbc98cf5fa044167fbf4b36e0f1ff9130991f8a961586baab1456c8fd5f1eba09ddc74e09f9d8c62c9d0e082e9351080dfdfb2f8432b5d282dd5dfcfa20d08994ae2822cd7c8e344de0e404f752d6d23a1daf24495797d7461f5bb3dc22455f3a823cd885c3dc2eadb3f8e5a8ea1748394242963099df59e92ab5539dff296639ae9ab4425bd235fde5bd4ff4ffe032bb454ef8edca95c3ae7be48506b19c07e4827fd7cb10ff22a77893c8bea46d0d5dfe79627aa91874ea78fc4ed27d0312c4704651cce45caf8f425bb78699e32d0e77c50aaa4b59bab05d3dc2a3b151b2d1c1a1d7dc2801fed3ff7f36706e6a0e6dfd41052a5aac676399c89f14a3afeb78485ae7077f3c36f6cff5a3cc634131e54366ccac1a6d2716101032528869986b55ef43275ed17f2cd3f18f4ff979acacdf6c2f553a2778323005ab5498bd808775a083c32c1d57466512e97becb0fae91c4bbe5fe8b8c3f2b6d00bf03b8a2450ba5890353fbeb0297a69b7e6168d84713b70d1ee919d1c6d352eda17b491a88436fa27ac348893244cd1b848781374799e58fe629e8be968d011559908815e1c95d53179e7a7eebea9df451bd9ede21bf26399d00ff94d7b4c23df3a8dd39c738910dc98f829060c8d9fff6e96de940eecae8b6917ff1374b41fca384e76a1c9f780f0906e07f1fc322e3b1dc500e96369529598aeea2aa877c3f0b6d9045af70caaa098e044055b5e8295ea4de52bbf1464685dd184b796a8b9c414c279ce40ecbd2b4edd41f861ec754e9906f833d9a700c79e18badf9c794b8fb8aa5df76c3f1d4ccb9ed0f3dbc0c3d5a8f1e3fc4e5019251da69fe68f08f5521196f77203746cca3f91738dcfbd6b7f37747b1cea7ea8d4aaa87629cf308c470a772f26cdd2fbd30f1745d40d924c400000000000000f8000000000280ee7554214ee355be40cdee741beda0e52a0761590530e40ef9a254caada8b1aebf4a00279eb00007ac8e07b91a3bb689223665b1e432c94349ec74f0532dc1d293aab36b831c1b7469976338fe3bf95bddd379f1f7b55641cda6901e5bba4a77d9981e7888326855ec04fbeb3c6e200c9f4dd839d7724c33f5f3c10f7568668723b312fcb12fdff9347288022df903858bccd750eb161249278160e0ce2497e6f08704271ed12206789e4230da07df74bd5f120a2f382d1c156bd74ae3ce093f5b4c242fee0947dccf993106ebae46b47fb4bb6397abd4c05cbdb171fd63f4b46235a0a2f7a88dc9fad442814a27c590693e2c2c511c5149f891cd13f62ed8bfa5f58994bacff938242ee982d7750058d46421c12f69aad82074705a8402701f5f27c688b4ff7fe36c6c2365b540153d19c91eff298b35defb6d2cdb815ed06d3b16d9256ee1d0e1cdeee72808b62b5501f88234ba24d6b5026049f7ef5c239648599942d1d631ba649645d22af4e25da184e7d95c6043391ca35020d06b56f694a4aa06c390bdac4f22dfe1d741385b8482b5d9503c3e5519f53cb679f3b10d01fef0a73c24ebb5d6f0c0bdbbc9ef5105e7e562c2005996307dfd2896b786cef930122ab2a0b08565bede6a2c9c9bf4a10a548b2e500ac4b63d2390a82a818ca723975bfd3933369367140a76dd0aa0ca31dea7ae2898bdde3bf90ddb6f0d1c7e4bb77ce485cddcf13389c846ac34767dda8ff44b67ea72aecebae155fa4c7778b540df713e5683ff2febd9d4295f6197c0af6849dab47a14ab93a07cc1d59dc330cf4a4634b578c046d3b7e7fdcb86d02527cddfa0b82b4c043a67c96d1bb6d6b13fc2ed900da1e2c21880399c15c0475a9edfd31fa78e12c1bbbb861ac19bf541c9fce464390e0f3d364bd08c09d2c90fb528bcc1e6c5cb8cbdb7ef7633eb32946be5e7099fdd18ad152176024cadb61a165bf10079f772576e71b2240a6760839e5465c6c809cbcb6216ba179d21e956ae621cca463497f8612968e2dc09772e2796d02c77722fda71ba035c19ee3387fcdd58f41953b1596a1234dff6bfb401b40306bb74f981df56a4d2c8b91eec054b80b2c3a10a665ebe753b8ec4a2fa796072c29334e2992277c62f742ff5843de7b417e1e421508dc59c489ec73cb671352d1a9868a3b8aa4c34fe67fec5ebef3a98032a062dff53c43693a01e29950f40d77af5004115c942ce7d5177c6d1958408165bd7eab9647063a890e1c5157cdf1929d3091deb0dac0353d5700663ff02d3b0de1e6f58d5c3a299dccf353f15105558bdf40e37ba42177871e0584b16003fd393a911d47d37b9e40a284531839529da6e0da3bb9fe7581d916a48d37a3bb593c6560020c47912a9a943b77689d94293d700169a8cfcceb0de1a8237ede013d3693dc4603991ad27c943c42a0a6efcecd38fbda9169c4cf318427f3862c91127b50cdfe96fed101a58687aae3860afe023f86a1b1d506fe3f5e06b3cf1b3023c37e503b879a684c1e894b072a1958ae29f0b09f0a54a9de831e0a590863e2ddddf5deca45be7a4b8b0efc4da34e490b1f3fd80a377b67037b4f9d661dab7f1e23db32fb46f57110eb2e8c30ffb74e306be5a76f58d7375ea09e6df8d9221d84a9394174113ecca94fa1e79d3dbbc100ab043ffac31a90001cf55f4abf7c71faa651cc52d98f44af13e36a8ca52b2cbd3a2c11e7d08e9a70ddd5a688d5b049a5562536c913c8adab20414996603a672b295ef27b57d94db0413862275f5f27e136b0301a02df354a148b494b2f9af307e2197feeca1b210aeb76a6d5b9b6b13fbabf6d3413cd6e87cceddacd9a372f85cf40fca3c36c31d8a1b18282b0259a9c2e680041900381243924420c239d4e5acdf340dd9f6351d3c2be996c005666b346c20791ed0b30a570d009fa0c8d37fa363962e7421d229b119a6c8bc512b8fe406d015cd8fb995f9bba3d8e753f546a55543b14e7984623853b979366e97de9878ba2ea06c9262000000000000008f000000000140773aac94c1fda5f03ee19a2a007bd78c0df583aa7185f3243ca6f1b8c6b547ea2aa70013cfa90002f8399d938e50266fe7df1b4b7e5467f857c81661e5f732e3484ebc9340903d3922b124864c27af9d046f9281a0f093c773261f1c1bbaae4f98ea3035888b560267788a6eb7fe64880615c87bfeb9645a780c174083e2733ecea68407be42b8e7316289dbe327900fc0a9e042a3e7d736509da025117d7063fc703e0ff51392315e0ada33f8c9e0709d93bbfd62d90c49e26af6df04bd09d6ac9f6cf918def2dfd75ca952242a75a623470b47013cade1a6feea6b2bc41e0f8d2df485d4fa4f3575b5bcf4df4480b7f37b10333bd3ba876848a775e6139ff9d6a1fe3fecdd0a546fac12309bbab9e78c3613e2877c0008dafa24bbe125615373026ca436d2751424bf1339537d6753abf75a6d252f43dc804e28b01bd8b3854839f13b1926bd009ee55cf6c2cc06677de38b5c88c794c760f46f4de41ec69fe5dde53da18d3c5efd625b8155337e6b5eb797929a39892cf9b726e57bcb45d8c858d62bc641ce27b91bc87f407c35e44c89bf6cb5fc555d779c6ce996cc24292128d9a39e23749c26c018349607749dd8c11cb45bdd8f45e1539325f5b8f90c7a4275148d02fcc8b21c1ce15a2b09e0c02a1780dbd15da16f62b057aacc96c3efa65bff7df9ca6eb9685f3834100ab1eb9b855f82e659453d2b86e8029cd1deccc61d1c7ec9e5af306181b01dfa3017eca4a74a8f5483d5355c1ce4357333c9a90e31d908a5b230e0dbac20f5c39744b604e0204b5ae3f24734b5c1ded8a2aacc88d6e472b697a71a397d3f980f2d2d4b8e4c5d693ca0bf39051e84b07f379026422b4e2ff95eff19ee8d3864e54c0397176ba82b8cfca2eb5ed5bb6dc36c0c779ff3743f550919d6d4cd70049fb3bc2a1d56d5ed3d5d85afdf2b4485160776f92b676821913ae78e15621f98857a3131fcd373102f60473737bb2c8eac54c850c2be3a6ccc2dc3e86b397907cda675fa985004087a90dea83e82f087a3ce84f288502257ecc81fcb6a305a22eb66c8ddf50ffd060634a3dcc40806e24b818d8c190edeeb9c0ab80d25380fb54feea46a5cf6a6b8471953cde524a790a7e637280d39eea275eef072eafc50ac4fd38555b75bd03c36f033cc260fda0cd312d37e41f413751eea22206d99ca6813776bbde26228d4e060eb4eafbba8989bb51015d228ca057aace2887b24f9bea3eb1ed7ada29b592d6693a8bc1411e05a686f22157fa7651cc3fa56247572ba29291950d058171835e9ce70ea52e8fb1ba5feab1b16f72528c526df1d6be5a878f2ad6f3b5bd4717cd372369b1cbc7a6b04c72dc757069ce1ac96e79dad14a46287743d9c1bda077a6a28791d3720cb729d647fb7bbb0a6c5b0652ff189885b2fd82c94a26f4b24fca10a95b047051a686e78a348c6cc9578187cfd7c0898d1dbbe1dc8c82d988211355b3a1e16a2c624467245a267cda21a8a7ac67b84d2e95d0ccd28f8962c2ad7a740c0547c3b48303e4985419781f7a97821bd6a1a841f8f52b7e5a8a8db9b7edb3a871082bf91ea9df971101fa0ea212951c4c9f0c3c2ea9656a3645ee28a9f43e8bebceb5ccb4e5e453504ceb2d47d1a6ac115eff225da9c57b3b74802e238827f22d1f8bfe2ac8b48b861f8fc4b610f41d242f5803d533c12ecb80af376260ac2055fa82d7834a249840d866bd77a44afbf9decc0a865ff1fc9aed5cca494519252d338d0b29cd01a70be45451d3b34268d9414a733d15d75be276df8917bdf467f2f66ebb5a530fb7d621ca6748a79e8f6faf5f25abeb2ec358bd60352b78ea6a3d5af01dccb22f71c02944385f24b308a8d5d68743555ad5f9e6636ba187f3915a63cd1150448bd0e6d66c2ce603bcfad3f653f95ec887ab3046309a02f35b53d35ca235846a106b7373f3adc3893a4c9ab72b0f9360746b24607f1dd6fb5fbaa1fff70fcddd1ec73a9faa352aaa1d8a73cc2311c29dcbc9b374bef4c3c5d1750364931000000000000004e80000000000128e05cabbc5be4b181147785247dce11ad61bff19a4cc2ad7c5e22755f253a6504af0009e80a80011d4b5285516db4e39e7131580124901b65c95e0090aec06dcc20229e888c26e9aad1c6876af7a78922094cb29ebcc154bbd521c0c1c7812dd9185df2390f440b8fd82c780fc48a909a501567570d5e4b958aa0db2d85c9fc42ed23c87a0bc7919984d5937406622eb6ff4babc9bffc907e93d9cefe2c94302495a47e2ccc234708c00de72491bc9efd09d60b57b40674983ab93a8ccdecfc4f8646388d25798e220bba7ff6e2683132afeec472f01d6e13404ab91ef22f047b0dacd59af726bd0de3ff9af6c33e20068757d8691542ff1882380e29f15fd71ed3b4fb1eb24e36613bc911186a7f3ac5fee442a18351aad187dd8f6300ac05022227206cc428fa8e5f38411006a5c26a0e1d9cffd0f0eed6423e774569904ac82a6efc415f11ccb73a6469e062524dd07ec6fdb8ca3e5054b19beb2c4b8baf9216fa1d297c2e8bd0b478c783df95eb3af32265c00a766ea53d13c3e60e27e7e9c4376d47345cd01dd6c3d557d3abad9a36c5ea4b2a037bac055b2934dd12f66302bc80f6dff8547824beaabc718fa548b378ef10479a41b8768f2b8fa61f6066437880a7d72e2f84980d0cb64c3d9a2329e2d8819ac3262c97f14a68bad43e5e2a63871e17db3cbb3d81a4af460cd14e8e40a5da66257e81b2741f73524676dd6a3f1bbf051d52cf844489469af914588229acdaab61b95290c9cf636ef3778f81e0fe0b19fbb71acd94cf3d41c10a04ab3b7ac52703e28008fadc9ece2a3a2fda1daf77c79e6437fe45f0da9c74d68d274c88b978180412b013d1f4b7b0d8f49ebd991fe32d4c68e2947a9cb722515d561c36faeec5192f52ca4d41935ed9832f06b91b080363a02ccb8d694da4f76985d0266f2f92c3a53cdf4f7bca67ea01a1f178a2e49e5ce9b9e597f1e47067d1d5ed5f0875a167a38422d5c3556f824d21d02682921f7a0b75f3ba74ad65b9033facee5301b0cdb48a19cffd82de99276a368f0e254a09148ad0217cb2f3bd265fc015c5cf71154091fb38aa0d156da6e923b327d5e49275f1f40649c7266e7d0f23ac1fc04f0e22160ba687651929ef9d69f143ab9f67d31896442a1ddd15918109b3f7805d579a6b27cde9a8689856ce2d37c43b25c0ea24c77f5ccf134625251d6cc83c01571e09ba6a126a3701c76da2753b31a1b2d70fbea404b99a13148af5bf84022db78cdc5fb2606218a74d587d9be974416cb6c50e8f93c264ea00dc6b9c98529a6b8c9546e4b4fda821510171bb35249f4fed7f9775d649ed8841d6bd2536f42835aabe3defce956e6d25d0648c1c453eacc41d868843c9a1ccbe2c5f90d0da4c1a0c1ec3fcaca0955eb33bdba8234ddec00d55d2e2bbc4e0e34d335b8259ff83351b6e1de3ed13453817f33b9b5ed0ca54eb2a88beb45f5c0134fda0c23a0d4579c6a8baa2470e3175dfad67dda76e82d167e252487f4e137fd484f4afac57a004fa8be43eccb496a2d659a5e68a8b7ea8318e15596da803eedfd2750774b2e1fcc6ea2da42a0ab01146137e4bdcf47dd8bd3549ddf0af3c0223bc5e959fc1f1646466bc2700b4e45beee8fdac74d212f7298a6d1c8b83400fa854288da5c64f6b166149a581fb9ef7680e1356a344669ca58f99c81731fe8a2dc70cec8403b2f5f28726e9480947cb4946a5d125cd31cde02c2078654d540d9f5c608979c41a1b8ae88ad69238a08c10cf39816caf53a9471a8e515a48de830502b45071cb4d420b288bea82fd94251041327384b2d2de8111fbbd8f9f1d582ffe5d14bc8ad1a81953126822b9fe133c2464a1bc53d4ab252029a950b353eb188f909eb3e3b00b5538c21a050187f5842d905483856382db3b6430229c28c32f62f2461f8458f023cab743a95c0cb85a236969f681ca6ade93d6b87869fb9a8e2b887dade14b0a84e09efcfe6ee8f639d4fd51a95550ec539e61188e14ee5e4d9ba5f7a61e2e8ba81b2498800000000000001d4000000000501dce99c9547fc94d05d826637ce13282e85553e947da39399fabc48ef3e96da883b30004f3d04000da562f17e6c2940b8bc99f425ee2e5abb326d5b3e034028555a2de4e6a4bfda70d13b3e5c9938f62b9278e620067f8cf30c4ec8b6bdeb13d1beb035ab863692450e2dfa6304c693c04d20a3b7d0fe173c3e4d627697fbdaaffc31b496a685cf6f18ad519c6c0c0092fc0a95473c0bf584970883365fab48cf93205ddfb108d6941d4c6d9c8c99b12d49edd1f76a6121e72b20190ef6dd9def162e977aab38b30604b6519ac9ff2a84a8c36abf514e1c955919601fe74afc0e33d76e1038fc91b7faf491f89a19c51e320cadd892ead42016080f5823a42207eae61db3d8fcfa9bcf0e53f589f3a39fe1bed96512cc82cef105f4a6c525dc4abc44c70e5c5792ef44d662998f0c3dfe0b1b6ec01b68b606900dd046b749197a14511d589b2bd8e36e0717dc70bc3c644378d3383fee9e1e5e4ae878b7afe0e7e9d2e199add8d10fed39094c257125a873aa04e3373851fdc6550736d13b613fff6ec5ca39d556a2523277740f4476e0034d1fc541f7dffa1ea68eff7dd58980af4dc5241fde9e0dc90c326a71aebb8fc5b30e15e13cfe3280712e78c51df0ea183be0e1592c4238329cbfca827283aea794a36968df93f32ee2447f9a4f214c4b867e04434f381f91941bd898ed8f55ed5b3b9d0d1a4fb3bcb34edfbdc1afb47ad0aab7624a530b04b93c5d9f00b57aaa3f46b1e48f9d42d11408d7e760873202b92e8471890fc0d665642bca315bc5a5fd3a332f9eebdd1da01768a76ec6056f54f6aa1fa05d249c06cfe451e16704f495326a5471728244a9bd5ef191c488f27797fd1f1f45df7146ca837f5d5e88426c957eed2f30f27dca9532433fbb5ea85d82bf980253ea7c979165d301dd1f472b33cfc89b9cc04b39d2b07b9d2036b333dd8970319f7188031911e958b1428583b2ea673aa60201f069ee6b196c998f3c8ae390727521a20dbfdd7e4c2d65f9bddd65f7f94f02125aed7a4127ba470f1c24ce433396ab5ebc06bb7bdde3d18c5c6864dae65df66e99a2d0b45147f40293bac6f0b20f99fdc7165e50d4b39ca03b9d305c42446271ca40ce4b55b4221ab0c09e870fb97745cfe8981e5930f0be8f929204b3b6cb7bde277a96ed51acad025c65a9d5f4d015d83b420e64595c44ee819e0e6acf28548ee1d32a04f9d300201708f1e9bf4b3aeb71dc023a042ecb8aa8852150001abea90316163d17dec974f6cdae32792bae26b1112178a3f84bac0e351e47805475a7f09b00d82e033293f68c0d3f6f5fe1533ecfbcf8c308794cfb25642626ba20a3febee3d14dabebfc801af4cf75fdff5b37adae3cee9ec63fb899751f68bd0f82f00e668b9cc410e91cf7476945599aedd2458b6e63f108a4462b53654e0fb2c88636000094c395001d9ab3fb4eebc1d11f6e712639b2e8ed1c4a784ed51ac6cd47ed94f2ae780c4f25366033ee97ff3fe563e224a618abd3e5c9d253a2a0c753326433aa7d91ccf20089b46976aaae4126c4afea92090814713615514dc464d682a25f4229db0f6dbdedc317084c70ceb5fa42fbbe7023c757b7c1a84b21ed37bdf43de253282fefce885add655eecd16cebd222d7f5e45ef4f20e65f84fa2c530aa06a737b2f98b90e6fcf85f0026ad30eb87c74213a49fe53727d8b977aa27303fdec8d0eaae4b8d3b85bea9ee30c9a794cf5c160283243180b91c7c2f821aaa5602bc91aecdd60c91dcadb847790291c727261359366f1d15ec5f255dc699f24dd981bdb44962cbae4425d2953479a8c873d27e507221062505620a48f3ee62b153fead2d4d81d9d49689b9596e5d738ec1ab06bd7551b06808f53dada566bc1dd9e1e65d258815c58af6eca5a77dc34dd396cec00c5779680a0ea1cee7f3379eae8e1dcc4107f48566d61d019318e063d9cdcb861a90428b256239f42a9022b7f37747b1cea7ea8d4aaa87629cf308c470a772f26cdd2fbd30f1745d40d924c400000000000000d2000000000280ee74ce4aa3fe4a682ec1331be709941742aa9f4a3ed1c9ccfd5e24779f4b6d441d9800279e8200053a226d9bc4a723efc8a3cdb05899ac12dc38ee80878c32af2a74e8a1fefeaa03175f4c0f2c13996b3498c11d137dbbdfc3ad5d6b1462866e48405383760b7f7dbd6b1bf79077d113eb112e7049c5ca94a101ee31de426e1583d7abd5dd1ced104259863ca1c77ddccacccc4c162b36f75485684f915da1e3bcf64771a3e036389062c12844c097b1001aa4e8dc88e4f6e301f583e8c6467a8e08bb7be1513f51d330b58afbe15917f0c69adf8e8b46f5830f2b7e1c28a2d52eb1d01cff116b30c6f899302b1805b81dc4bc49de05ca66019a8e8d3512e2af80eacdea1fed95243ebf696668b4af2d6972cdb450a1a2d53add4b1845857a9165aa53e538205c208295242f1b3cdb756881635a38d0fc22b73295feeef4f5b30c4be5d1328ae35c15a7aae6a6bcce93be28715be6a6270d37b184b8b44004396e2ef615e227f1b4ca07135acd75db37212edffdee5cd377998b26f2d4ec0ea1f52e1881f13abd7d7fa0ea9d3b57425686c7cede9e3b43c5557299b84d2b874b69685428c27ef531c753f2563650b6f494a9df5ea162fb59f46a7fda1a800ac7823c61549b6e06089655d11ac214db1d772c5523c9b4c2df9d9ec8b5b3b3285b62e14ec1c7d7c494d5422d9f1d418a9e29e6c502e97a9ad630d2bc33376c66f20114a0d653fac982b2da9e70a18ac808616fe601386c02f398e9dd641b33fcdcd74c234fcd7274bbad73680af1927ffb8da31149ff0e69eac45f161569a511ec29bc7496982c0c48aad406ba4ec2d4460bf130f6349aba22945c48c3b8881081c810389a95a183df19f90f6f529da186f18ef93f7ee1adac1bf95be23f3331695923c01c0e91f076340557993f3bba8aef14d811fb014d907f8dadb96f748ac8113b2b3d6e29f073b7a49e9d5557d45c1ea009cd89b6b198476d7dccaac06ca9d72ce4cda75442fd0788473df9c6fe78f67fb444bfe2e497315522fb2f0ad6652a2cbabdfcf2a3b363e6d3d3db9f95662eb3ac641e2f28e7c61c1cf5830c81c48c91417cf11d6af0c5fcfd0ef4fc2cce4270a525e36ea4cb14e99014701fc4c61a47aed5bd7aea1a0a76662de679c5d6d9422b7260b5f44a152a0e1290cc49ef82d041c110c0defa866cc6931529a3e3508c257b8be854789b5ae743a23ada6ee971199a8e2da360229d731fe3fb1a8a996796daa5cabc80c2b29f201fb8478f56a8a00a446d472cb8e86bd7e855809bb5c463271c9caea0fd422ddafa9028ceb452f563fed3c7bfde0d02c91e4d10c848ee85ed9fa39258fe97fc197abadab5db46b2875d35f11c7ebc5605fe37bd579d19b3c4912222e5e88168e5b1b3f817788846293f5cf8e38a76f4b0024a81e8fb5a0496d5fea7c7c6311db840c955dc2b9cbdddf044d6b3a23b7904915451f173b84b359d801c483c90c712b67aa4d4ddc65646f664c8573a51fc26caccadaa2eeda61c0425120a331690f75e30196b8f3f6cf181b2dd792ceecd7a150f84768eb9169c758133f2c37c3ebf0f91b334747d6478755a281f59e076bde14f22b4b0f9082bacaa455b8c2853b407dd9fec7470e1079e7eab764680cf93d8c3f763b8db573cc8db3e9f83250b9f193dcd09e04ae04f3179fe22cb21b8cb409847d97bbdcf96d1917c06b588fc757aa70c8a993734de942ebc97cc398fbbc1240956d0f494e0c714613b3357aaf3b4bd70667524ad062d194be5577ed8f2cc2361540fd22626798bb460c38a8e06731a81f81b8ec79963255fca1d39f34de095e0554aa25484d1d740dd59c2dcef8504a1a64f36b6aa6b6a4db843c58578acf11143891b866f2e960d6926864106f16452257c9ec636ec6298cb4eb46949051d74dcba6acb4de284cc8967e40ceebd217794744a243d3dc34b4dfc301b20825efc1d202b02d96970f9ad9f852101f9bba3d8e753f546a55543b14e7984623853b979366e97de9878ba2ea06c9262000000000000006500000000074e7cb4f018db0ab1db290d3baedfe8f32038e99ae1c2a0189ece7d04ffa76594f3a0790013cf2b00038613be7e2dce040baea24ad4f5a2ef0316e9ad56ac610a1fb4d9b9bfc53079c8a7bb0b114c1be65b33c0d0dada17d70986ee36043c48f08d0f567711ef54d1f2f2cbdcea6e7c6bd500750b3b81dd92f6752fccf8f73bd47e6a53f73b18d1c923d14d1129e105abed6e33442575a9e8ed53e66b5553230defb837196fee792adc17684d0aae1e246dbb23ef784abb0153000b866986866971d07f09f3b78973037f24c909156b29bec7a47c0319102a4b3c8f042ef5169a6157652fcf43b4e41c18c9f64dc23a50d052d713c31048a74f9a49dbe16d7c842928c4234d6652b55c9611e2bb20c25a6f77fe96c6d70b8354625fc0ea7645cc9055f8045a8b04e3411a9f5602ac10df248b51110b88e417311e4366055ed793b069d81384a379be3bb09a9280a4304fbc50e822c13e99a8164bf1a14e134eb5ca30a87097380d518d101b4361fe43add3a5aeb519f3312412f0641512516843c638b13f143ee0aead4cd370edaa53c6e31d514142be1fbcd4ad5efd81d38bdc59af9be1ef225b1428ba628642e2dc320b28114d514fa44402758f9d32c0c69a693ef0d72737220c5842048f3bb2dd8ce6834e8f9baf688ebb3a90b76bc31d6575584e3d313342e63c4a13784db66323283edb58b422d8a5c228189da93f4f5d40f8a32a4e5821e4e5fa3f4d8205385194973f4693b3e9c3f6e12b012c16a8cc691326ae997d1915d9235cffdd9ae5130e8d3d2b5e06b0baba2ed12296882d61a0668ccb8046894c898314a0069947fc6b36b37945ebc6141023b14e7e763dce0e7499e54852d01e24bdaf472993d03f178d5d6dd5c8cfdb0fe6c488f4cd3b273b873c93da08bc9189d96b33ced65cbaa468e6de7e4e7ed3239c4b4552bb15369c833352a07c626bd8381a459d5de205b2c8781e9ed97dcb0768fe4d36e4458ff769570e34e52d046d922eb089615d43ea08df87094b4222769128da4f3adfba770ef9416bc0773c4da3d791b48a6fdf94c0392a9d70cc0ffe21763bc3b0251583204f03bf7b4fa160ff76b561c2d745f7ba2d8afce6b05bfc8641fe56c3827a7da3bbe05de36c9a92a7c24f94b91f64d4dec5075b39b5d404af44cb0f77e750be9fda932ac5920fc7d96e86bb39f112ec504a993193862e1c1398bc26779f784b954fecadf80ebbb20fa0f53550549beafae04c892e61ef64453527a2e4d95d4e25a41b110db4030f776d7662413365d3afc2ac7052c6185ef9313a36d91b54543d0244d36cef1455a5ebf1ccf8442a8ab3904f659b86dd0251b0b56f68d0d1278eee22ff0ba0653f5bd78767734502de72d1dcb3489a1b67996db590c7a3f902db1894842fc5a244ea8b86cfb704aa5d1a3b0a7e24c033718d4895f385704bcd8578a6855afe40e3e7dc3795389e0c8c921cbc23625c12ddb99fa9eba99bce9afd96df35214e216f6e7c1ee728f13cae0cea5da15d82b8b3f18dcc8712ce1b7765468afceac97610abc730637b29265026450c935e4ac4b30923bfbc7363aa7f30ef34d4d5be82e5f746573c959d9b87d63292ad78dc8a9304d6f2a30a55765bc3d8a8f363fc6f8c3d65778f15e0dc108ba73b5c8ec5d1b2adb52b60c9839c4af34012123d0f43682c2090e1c77b6e6c9a92617e549bc2cf1cd3041f0339cda1c1b5d6cae3056a1826153548b340f57dddafef0d412167185b1581a65f8dcef66856761e6c906fd09b96ec8aef9069e996e75557297252b6f39c78e4b44688c4abbffe4334cb5eb6b5e23d54060edd5f73864839dd6052068ada682844ab47b7e6f2a1f022a4c778c38f28d58510c6bf047387bd7cadc70ab1dfe9e37c7a50b53a8e603b1e27120e9a6b7e22f9ca6561965c97f1ca675287e69fc1c23fec6d160a5e2db90e066b6f614da266df9d3a7a47b12235d0be783364be6a25fcddd1ec73a9faa352aaa1d8a73cc2311c29dcbc9b374bef4c3c5d1750364931000000000000004f0000000000a03b9d66e318c6046f40b6b7d029d8248a7f0268ec15808698aee226da851551c317a50009e80b800194aaef95c75af7f46c4d6233cbea0a85ad7f487f6581aa181d4e4760369c232d48fbb294f4fd13322f47914e111f90c62e196f77d7b0901eb69f632c582012c086192fe4e7a555b30d1b45d19cda55d6fc223bff191f3c6266407b5a64eb55ddc987f188a3b559d84d17da0a5022ff5f60572ac58917fd85765753f0f076da42b9aca114f8cddc162d853fd415fc0090e3df87b945ce8226f3ac24dc361a7f9aae541d61dc10144bc705ded474b7ed65bbfec89c701f54014dbbf231e1196189ff18ce9d8249b41a88e29b9bd3872d66d739025ed3e9c64b0366dbd5360ed6670b118c0e72cb22f74852c071bda46ae4bddc7a2e0c8f2dd84c3caeded266bdb0326d2c19d11071cada4ac376fdfefc05d7e0ce875ac4e48fec1567251db2e718359a8f573cb621c76944d24a56f806fc99397dc6bd51dae76fe34a7ade489a677ccf8962ff0f99388c6f7f235c8030f45ee4d79f89cb83465e1681c17d2d1490c289132c935ee65b813c85c20d4f4b90e1a4c586ae9dad79c220e1e193c885ecf59808b66ed9d5843d3c9d7c412368b2844a14e878a9b789c4814bc62e01e5f4751d081bae4196e0631a1eee982ca44b05f5118c45d08d0a8443d45d226e86619d90cdd5663a967d9b79c5d4db08fbf8268d1e085261c12caa03b360c863774777c60a0f11f37e7e1de5196592dc7627e93d4cd7063f362e0688fb283df04a900693a37913a4b9013638088cb9d0c93e741e68fd8280485b2516b20d28ba7f1703df27eef41bddcb5bbae50935923a33a1a2daa43f4461cc2ebf498f41ca47dd6fe610986721ffdc2cedf2d2438f606b3688e818d0c4ad3cd92a8810dbbea2f670aff991634a0013bf14176fa82fc7c76c813676b4300b198cb8e372b6b865f8b4ae4e8c0b86a5c1424e3b344d9c32abfba0c97795078143e1424bf57df3b9f7ba0fb6da4cda3fc40d6cec4fd50b14dbb3b0c760f2d22237a0e4b2eace03b8f90a81a88818cca73d7ff04d83bb819e5065d18857663aa016ce2f33f5d43b476368342c91fa63f5171d035f1dd62069abf18a8aa8e5fa3209ac32c9ff2e96e131caafab440708543e5b661b09c8146344a207a847bb881fb6451fcc4ee2c11276636d8686c0feb2c3a9980a178645bac8cc100e01b06e136c2890e3fbf8d885f70a0b4c9cea35c673a86cb8b032155237b46d8210b927692226c01e7d6f3b9d85de09c75b96fce3742688f56c1a3347774a7f800c37bef0de2715898d5fb9c66aea465d8013324f07df50330f92729abf275f44fa2e77b22593a610336525433dc75b0222074b4565dc86228b49f7506068916f4c7603b907a0376158fe7ee6debf0f6e589c38a298a21f7557c605119a087499cceebf6c111d25f3d21632bfcdb5e2232f5a955c4551ec83572aacd5be99a969ccd6f0b4a74c1bf88bd4425a18d2f470f711e55022cc2c78997e715cb9b4c5d70994d89494380c9e2ef0b7e1da3cc41e54f7560a712ea307a488666aab16a72e8f1ec29a3fe808ad656ff754ce5f42d403216e45344f60ac3f93723a59d253a909c579a8b0a2e550bad3b69535dee2f1c1e4caedc941e5b1f94d7dd30c94f62c830eef02657253073a11c18834dd7a6d068b3bcd72cbfeb182daee78619007de137417928ebfb185fb2a274f5bbb60d097f69855c2a616c0d59881ac22319a165116dce042063d88f35bb68969d9aa36ebdc1058d31495672d40c175c02d64d9ddb59867645780897a611ae7f3ee41d0c2f4a32f52e25bf916e46848f6e3f5ff31a2971bc8a0b664590be84b89d3bfcf12f85454521b0d0d6764fe1f23cd76f148cf83eecda18aaf4a4a6d5852c95010a6654b834a99d7a41fd1430a8cb42b3607779fbe6f564f1470d671246f1cb4e500c09ba129f7962de4c0ee7e74474262007e6ee8f639d4fd51a95550ec539e61188e14ee5e4d9ba5f7a61e2e8ba81b24988000000000000027c000000000501dcea4f082e6849eab4cd7c613679487d304c8fe37050f91e02e18e1f7258d419b11c004f405c000ced5135f6e6adc9f9e8816e3f8c8274ba23df4fdb06594dc244d872869ea3b4b0d0811a1e67aba4c9d9534b4e489a3228cdb4d53b1dcec645d2019a969e3eb8406346a9f8e9e844e30b20f118298bcf8affbff4b6bddd9c4c8d8d1535e8d6829d8ca0c175bcf7ed6c7ef99947576837ba8e44b72b796d75613805c11dbf9fdf66bc8a68ed9108d29b97114ab8f568cd7fd6b0c7d5dbfce90c5093b9b66cdf5c7c8416ff92e1a18e75cadacb088c865187cdbb8e0cdb51f8e3a8770067d3e9696a18d14dabb1bbb2d9c05b06a275c6ae2e43ea9ff28188a7f137b822063e1430dbcde1580162666cceb74140c1c8ca9e5a4bb7f3f4cf42395ca2ef6da56b82ddb793c202d39355ad290d793d3c6ee37c22b632b8c274223a6c7b2a100de769db3987c4c40fe21fed262fa87c1b7d0130fcee4e43eddb3b80c57052c403213168dc6fb26f3d9030a130eff7daf7eb726a3a80b5ec3db460d913cbeb0bfe0f57ff9441a2fe135204734dfbddfd9b8797999d41ce6eaa6d2ddda3ff1471727784095e17149f6a93ca4c09ff35c9b4793be12cead832f33ed4e40f91fef0874385afcf38fd6935d7bb0de2e2fb92a3fd3009289f00ff9e586aa9d6f2646c51e0779b60831bae998d9c4d3ce9ab77b1c30bf2872cce5aa200bda8ac28dbb68004a6fd482a8e4fae9d5b928e1edf8d4691ae0cc4997feb23344df979fbdf20b615f81eb044828b0249a11d6703a1e632c42b8484a5ad026d3adbce3d979a799aadea0d13f19b2374cd42a866b838be0774778e6683edd99130c7ed3f153f9378a642de496f7ba3a3b4258fd55fee1b7f7db979df156af4ccf9cc2350f440b3243e50a0ba00b09e89d8fcadcf8b2622c635686a4338947f25827fe6355c8bfa732f7b98327b9b467a58e02f8cb84447f9b734a383d21e3e8c4ea15e4be86761dc8ba6640328c59604ec6549b56c0cb448ba8afed4e5ca9c1ee37857830c9a65cb2bbe1c6e68dfded8f0f8a1ac731e22496d4d5e145b836277d421b327ec83a22a7e805aa8c3e8d197bf66d4c4580693195e3468d5babece48c6a60b3b75ec26f20c336627fa72cd6ebf51a3c84ea1814bd23db3251f9c8c143983ca53e2bade6c528b2703e732373e1b189d6b9a4386bc0387e6a1a8ed12d0c286881b6e16a3040ddd6db257ddbdb9158cc8001a1f63a434556a9f13f217f5b6c29893e5973d914273ffb52017f0e002cf3f602ab21167172b492fa47ed80482c9b703c40f09dd8a69cf2e3ebb095013bad8cd9234174b32944dc02d688da6bcc224ff7023eceeb5d65a2b3c345c3462c09798d5b21ee9a85b5cf38c9c1b66f0d03f58e8c323dcdcedacf6ec47c3ab287375f524f4edbd263abcec55ba1cc43b0506b5394247ff83af0a640aea5e138cfe7210e44368983af6d815f04db59735c47c8b603a1a8e26520b037256e7054439715c28f00623cd1699f6bb975c1d4258860b4a9e037e02cbf9392418d168bbbb2e76fa39109fe691cf49af818344ad76f1c460b3831521e32004992a2527f4e13e7d3d8fbba7ebbdccce18c9f26cdadc205b421b8837a00c3ae32ca923d32c98478b89a701bbe3078442b6dc7e15b182bdb5fd6281ede05f642fbfbb1a4dce9b70707346dfde871e9fb256c706fa162afc782aa2eb10f3339eb05e999701ac7bb0f26e0c578a7e30d89cab9f7ac2b0e0c3868983408c45f4765edddac96e6ffe3a50f993358dd72770ba44bfd8d97d965cf109d63376e08399bf3a8336d32fb18227fc9e2919a6109aa03ffe7f1f6ff044740e1629e3c78f846f8e0bf843472c143b027bd36cbce575eaafcc41e2fa5611a3625fbae47a5c035a7d27fd483ae5a649e752112dfcf7e38ca719c0bac094babce20b1605d1cb3f6dcb70f59eaab8ae5e3b3a25c8db0cebcbbb4f3543f37747b1cea7ea8d4aaa87629cf308c470a772f26cdd2fbd30f1745d40d924c4000000000000010a0000000005f5f168fa12cf6aaf21700eb1d6ae84fb75afbf10b0110a80e1b1107b8e26d47629f34600279f2c00052a9b5176f20ce77ca96e629568a6efc7ed1bf663c369ef88fb57f427be44d03da541223ecb31d71a093908a74a954173225125992942ea553565562711f3de87fbf4e0f8d5ffe692313f91232dc7b200f386b3d095e09654acef0c82f098f412b30346912aa778eba7172ae00a346d548f1d43dd6a92612206f32dd3f48d5e75787e98c15c273ed6f1e651eb12ce06217b80e93838b73fdd3482b8482434dec1202729049f56754e7ba6750b3378441c23ce852a1f1d1e1ae8099bc90f7664bb8f570b4cbcbaf9b6f6a93f3e057db0f68478ed3d4c42d0e297cfcaed1aaf1579bdf7a4e57779537b196b76291d5ef644391c1c50ea6d10a22c24a6d0eca264635dddfcb7f1d7072e7cf57d6a0ec92a7cb738427d1242e7fe7506be1f309c0584a0e3b90cb4070ca10df4abf91117430dfecf0938b93b0c3180d2d2247ae0f58cec7ff83b28dbe053cd0273ee97605be73e02d8c8a9c5be2110497fbab754135079ea8b1700da94155b1fdf4f12aac52001a1b1a551bffc5614d93b8dfab94ec3e1aab533f8b8f5773e3242051fa261036023277d7639cd8e2d3bba334e1622668a61372fa9629dc2c1075efcfc2f32e9df419556db5ad00b8743e0617ea031c7de2d022af8172d7281c0f2c78b9f90be68bb3696a1aa2c69b9a96468492f8c0f4093d54adfb02095573fe3686bce4de8800e725cc85940fc80e899e8c936244c47e218489b00332cdf20537280be8447e8ca613d5b95feeb9047a7178010fe3b7ce389955eabd7be55ae8c05d6fb06e2915e548a26f8e7d0b93ad0be46c0600401b763020afcfb44dcbe962fa37e7cead7fed5604956af87649fdf68e1301c1189bf58a59ff016e1665e2833ac074a2df2b4a7af04c151cdb67424eab1adfb33839e9f5426a47b24cdaaa1fa7d65887e811a9366007520c383163e01cb9a617dc3e8419d348d4d37b149c86fc82f1c686c71622fbbd9b5b6ced7b97d75c5768e43cabb584b93c723046b0b0a40d5c81b7527fe8e166e4651db40a52ea697e5bf5d78519eef061267db4c0247628b2cfb28f3c4191a3eabb14bb10f3883c09924f27a81cde81f27d4c1c142e2d24a23ad5044a6b1f86cd2ececa0d6ed5883f8d489b67b93ce5d7de926a8dd0347afeed38eb0250af8a9d549ce55504f60021288c600e797f766b96c6a635f6848828d0834483ebace3e3a90d0ab5f4849fd88b6a1ba04586253877a36ab79c14a3bd313a87ec75398de5cc5fdeeab17d8787ee552f3f7646ef15ca8b07cd167d324bf018c163df7d910f359fdd45ac184bad4383cefe961bc669b041477ccd714ea428cef73785ac2d9c2e02f39d6b5a6347326143e8f7a33b584842a8c7ef699b550c82cc8ee96a93b6c12ee9d37c05d27bb138df7f00518285db875a6da8738c3cf64186badc844e541bc57efc993f45ead325a5b0abac30c42502c1ead7e9de729e1f44c1c6d8738123e20de0153c4378daa0287634f11991b348ea03e160fb2cd0dbfd7202cc6941a8ccacbaca176bc4b8c76079ffc9b774c5119504ab852bc8082c1e3cbc81c2894c7c0b5869c049ae81cb23e977e62f725d6f176cc64296ac19bb2fc244f3a0e4cc62db140384144b4eb088c08bbddb8376ac801e66e3edeccd8f483c178fc878f7173b4863754bad647188b56e3df27bd26f395a534cf5276351013567885f6fabbf8d10757b1768683c1083fe2b09177b965ab347f3e609f87221b6b522bf8b1a7ea3799bb1f8ab74212d0a3a832b8ab721e4b5ca60d9353588c8c73d764f1c5f0e4437f4d1fb3828c1fa93e2117193e061dcbf8783dcf0992d98d00edfa20ea3a874135eb2832a5a00e3abf3d812cd3cc65571b1cdc690625ec9bb6935136ff38290673cc132c2692af48828bab18e8516914fd4f7ff84b6c7f884b25f9bba3d8e753f546a55543b14e7984623853b979366e97de9878ba2ea06c9262000000000000009100000000006ad79ec1d9b5ab5362b44b5e2a46a69fdbec29625c11f0daad2dfd7dfcb57298f5803c0013cfc200031936049dfe37de5cd8ba76624f67aa9a0100e563277484829150b7f85a9c21096b7e9f5f7eefb4542d172870f76ebdeecaa16d63edcaac1edcb600afb720083d69ecdd22a6779348c0d0270d145f91b6879a4c459125ccd378248a85e1a6f5ab19501250cdf859456b5d166fbb68a4f2624c540638852f74dc0fe26cf2082e84cf8f38770d0d4f341b9823afc8a8fb1ee361e9e9a7d928f5f78a71284b418c543d7c95fce0bd3eaa4be93fc2c06c0dcd3d47c159cc7d82ba01f23fca68961309731831f062d97d19a83ca25294ba199a024484207a189a8e5f6addba9f87ca6f4ce366d909db0e5bb48ec3683d899019ed29c6d48f89a04a245adff4266cfc66b364b01fa4c219d3c9125bca917c7bd4536a6bbabff51e8150a8f3901e166a76e63092dd65dd6eaa4eae335b4bd7abbb1d072aa0f13466b60604a8f7e0046c3bf123c5ef8c7b9dfc2fc56e1a384596413264f25841b731063033b92b146ef7596f6c85b0c1b41f87c682cf8200ffa4a04b1d8686ff357104c507925a24cc0bd0b34fff883a806c8b5fa7cdbc7df29faedc2a1b87c194ad6d5d248f134bfdc4f092feaf47afc43234114a1401045a6d550acb82278542886840058889cf3562822eba983ae98fcd48916d579d94a67881b76de9d308ec811fb177264daedf8f8f2e0135ed39a54e299636563eeb34e2b84ef11a53cc0cf17c6306765fe98a82fde7265458eb9679c7bc934602361139ec8053ed24d350ad0bcb85ed5092ed9ef9cac7158c98eb37db6ffa7cdfc166ede117bc2f4d58a5e553e52d2c211ac59bc8c36bc54148e71a28b916a3938fefb62742009ae20855806cc87d85d814dfe2548686f0fc1ab90ca47d31321bf866d2c8cd0b6cc7e5f2ad6bd9ebf6ed3c3390d25c21f78a06e3131e927668b5ed9396488e479378c244dbaa8db1ff30e61f09de1bda9b1f50fb434af823545f17e16e1daee164a75e4c748a04945c6c52e926c3414485184986e168884f9aafd38533850a1e133d051e33ddab6da662ef9eb2e3526ac41a49695160be9cceb0ff2e2bbf605851ad1c1dc04f7816238ff7da6839721af26545ef7e47b549c6b40c27bd614c4515e3ac6e9b37dd3dfea4fb3f7f6745200681e933d016600c1d80af4168d2f83829a780017fab68e1d6cc4b7662b1e4e04f9257db48ce0526cd62c302b0bfac9ac81707fff48783875a499c2704328c83ed59de68f1600094e841cb0f0223bb4b492e1fc5fd5859e1b7e990a4751c53d5227bd86dda942f1ad46caa6244d4931834867c7d65547b026c0776e0ce8bf4354fcd8913537e888dc87b6866c4b8823ac5c6c142e272b3b8f91de448182a338c6a081d920262303f464b559154c0095a6e9852992c9348caa35903c6331e83661966e9d8f2d9f5517c4e99c1cdb92a0c42808f69285aadce7c0365f2a38710719ea49c281a1ac72ba162bbb93ded733db6b6a74c51c9a35e8e6762d845c827db9a4a52f6f5823f274f22baae3cc5ea30e5c07eb4b9ec9dde893d8b79dd35aad45ae7c3f653d3040fe9e3962b8183cff7e204eac31b82f61eb3777544fce8c4baec6b2d8b2a33733928171d9b39bf03d8b8732ed247fe802d3ac28cbfa585bc9ca7623cfb48943b783a00c753bd63c2484c75b2d142d841f50bb6497597e0438b2b0c43075d0e57bba4697ffc44d0583603328103ee387d6e19c4544b9d06d43ae968993eb83eaf3fedbea6fdec601634fc031781b222dedcd4893df02696fd00f0562d43676cf15ce94955de6d600922a1b49e509e5f2d350838c0e01f0ca8f9b1463a8ce3a22e445bcc910b21e20849d637d1799c155c028812d32b5a5cc35fc5bba78a925b2320db69614b51ac0dee930103b312c2249d443637c99f4e6f9cce31d935713bcd2788862d2ea9d144bc0756fcddd1ec73a9faa352aaa1d8a73cc2311c29dcbc9b374bef4c3c5d175036493100000000000000388000000000a03b9d3392a8ff929a0bb04cc6f9c26505d0aaa7d28fb472733f57891de7d2db5107660009e7a080019d792ca6fae1d7d0ee7ec503947a56792b105520773dbe53b919b368a6a02c6e5631230dbc72a2c42e10d56940d2c8e5d73ddb7a1917dc66973094c54e7beff5b75bc1d3c668a8a5d56c8cc35d51502dec02ce8e77d9dd98d3335a84a71b88f43667e657860b433caf14528a250a752c8892facb9d84e1bf9047d35d474ea3afb06d0aa8812f7765a525dec9633ea9e2cab4c5329790de66ad4f9560a42ec08e86bc61ce1072e274463e24fa5f94ca21753c27e872e8cac8a544681e9e1ac9405a356463bd9ac505460c11307af840c53bc46e54816e0d0f6062858b98cbfdb5f0f3bc69fad36d9e3274e9a99b784cf36d2a86828f72c04ab38cc19452a698a23ba32c20577868b26890e520db8a907c0a0656976439244f8a1c17c50e3e561b473de7e0110a00d0a65767233ca79ccff3219685056edb875fe586774e8fc49d50047afbf563f77489838bf318f92eae64613cbcace67eef5179951002482048b9c8db00b09861f9911147e30958bd55c74b4f50df9c168b3f39261bea561bcf7d05628c673a83c25d7bd423115420a3f2d0ce1dec76145b03919f51ec9255b7b6d00f5656ab8bafc4ca9ca0aba75c9d26c0fc9f8d7c148313fa73642137e5a808090cb781809742c90d8a1ca1b53baa0830e1b9ce25eedc7ae3d0c639b97070f06d0789f4d81a9b1c7375233c59a505bc39de10bfca357caa18d45a320eedbc31d3e3f965c11b98c34b3b324f66a572aa93b2c1c86e244e8d71c5326272f9bf5f5b160e78fe558544ec1134846c06fac202d9aee94624d1583cc20ee71bf26fb9a42c2b2ae3ad0aa4518f1781e65f01923de1b107ec13dc171fe4a39f13066273c31ec8d1032bc29ab4bbe85f6acffadb57df54d9eb59613165460dd20050e169a7341b12c0cf6a1fe6eb56a2583a659eca6eace713736a3587a6201d9470085b1b0260cc6a08ac590f4a2cba169ac7a02853947c60607e5e1142323f199f8127cfa59728cfcf18700e1df0b1c916c136742f3a374232e470da31d4df4b0c20856da8d331e05a3194baa668d560bef875fa223685bd7790495d3789ace5d1e03199f8d6109aa710387962e5b19c33191af3333dafc2d520824d467dee8ece4b5c5b9a7412721de2232d6a3ed9684a31f5a3b6ecc6e8f89a6bb4e10067ddaec88e81bfc9b11a6919bdd8005e087227c938bc810d3bd03545d1af15821b983c838a3922eac4603985f498e3c8fc16d64d01b9c434c27373ec3651e8acee2cd7c2e729dcfa9a21d82b3d166e00b7d70c8cf0af8fe1d3b117a6b05151504dbb4a2d7dfa6837ace1af5bc791c0e39b5452f3e921170cef04e166024a4896f009759043b543eefeaafec6a83b67eb46600c5699f9a040c80c0b9dc9d3a013bc0cf26c10a77aa6f87330f7f9ad9e274272706b61582ae8b44c90ba6bf17aae6a4c7af86b114d95ec7bc034e8f30a618858d9e5e53b4658272e1730bd09eb58ba616aa10299632e4efdb6cef47357f68219907fd8203e8e6bc0629ab1fa877e41fc81bcfdf36971cb7398d410cbd8ea90d3f1b36e925b9cfb86916a70aca8b19cafb1dc66ec8328b785020faf27371912385d863b4a6d1c6c1559b0872d8d1f536bfa5186d36bf431e0153b8006b6a553004e54d8b1169d70db00cb277c0ba7e24019247192b7a580d598bdd373c75432b5d23f72ee714f29e4058be420ea6b251523557f4bf783a914f3662f677097a3e87668781a0f85b135c7792d776e71e6011789ca304903c5aec9d320183e736d2e27086522f4d259f2205b1723742985a7d50d6e77c1c976ed81c03431082f5dcc8511958d3eee2459f2e2b328be5b661d98b9e6c8dafecefdd3084e699dc35e82b3da9e6d7a0c1d330b0d578e156908e119710044613b47eed575d51ee0bb96dfbb215f0846ecfe6ee8f639d4fd51a95550ec539e61188e14ee5e4d9ba5f7a61e2e8ba81b24988000000000000025800000000000639c0592343e0cbdf6161c89bc1d5bab589589b05e168b084d44057d21d922944f414004f3fa0000a470dc7715f842502bd608fe123294c44f648d913bd5ad7bb0f110cda06c37bff672fa69012c4299f1ac5a62dfa8321e0710fcd595917a9b8888ca6aad9294292f16e9c8c89346a57e1ce6b3bb134861f35bc3685b9f7bfc8b3a2466c87f2c08cc07eaf714a363cfa3927e45d136eb14170019625e432ec1791f57dd537141124c502d6c3936b3082c053ee348dc4d965e62d9a6c8c4e54bc5cfdbe1bb9f7dde179f9523ffdfe3357541adbe49e6d22ead00d22465bb96f14550ddd81dafe60aaad9085c276be913381acf45687d559b4b8f6b9652676c5185595a5ad15adfb306a648e0f874c4ccc0c8674e4f6c5efcc9486a7267b2c18b264907195a190bb1ad40a31a6a8db1644a06d992f6c180769e6821f061b6129c018dd105a3cdf142737fbe7227913eac10fab0369f2ac94e547ef29be6506059dbba837802b46d9ddb845052730ad240bef288e2098696a49a877bd28cf6f8ed3dba903d8c0020c323fe6eae50863a15d695d21ca15e4ca9b3779ed6410cfba163b7aad2e54b9ab85b89ab837ea14fc92ad6754d8cc0117d0aa97316a47a02a8d105ae7c759882de1d6b5fc1adc7057276553c480fa356d883464a512cf89269cde88df15c2a5078c87464a2bc22f39c1f0838c52421020b05a702469c8c25b4e382eea70aa95b4f1d67a03033080d859e190c68cdd46f3333cfdbbd210f9580a39edac924385229fe7dd5991cfd0d8250338c45dfb8986c01945ea0a901eefdaf0dc90a230f37f7ea018d773b124714522d63464866bdff104ba985118064c8203ec6cb2bd0a71b93125643c7af02ec03fab9d4f5d31fb6b6db03ee46675f915c7b529e45f20f1e1212ef69a1e150cf5a9cbfb024560c0d76fe3b0848f57fbab1d885ffe41df8305da9cb070b1ba715182543935f0d0a0163c6af42a277d7c45caa42fd18ab2916a58521678b8ad6e657583fef63d04b0c5a26381f13822c4cc828dbd6d08f74438b10d1a577205778d80b6b6ab08044c4b19cf21f6d1485da617973451cc422fdbbe2dcd7960291b4bf15686b66c3cbd4c4044fbdec5689022cb3a32212f6a88dadcbaac4db73b0ca7c5dd9ea419a45f184596a9887d7612b2af6ca019c3e62a7b97e18ce2becc9581b946e62209b82eb6ff4f0069328dde99847c96f378c38a822d01b0b2f186c771cc1211478929d30eb2b434e307b32c738c588bccb755e44b81ad6037aa3871746c26cb300f7bbcfd19b9167272605b88d74500a6e4a1a3c9d19c62011903b3b56aefbb2a1981ab4ec1d03c54fba996c83556dbf53763bfe8b78a55af6fc5d70d6936dd075aa86aae8387cd29970a6e6200f833daaa117379491f8a3e4434fef975b811a8c2f01487ba623c0e16239b374211691b96c3b98ca22d14d8e96972c53c58510181a373fbbe8fc7e95a3c5ec21444784fd7b88e2e87bd5a1400db5c4f25cdfaef4467b7ba76293ed7d703f6ce43a5096c17096a1bce819855a480681c55c53c3b59c0a8f4dc8e4a4f8ac34f2b1120f1c761003fb94f2a53478c97d4b2e08d74c80d8ff2ac54943d1e964bb495de5392ca93c803a11acd22d2baab162932da68d0ee5dcdaef9ba3af201ebb671143b66dfe17a8844d5698fba637df0a8cced4f97a90d5154300e74e9dff7ffcb4068656a4b309e99104d58cde5ae9a716e176d2e7fc0a9146afa537be43c76cf881ec4a77db114ce15ba64270b6e4df3c66b2fc4c9a9de385cb0a7a7aba1a695e54895c706fbe440f60324d19456695b199745cc24d05ebb354753e962fd05bb7afa8764640d72e7f36b3cabe6b2fc5e2b752c775e3a101ba8891024c5ac1417d6b8c4092e7b86c0f6dfde51791814d46a9cb4df2bb0bd6c98b3a5ec5a481addba93d3309cb56b10da87718a3ce108e0f4afaece111da1e9de0a4fa7f37747b1cea7ea8d4aaa87629cf308c470a772f26cdd2fbd30f1745d40d924c400000000000001360000000000031ce1e97c39192b58f4b856beeef7cf9bb11a2d36b9d4a31ae32cfcbcf189ccc28d9e0027a02a000682d3bfda48a2fa13758ababfd480a0f8f78b128acd567694a520dae50f83a924d8a72dde148f649981010957bba6227103b164e8f725949f989525e432d109b0217be2430f45d0013c3710dd3c1d0d13f49fd77c1f9a8441c56ab8424a8b97eda48f560939bb8585da09dbf93031414e5a03405b4b123277e754128468d9f9a57577457c093649a18883c196486fcbbe6d1f41128da592cafb21c5d92439ce17842ae46c27dae185c311a3efdd547a0f17ce4111678ad314262bc70f0211d2b7d0dab1be516661c7f790236c66f2ff5fc035afa97c9e127934d7e8f14c3c89a40349d96fc99070f71ff40c5b4e855b489e97fbc7eef545f2ef92d4fb8f5a88792ed581c3765e2627a721bf90b9faaf70313c688dde77111989032a5f5bff5a74778475abbb5bf8c1d5bc4052c107bb92981050221abdb62c0635413f54083587fe618204ac7bb06626c464b085821ed5ec65c2ce909dd643012159ba10f8025fc644eec26f6c6f9b1cbfcf965d43490073abfd9eaa5eaf1d47aaa02751c880303ac74084f3bb7546c543375d91ffa624f9c3d41e863f0f3b5c5de749dbf17a10608475e8383408ff6e72c771b5d2b3a03f6011694768cd201a3f76e82155d442d5fa167ee306a2e445a13fcc07a5358c03eeb58b3bc0fe4e9ec1e3133633f3df04e4983c59e3629b57d86663e1d8d2ff7c981798546f740c0d2d2ef1e295d101b86538f8e7f0ed8baa28823e248f9f6bea69dd410c1b5b241baa6cffd4de6b286ac5f501884c21fa5f08476c8704c893d1688072861eafaea36b96749d662fde665096400b72946d6959d49bb849e62a32d1bb04b3be59fc456e6f1682f819b9b702d55167244c7faf28f315d98dde3198c588cc7ee48cfd9704947c5a673ebbc1f476da02ef4eead2c6744a575cc85f779e0965a77168ef23dd5b76ef4b526418640f11fb3fb06a8c5d5330e8c87015ab74346dd050768b632dcea817a29a4f54f0c48babfba76ca7f579d1be057c193797292cdd843e978a620d3bf8a0f6cb5f3c63ee09c926205e305c5f0b8e17b774c2b6f5a6a328b6d88ffe5954d3d1db7ccea9b457b0fbf57dccab96b0c41518fa2fdcbfc819fa60b11230f07e63f5399c1469ed6e9dfdfe41a47f77b3aad3f60bbb6baf126e04c625a2b1b6c9a9ea41cdc9291fa1be251d914a18fc211accef073b775f16da994576edf0196cc71d2594621bc657f5e464e61d55d4638887e013c0a0c150faea5752e2579888e29c5593fd6eb3052b18e9794eeeaed9c58e3d2cb6716a016f5407cec1ed4771377f6b532d474ff2255e163dee40d0e8e27d7af6e8940e8c79928d532cc70e18d65e00d1974c302437dad40b79eca45c6f3d4901e8bba60dba5859a78518e508ace212a039ab9b74a4f210ed01185a2052133857ceca624ab102a61993ae09dc931dc92f3f58fd7f441f6471a757c739ca8e9e385f83339b2ecad4acc34570cbade5e0531cea0b75dc0314ac726c3aebd91ed7190a188433e024cff805e1e7b22a7be2cc056316c7ec9bcfebf11af2680a24cdfb753639fe040a8883c3162f1c69c8f5098c328db66216c89a7683b7d2d9b7feacbc195766357128578078db0641ae145ca11d7673ee5f33d8e86e88ee7d1769b871eac177caba81085e8cc299627b0eee34e70622e63bf5f3a3a22dbb60d00525baea83245477dc748d64d15f72c6b72f3e992e51978cdbbd5b89928fe7015add2bc96028574970e3de6969f9e19149d511a4345c5f0fdd56ed48e1edc82f8788d9a122b840422f252969ac7780c1e2dafc4c4ef058bbe100864e7d6d3e07864f9eb752d30db8c3aed411ba1825bafc498e6983762d5f2c3a3c65fb77a675e988a14e46aca5fff7c1369fd66d121c415f992a5da45ef08b4a3642258be2efe77fc426f2000001f4000000038c13612800000000ee9a47acaf373d15c992a9d18e0208d7b8212775d7c8d2512c46d939513b6b857cd0423a069fa8b02eec5c8ccdea649987e612cadc64d8b52781e5beee9d074e7c244707b80000000000000000000000080107f37747b1cea7ea8d4aaa87629cf308c470a772f26cdd2fbd30f1745d40d924c400000000000001360248c876d3527f2ec973101897e23b0a1e90c3982bb82492da5ea4223b57cec835bcd132b320888f83f27f33de0056d51649ea7bf4a00215d4100adfc3d6ea090f3e1917d0b756d91c6db824bc60f92eae94fcadf6f500f96d3308ac3985cbfbc816dbf6340e6f52d2da25f3ff65343602ce9f80c9b63f68e57d80accf7ddb369365f16b9ee2f1e2e683a012447f0637b46f4a1a2fc6c1283535f9d0d8ce55dbe57938e95af2df3a6ec73f977590c4dc855cb0a0272ce28c00172c7f7ca904691e767678abb3a8d83c2eeed382deed9c975e209a5dc3adb58aeb5d1c891d012c2a7d904af237abb4d6d1526572de1d8ca80880f8f82b3ff4d9c9ac09a6748b013e58bd22058ccf8bd000fe9ade490aa09234020b0433fdaba81da8fab3b7bb286669915705fe0107f37747b1cea7ea8d4aaa87629cf308c470a772f26cdd2fbd30f1745d40d924c4000000000000013a0249442ddfc8bb393edd3b800de1fe33a241d65509dd7525b13d5a1c2e67e3e3174197e74d8f240d641a4421a699fb8a951c61d7d8b75a2015b43a44d23ce000b6391e8809c7306a6b0bfb3fded05536ecd14d3cab3f3a1e9e7cf733bc36dfc1be1e941b12a88b8571ccf992fe31325259599c801b03892e9594d0b56474a0f294d80c47cae0d889d9a6a2529cdffd1b40dfd9c08b97a1374bbed10528ca16ebe2ec36246e1679a127deb8833a3e1a56e159ae6063116000d23168f46ddf5a40fc2102cdd6bd88b0c3e4845138f64b77b68587ec1246fcf1c8ae47ced44da6e03a96ee8be77ac7b6c99208d9b7601aad25a59d55a7881025de3bf3f437ad1eea260c46a0118dfd61b7e3dce199003b001359771c6fa1975746bbe9c943351dee4e7647c8face0107f37747b1cea7ea8d4aaa87629cf308c470a772f26cdd2fbd30f1745d40d924c4000000000000011c0248a26551fcee420fdee06a9b5adcf39d749a49c6b7e58cab0cb21815f0ce322da5c948a0010dc6ab03d661adeca68c80ca1a9ece446bf8377a689b5de7f18059b01fe527bd4438ac4e7d279069475bdade68d7838155980445fe7db7c46fa89bf86b21524dd6c3ac3ca792e9d72f0cb1ce51c6c333edefa53fe31f05a502f99f17555c6fa043f0e8b0e28d410b01d8662b484f84b7abda6a5a17b314690a6d3402e2475f0086b86ae5596bbb10b2ae4352ddced3646143d71d11cad608806a7958df5a1d2f134e0c165d34d7a9149e3ece4147fad80078daf4b435747e30cc4b048ac25fc7bf9dfd52105010a62352c0d67324861bf28766dbf00dca8ad098f359e269220ef8319aab6f972725eebf52b41aee5f84611a90408f9eb99c4046b45ff4b287240107f37747b1cea7ea8d4aaa87629cf308c470a772f26cdd2fbd30f1745d40d924c4000000000000012c02484eae46f3d5c9317e868e1af84a940516ae4bfd1bb8799ffac9315bd1a883996d6b44ab3d1abbe03472fcf81ca3583495fab3597eb4f10ef4dfb92875f749f348ac064f863e296a0339655c87d7e9f97b78c6e5fc69cc63ab375bc6979723b9752412c06af648483d0ea66878a9d2782c1deccadfd2619318ebd9b2ad8d47b4bfa615cc06716d986faccbc6fe2d3ca697a01f34c5b4d54c7e99f7820992508ca4126b2cf699e1339505b112c313a2e67f472824adbb5f4fa0078830e98a3cc1958d0fdb11519bf9477b515f454bf704ca5f20f8d5d142678d733ff9bc10b5c17657edd5f8d7b3a5f8c23a40e2d04aef19b018af28880fa10a4ceb1c40cbff86bc29c7c0fa4d433ede8ad63575593c3b0dd742849a1dc88b21f5b48cf91071ae44477f906e0000000000000142000000000000005a001a00000000000000ca0005224ff0ccf015f911d000851f05954580326967d37ebd26d2056c15b75b037db60000000000000006000000000e9d60fc000000000e9cf968000000000000010a0005782654cec9c85ac64fea56b933444d32c99c4ca174bfe698cc565445e7b1fb2e00000000000000ba0000000005f620480000000005f5f16800000000000000ea0004ec21a651106615a08a60221839db142a25387445244c4a312f57483c8809835c00000000000000dc00000000028106ac000000000280ee74000000000000014000045d5713e5c7bbf4cb7f0bb0458b626062db21d0a1794adb35c6fefd705870cc0a000000000000064e0000000002f6783a0000000002f65d0000000000000000da0004ec21a651106615a08a60221839db142a25387445244c4a312f57483c8809835c000000000000009800000000028106ac000000000280ee7400000000000000f80005782654cec9c85ac64fea56b933444d32c99c4ca174bfe698cc565445e7b1fb2e000000000000006800000000028106ac000000000280ee7400000000000000d20004ec21a651106615a08a60221839db142a25387445244c4a312f57483c8809835c000000000000007400000000028106ac000000000280ee74000000000000013e00045d5713e5c7bbf4cb7f0bb0458b626062db21d0a1794adb35c6fefd705870cc0a000000000000064200000000028106ac000000000280ee7400000000000000e20004ec21a651106615a08a60221839db142a25387445244c4a312f57483c8809835c00000000000000b800000000028106ac000000000280ee7400000000000001220004c4c42dd52cf3da08eb1b30543a8489ce4c5ccd22831c6af7f5e3d4fd757a51d0000000000000002c0000000000d5bc840000000000d5af3c000000000000011e00051fee3fe41044cad58302ef697994aa4e91774aafc65e0cd502464c1ebccf47bc000000000000004c00000000028106ac000000000280ee74000000000000013c00051fee3fe41044cad58302ef697994aa4e91774aafc65e0cd502464c1ebccf47bc00000000000002a400000000028106ac000000000280ee7400000000000001240004c4c42dd52cf3da08eb1b30543a8489ce4c5ccd22831c6af7f5e3d4fd757a51d000000000000000320000000000f441d00000000000f433c000000000000001c6000dfcddd1ec73a9faa352aaa1d8a73cc2311c29dcbc9b374bef4c3c5d1750364931000000000000004900000000003d0cf0589d8fa4866f14c8636a0ea302cbc57bfea0f048b0ceb344aaea4ef2ade133ee0009e7e100015c2f73e9da40bec4a273cc087e53f0fbc1d2fa6c58802c211e2e17bcc5adeea303e042fefc1c2c90dd4e1d94e06f418a1c1330772477d1047a577894efda247852ab1079f6eee56cff9d30c9b8ad25fd9eceab191316f0955a258a94119ddd8e7913233d8d3e314321774cc0104c30cb3303b27506d0d6c58b4d34d6ab4a66eef540bef4c8566bd08a0f33bff0f66aedd85fd3dbbd6fd663ec26113c35d7c934d6eee110f9f9407f0ec82b6293cc32b559ed6a3ebab084fb8216eac02964b58a747d7aa89a5fa6b876f07b64ca82956b0a63197599f19e2548f7cecf26ef20c6ca55d800c4b0088bd62cee20e4bd1b2739fbab2865aab9877242b365070c682af2b764df69e01b5ffc05c43682529b385e0eba9d13c90d01d3f8db1b4db928ff0fcd050eb3c31325cb8f5c3df78c02caaddb2c10cc2374f337fc7e6ead439c9a318d4b6188c40fe32fe4071ef56e7600e562cda5d463ca8831920c5d68b4faac589870c4471c5be098bc859890789ffdb4a926cee791458d647302a001a6b16ee9c348dd04d3f96c19c788c2bf560d1888d07b77ce08420a16d57d9e70db917f71db70bb1ebbb243aad1b8d617ab5050e53b653ffeb7252f2690150fe51b6bc7e1a485a3d12e9b95211a3c74dd7ff8eed406c42801a9d3e3293816b52656816f320903aa510369c1c987b68b259d7cdb403bac4be9ce624ba1fc833b9be10bf7acba2280b31c8fbfb3dc1f48e30902dbc786d393c7ce1a6bd75910bdf639226d62caeb4a94d66f57b5058251360cd0b095e1349afaa7c13350677c98666be44f10156267b0a653657e08f98d7f877c9bc8e26b1ad241b729652be565f0d9a789a2a6dd6d032f20250351108506d4f837414a03be53a8acd2786f33ac01c3c04598fc26e73de678fa14b9494d0d0147a6c8de32b2d2a62b6a9eafb6cc3a64311a56c00078b2017f169d62e590869b40eed67266cbd1bfe6f4c94bbd9a9e641d54e70c04cdbf340e0128f8800a32ca18971be40459deac62f82bbba7e9baa3479ed964f5e5468b095de7c8092d28074697097c91c37875a2a5766642a1b517bf611d44981494352d5dc6c8dcdc21a0a7f7c09a76d7e842ca6e9ad936660b93867987a28ee05485964e1f5d42fce5993304044e75afed970276b0830bf8bdb6e95046d2edf3df605218d22a3e443817214536173b12ec9e6cd59d064d01ebaa4cd609df28cb84fca7675ca66fcbe851f455319d14ef3c6c04d19bf05579fc0e118fd46c4199eee7ded75971d42d33d1658b18151149f99da1179dc59433cc028295252212b4b4c3ea6d268aac91187f4149c0ec10212e10f04571db689241cd46b34e9c17be5b7504d202ccc55af77435f0b2164cf00a724c66ccb4456239fce6ea23b413dae8d871d94fb6a726f4d93ca941f8c6548729f38faf78401603e893184e3bfe07d537c7f3db13e11e120efc62f625b195110554d4d807819b2922b42a35bd532da3746238e3fc84889cbe8f2a3512a9e479f977c506b746c895971befd4dc370ae65cf066efdb6a567167343509f2fe71794ace2caddf0c4d5680f5151f47f164c9aca5d9b98549d3ecd6b2b4822b0bd7433b5e3fc2c52054fc874bccb9c9e2b56de44305477d2bf19084f05aa848c62c241e62eaae29238a2eee91549fb4271d05935623452e5ab099233d5173035a611976f0a175ee2d0d6cbe620b8b957609f67541e79baa15a60c44b8907caf7d78ca6f9474f70618cc3672092db2828f48d7746b1783d329dd86c174df3630210f1ab4a919276cce6c078756fc913427c58ba1755a39b93a8489e3ee415f3ce1ad9217bfd6720b51eeace1190b36e96cbb4df4c59e1852ba221dc9c8d6171f4b8a1857a1985cba32562aee79354e8818a54ddc553a8647691cd05582448636e9217e6ee8f639d4fd51a95550ec539e61188e14ee5e4d9ba5f7a61e2e8ba81b2498800000000000001b4000000000501dce99c9547fc94d05d826637ce13282e85553e947da39399fabc48ef3e96da883b30004f3d04000f3171d3f36795264598b83aa0cfe25f94cb21806bddb497a516374dc156e23a65f402a241a1848fe8a6a7fe4ec070431db4827b650a547638c1d82fd0919410cb54b0aadc1d9f961831e19a33ca2c708da9d07dc06f35cb3bbf9654d6a8a1031fcc5abb7159b31f60da3c1f075d7b071ba20b6158bfc23acaaa655611c40f16a683f496173a97de2e39d6a25f3e102335756ee442194955e400cb04293fd0e72b5c7dc9172bee2e3a4ea4b0344a13f3571e17bfc64aa883fe93b35a01c1e643888ac22c660672df68b7534a320894c50a31c5f8ec19fbf281cc4609915ea11b0956f0accf8792faa300fb343b6a942345debfb218fae57caff421721fefd500c1e7ff95eb6acdddb46c1c3613b1da9ebd891cd4d65ff12c42dbe433c26959fa33fde799d640119982899a24c3554bdd1fd586101aa6cb5c4dbfa9ff9f0f0bc6d3ad019e21498a4c4a0aa4e89843122f9302fb2ca6f4f80100b60c5f8651801546c8d433d592f7c32613456b609d326fa735ab9ff74e4d9a8280373f4170359f2c358d157989910d79b8596577793c0b044e9fd53918cdf7c32b4b1292b8ddf36d2a3962eb63c92598f93be8747d4e88887c31ae0cc4ed44501b13da85fb899e25375fea1ad162879ac69ed7d9f6b8b106a7654b36537a75f20e12e948fd26aefe4bdc6b69ae4917d4126b28d71ecb92a11eeedc21d259cd73a43eaf6d8d76d6f318af786c4364677a80f90640411b7cf4bff76ee97dab4acb974afeb593630f19f8e2085fd23ffa9e8553ad46776343523df0d0fce3c4a44e8a60f364aef9be25a8d679d63236c6bc3981c1ce2d7c3a8de6ef8a575000f2227fe2f5a676e327c17ed9b36d36ae1e6cafb3950a0dbf04981c9992717f88ee9376feb6f777e26fa956c02913f0fdeecbcda67ee193e5217b887ba4fe811b66146e96ec9375eb9ed8ef76b5419f8dae05dbb669e2a6ee772ea167152ace5f0b72113063ac4bb685dbbe4cfb5e9f46cd1907b4b3933731c14a3dfe49bdea877c2450f47a397a201eaeb8fb570ee4095ce4d2ab452d52d6b1755dd5a0d68662040bf526d037f7c448ac8df58b476e586d3e5c941de9f8f870784f06700fa45c36465073c609982b9bdbcce37eb9f7ea1483077fa9a00a821ead8e6183f8e8116dec92e241d3e1522872f783d16a14c85aa86d1eeedb440ec5113d080ebc0958a1f80bb4176f759b15da769f186fafb52bef55160ddd32128551861ad5ccca39cc45ea1a3bb949a0640399e42459ad0b81d55beb0b48025c9e5cdc7b2ced412f4f61b9e50dc2cd0e2bd20af40aa5fa7cabeb5ae3c227c1f9d6aa737f8fbcffb1f591de43469b74db601b7e89b41cc6800b692c59b72e508fb1a9e2f5cc813ab4bc3906a624641ff72762249b9f3e14776afc92befb140daa8f9d78e626138c9b978ad105eb1ac0b2179e245c0c3c2f24b3f076ebc5b4d022ca7fc9b988500266bed26bbc45648eb78ba17208cbac5246d5fa6cdd977313c4fd518e54ac662cda56894c326b8cbfb138ba0e6174df7c37522b40431d9adff7730ef74280a892a6129f06585a3bdf41b2523e68354694df77d10a48e9bc052f782a40774047c7dfdbc52e7a799997a0bd477b303b72e987f01b4005583d467131ecefad367b9ade4e5fd6816e9bbae14030c125ab3840df604b1fc667ed5f299da8f0d9d1bdeed0d29a419ddd3474dbbc016ec26864794f60efdbc32dde54c0f93927761fe4169cf49c1b734adba9ac2510761b097886a4a6f2d2b8f91a5cf8dc8c7158c8831033f470e37204a38e76ed6072428dccb9b069aec40f8c40ecfafbdb52d1b2ea55deeada14aa9606e2d4be0c06bcbd1fff172b7cbae9ad18b7a67a7a697a959ffab55a7e99ae45309e7eae69f1adf2e5c571ca255f843d45516e913bf1f1df3ff37747b1cea7ea8d4aaa87629cf308c470a772f26cdd2fbd30f1745d40d924c400000000000001400000000002f65d0122ad1e78148261bb2e7f25f0bcfddc19b560dc7f6385c0c5923715a9b38634fc0027a02e00064696956cd30057b714b18f40b703af4db957c8c9d0abd2c36b76880ea16f95401a9bf4d0bccf878049f72b60ee5b9e2f82abb5a697a5d1984af52dddfd2a21a6387139f89b22c6b39db7f2c949c0d4c3d2b6faad6b254cf3687c92a19fdc44012f8120f97dee55dafe38dfa97261e859d5f5649189071053bd15acaac40f835c39c667e986cfb9ca4709898e9f671837e32b558b1195d91410be431f887b015e3913b504d72a2480573c9ed4cd4e796a8bbb3ccfbfe1bbc667e41ba2f01722ba6ef6c9db98ae51234f41870b82d1eff394ee02d36c7410ef6317cbb8bbec2fab8cb98173ba867643f3783111fdd38e3a18bb838e847bd8be65b738fea2ab9bb876d0ebdd65b082616838d1f4d019e8a33073eae16c5699d7b2803e53021333768f12a974749086d4944293198eafc391a36e0f33351c4a12971cb4af54876ed2649c29af370984fc68e3d978325791d6f0fdf2e8cb772494c5aa4d6827cf0508b9ab1a0470740caec5485bee36cde3e4317947d4995379fd24990dc0893ee5bd0f3a637227864f229a11c5ad0b42f7574d5e27fac33bc32aa1adbb77f1f77ddcdb76317cac6c72c2507c62720f79a44661a12502cb9ef209c50efebb361f4050c2829e018477b5c76f5b7501646061615e740180e10f973916d9e3180e95cd6881e20dc168c5ea5f13037edac6b7357cced4a0af03da974a6c83133197b84081e3bfbf4ed3c576c1a4ff2f77f5e702d31872e5c647df79b22c6eba9c8bf9c2210811b3e0345cf1c446fe9cb3674c412edd652c9d885f72b96531574ce3f4c7693ea9d05b1327537912f939d2cb02176bc1de9e29f19e1c822842202347960e076aaa062bd4b0061384ceebdb57546d0bf53f399f1ec4f40fbc50d72d2c4dd66d9edb78dfe998418a7ef052001833cd1608a8751c5fa4eb6c6fe2c2e27a49dab8e20e4038d3d8df08fe0dd9803641458ef32c590fb9b2bdb1221500b1cdcafeb617aa3bc1746abbbb3d7a09e08e3708a599ee5d1a9d3d290589f9ffb18c0849cdef7d03160c9ab6c2525de4c67afd0220b3fdfa59b7078ffc8984c8fc54b0ac35d558a2b647eaf8f5d04eee3a704fcec63164e87041749a88406fefd97c2195ae9416eaefe7d106844ca57141166be471a26f072027ba96b691d0ed79224abcbeba30fadd9ee1122af9d411e6c42e1ee1756d9fc72d4750ba41ca1214b184cefacf4955aa9ceff94b31cd74d5a212de91afef2dea7fa7ff0195da2a77c76e54ae1d73df2428358ce03f2413febe5887f9153bc49e45f523686aeff3cb13d548c3a753c7e27693e818962382328e6722e57c7a12ddbc34cf196873be2855525acdd582e9ee151d8a8d968e0d0ebee1400ff69ffbf9b3837350736fea082952d5633b1cce44f8a51d7f5bc242d7383bf9e1b7b67fad1e631a098f2a1b366560d36938b080819294434cc35aaf7a193af68bf9669f8c7a7fcbcd6566fb617aa9d13bc191802d5aa4c5ec043bad041e1960eaba3328974bdf6587d748e25df2ff45c61f95b6805f81dc512285d2c481a9fdf5814bd34dbf30b46c2389db868f748ce8e369a976d0bda48d4421b7d13d61a4449922668dc243c09ba3ccf2c7f314f45f4b46808aacc8440af0e4aea98bcf3d3f75f54efa28decf6f10df931cce807fca6bda611ef9d46e9ce39c4886e4c7c14830646cfffb74b6f4a07765745b48bff89ba5a0fe51c273b50e4fbc078483703f8fe19171d8ee28074b1b4a94acc577515543be1f85b6c822d7b8655504c702202adaf414af526f295df8a32342ee8c25bcb545ce20a613ce720765e95a76ea0fc30f63aa74c837c19ecd38063cf0c5d6fce3ca5c7dc552efbb61f8ea665cf6879ede061ead478f1fe27280c928ed34ff347847aa908cb7bb901ba36651fc8b9c6e7deb5bf9bba3d8e753f546a55543b14e7984623853b979366e97de9878ba2ea06c9262000000000000007c000000000140773aaa10a771aadf2066f73a0df6d0729503b0ac829872077cd12a6556d458d75fa50013cf580003d64703dc8d1ddb44911b32d8f21964a1a4f63a782996e0e949d559b5c18e0dba34cbb19c7f1dfcadeee9bcf8fbdaab20e6d3480f2ddd253beccc0f3c4419342af6027df59e3710064fa6ec1cebb92619faf9e087bab4334391d9897e5897effc9a39440116fc81c2c5e66ba8758b092493c0b07067124bf3784382138f6891033c4f21186d03efba5eaf8905179c168e0ab5eba571e7049fada61217f704a3ee67cc988375d7235a3fda5db1cbd5ea602e5ed8b8feb1fa5a311ad0517bd446e4fd6a2140a513e2c8349f1616288e28a4fc48e689fb176c5fd2fac4ca5d67fc9c121774c16bba802c6a3210e097b4d56c103a382d4201380faf93e3445a7fbff1b63611b2daa00a9e8ce48f7f94c59aef7db6966dc0af68369d8b6c92b770e870e6f77394045b15aa80fc411a5d126b5a813024fbf7ae11cb242ccca168eb18dd324b22e9157a712ed0c273ecae30219c8e51a8106835ab7b4a52550361c85ed627916ff0eba09c2dc2415aeca81e1f2a8cfa9e5b3cf9d88680ff78539e1275daeb78605edde4f7a882f3f2b161002ccb183efe944b5bc3677c98091559505842b2df6f35164e4dfa50852a45972805625b1e91c8541540c65391cbadfe9c999b49b38a053b6e85506518ef53d7144c5eef1dfc86edb7868e3f25dbbe7242e6ee7899c4e423561a3b3eed47fa25b3f53957675d70aafd263bbc5aa06fb89f2b41ff97f5ecea14afb0cbe057b424ed5a3d0a55c9d03e60eacee19867a5231a5abc602369dbf3fee5c3681293e6efd05c15a6021d33e4b68ddb6b589fe176c806d0f1610c401cce0ae023ad4f6fe98fd3c70960ddddc30d60cdfaa0e4fe72321c87079e9b25e84604e96487da945e60f362e5c65edbf7bb19f5994a35f2f384cfee8c568a90bb012656db0d0b2df8803cfbb92bb738d9120533b041cf2a32e36404e5e5b10b5d0bce90f4ab57310e65231a4bfc3094b4716e04bb9713cb68163bb917ed38dd01ae0cf719c3fe6eac7a0ca9d8acb5091a6ffb5fda00da01835dba7cc0efab5269645c8f7602a5c05961d085332f5f3a9dc762517d3cb03961499a714c913be317ba17fac21ef3da0bf0f210a846e2ce244f639e5b389a968d4c3451dc55261a7f33ff62f5f79d4c01950316ffa9e21b49d00f14ca87a06bbd7a80208ae4a1673ea8bbe368cac2040b2debf55cb23831d44870e28abe6f8c94e9848ef586d601a9eab80331ff8169d86f0f37ac6ae1d14cee679a9f8a882aac5efa071bdd210bbc38f02c258b001fe9c9d488ea3e9bdcf205142298c1ca94ed3706d1ddcff3ac0ec8b52469bd1ddac9e32b0010623c8954d4a1dbbb44eca149eb800b4d467e67586f0d411bf6f009e9b49ee2301cc8d693e4a1e21505377e7669c7ded48b4e26798c213f9c31648893da866ff4b7f6880d2c343d571c3057f011fc350d8ea837f1faf0359e78d9811e1bf281dc3cd34260f44a583950cac5714f8584f852a54ef418f052c8431f16eeefaef6522df3d25c5877e26d1a724858f9fec051bbdb381bda7ceb30ed5bf8f11ed997da37ab888759746187fdba71835f2d3b7ac6b9baf504f36fc6c910ec2549ca0ba089f6654a7d0f3ce9edde08055821ffd618d48000e7aafa55fbe38fd5328e6296cc7a25789f1b5465295965e9d1608f3e8474d386eead3446ad824d2ab129b6489e456d59020a4cb301d339594af793dabeca6d8209c3113afaf93f09b58180d016f9aa50a45a4a597cd7983f10cbff7650d908575bb536adcdb589fdd5fb69a09e6b743e676ed66cd1b97c2e7a07e51e1b618ec50d8c1415812cd4e17340020c801c0921c92210611cea72d66f9a06ecfb1a8e9e15f4cb6002b3359a36103c8f6859852b86804fd06469bfd1b1cb173a10e914d88cd3645e2895c7f203680ae6c7dccafcddd1ec73a9faa352aaa1d8a73cc2311c29dcbc9b374bef4c3c5d175036493100000000000000478000000000a03b9d564a60fed2f81f70cd15003debc606fac1d538c2f9921e5378dc635aa3f515538009e7d480017c1ccec9c7281337f3ef8da5bf2a33fc2be40b30f2fb9971a4275e49a0481e9c915892432613d7ce8237c940d07849e3b9930f8e0ddd5727cc75181ac445ab0133bc45375bff3244030ae43dff5cb22d3c060ba041f1399f67534203df215c7398b144edf193c807e054f02151f3eb9b284ed01288beb831fe381f07fa89c918af056d19fc64f0384ec9ddfeb16c8624f1357b6f825e84eb564fb67c8c6f796febae54a912153ad311a385a3809e56f0d37f753595e20f07c696fa42ea7d279abadade7a6fa2405bf9bd88199de9dd43b42453baf309cffceb50ff1ff66e852a37d609184ddd5cf3c61b09f143be00046d7d125df092b0a9b98136521b693a8a125f899ca9beb3a9d5fbad369297a1ee402714580dec59c2a41cf89d8c935e804f72ae7b61660333bef1c5ae4463ca63b07a37a6f20f634ff2eef29ed0c69e2f7eb12dc0aa99bf35af5bcbc94d1cc4967cdb9372bde5a2ec642c6b15e320e713dc8de43fa03e1af22644dfb65afe2aaebbce3674cb66121490946cd1cf11ba4e13600c1a4b03ba4eec608e5a2deec7a2f0a9c992fadc7c863d213a8a46817e64590e0e70ad1584f060150bc06de8aed0b7b1582bd5664b61f7d32dffbefce5375cb42f9c1a080558f5cdc2afc1732ca29e95c374014e68ef66630e8e3f64f2d79830c0d80efd180bf65253a547aa41ea9aae0e721ab999e4d48718ec8452d918706dd6107ae1cba25b02701025ad71f9239a5ae0ef6c515566446b72395b4bd38d1cbe9fcc079696a5c7262eb49e505f9c828f42583f9bc8132115a717fcaf7f8cf7469c3272a601cb8bb5d415c67e5175af6addb6e1b6063bcff9ba1faa848ceb6a66b8024fd9de150eab6af69eaec2d7ef95a2428b03bb7c95b3b410c89d73c70ab10fcc42bd1898fe69b98817b0239b9bdd9647562a6428615f1d366616e1f4359cbc83e6d33afd4c2802043d486f541f417843d1e742794428112bf6640fe5b5182d1175b3646efa87fe83031a51ee6204037125c0c6c60c876f75ce055c06929c07daa7f752352e7b535c238ca9e6f29253c853f31b94069cf7513af77839757e285627e9c2aadbade81e1b7819e61307ed06698969bf20fa09ba8f75111036cce53409bbb5def131146a703075a757ddd44c4dda880ae9146502bd5671443d927cdf51f58f6bd6d14dac96b349d45e0a08f02d3437910abfd3b28e61fd2b123ab95d14948ca8682c0b8c1af4e7387529747d8dd2ff558d8b7b929462936f8eb5f2d43c7956b79dadea38be69b91b4d8e5e3d35826396e3ab834e70d64b73ced68a523143ba1ece0ded03bd35143c8e9b9065b94eb23fdbddd85362d83297f8c4c42d97ec164a5137a5927e50854ad823828d34373c51a463664abc0c3e7ebe044c68eddf0ee46416cc41089aad9d0f0b516312233922d133e6d10d453d633dc26974ae8666947c4b16156bd3a0602a3e1da4181f24c2a0cbc0fbd4bc10deb50d420fc7a95bf2d4546dcdbf6d9d4388415fc8f54efcb8880fd0751094a8e264f861e1754b2b51b22f71454fa1f45f5e75ae65a72f229a8267596a3e8d35608af7f912ed4e2bd9dba401711c413f9168fc5ff15645a45c30fc7e25b087a0e9217ac01ea99e09765c0579bb13056102afd416bc1a5124c206c335ebbd2257dfcef6605432ff8fe4d76ae6524a28c929699c68594e680d385f22a28e9d9a1346ca0a5399e8aebadf13b6fc48bdefa33f97b375dad2987dbeb10e533a453cf47b7d7af92d5f59761ac5eb01a95bc75351ead780ee65917b8e014a21c2f925984546aeb43a1aaad6afcf331b5d0c3f9c8ad31e688a82245e8736b36167301de7d69fb29fcaf6443d59823184d0179ada9e9ae511ac2350835b9b9f9d6e1c49d264d5b9587c9b03a3592303f8eeb7dafdd50fffb87e6ee8f639d4fd51a95550ec539e61188e14ee5e4d9ba5f7a61e2e8ba81b2498800000000000001d4000000000501dce99c9547fc94d05d826637ce13282e85553e947da39399fabc48ef3e96da883b30004f3d04000da562f17e6c2940b8bc99f425ee2e5abb326d5b3e034028555a2de4e6a4bfda70d13b3e5c9938f62b9278e620067f8cf30c4ec8b6bdeb13d1beb035ab863692450e2dfa6304c693c04d20a3b7d0fe173c3e4d627697fbdaaffc31b496a685cf6f18ad519c6c0c0092fc0a95473c0bf584970883365fab48cf93205ddfb108d6941d4c6d9c8c99b12d49edd1f76a6121e72b20190ef6dd9def162e977aab38b30604b6519ac9ff2a84a8c36abf514e1c955919601fe74afc0e33d76e1038fc91b7faf491f89a19c51e320cadd892ead42016080f5823a42207eae61db3d8fcfa9bcf0e53f589f3a39fe1bed96512cc82cef105f4a6c525dc4abc44c70e5c5792ef44d662998f0c3dfe0b1b6ec01b68b606900dd046b749197a14511d589b2bd8e36e0717dc70bc3c644378d3383fee9e1e5e4ae878b7afe0e7e9d2e199add8d10fed39094c257125a873aa04e3373851fdc6550736d13b613fff6ec5ca39d556a2523277740f4476e0034d1fc541f7dffa1ea68eff7dd58980af4dc5241fde9e0dc90c326a71aebb8fc5b30e15e13cfe3280712e78c51df0ea183be0e1592c4238329cbfca827283aea794a36968df93f32ee2447f9a4f214c4b867e04434f381f91941bd898ed8f55ed5b3b9d0d1a4fb3bcb34edfbdc1afb47ad0aab7624a530b04b93c5d9f00b57aaa3f46b1e48f9d42d11408d7e760873202b92e8471890fc0d665642bca315bc5a5fd3a332f9eebdd1da01768a76ec6056f54f6aa1fa05d249c06cfe451e16704f495326a5471728244a9bd5ef191c488f27797fd1f1f45df7146ca837f5d5e88426c957eed2f30f27dca9532433fbb5ea85d82bf980253ea7c979165d301dd1f472b33cfc89b9cc04b39d2b07b9d2036b333dd8970319f7188031911e958b1428583b2ea673aa60201f069ee6b196c998f3c8ae390727521a20dbfdd7e4c2d65f9bddd65f7f94f02125aed7a4127ba470f1c24ce433396ab5ebc06bb7bdde3d18c5c6864dae65df66e99a2d0b45147f40293bac6f0b20f99fdc7165e50d4b39ca03b9d305c42446271ca40ce4b55b4221ab0c09e870fb97745cfe8981e5930f0be8f929204b3b6cb7bde277a96ed51acad025c65a9d5f4d015d83b420e64595c44ee819e0e6acf28548ee1d32a04f9d300201708f1e9bf4b3aeb71dc023a042ecb8aa8852150001abea90316163d17dec974f6cdae32792bae26b1112178a3f84bac0e351e47805475a7f09b00d82e033293f68c0d3f6f5fe1533ecfbcf8c308794cfb25642626ba20a3febee3d14dabebfc801af4cf75fdff5b37adae3cee9ec63fb899751f68bd0f82f00e668b9cc410e91cf7476945599aedd2458b6e63f108a4462b53654e0fb2c88636000094c395001d9ab3fb4eebc1d11f6e712639b2e8ed1c4a784ed51ac6cd47ed94f2ae780c4f25366033ee97ff3fe563e224a618abd3e5c9d253a2a0c753326433aa7d91ccf20089b46976aaae4126c4afea92090814713615514dc464d682a25f4229db0f6dbdedc317084c70ceb5fa42fbbe7023c757b7c1a84b21ed37bdf43de253282fefce885add655eecd16cebd222d7f5e45ef4f20e65f84fa2c530aa06a737b2f98b90e6fcf85f0026ad30eb87c74213a49fe53727d8b977aa27303fdec8d0eaae4b8d3b85bea9ee30c9a794cf5c160283243180b91c7c2f821aaa5602bc91aecdd60c91dcadb847790291c727261359366f1d15ec5f255dc699f24dd981bdb44962cbae4425d2953479a8c873d27e507221062505620a48f3ee62b153fead2d4d81d9d49689b9596e5d738ec1ab06bd7551b06808f53dada566bc1dd9e1e65d258815c58af6eca5a77dc34dd396cec00c5779680a0ea1cee7f3379eae8e1dcc4107f48566d61d019318e063d9cdcb861a90428b256239f42a9022b7f37747b1cea7ea8d4aaa87629cf308c470a772f26cdd2fbd30f1745d40d924c400000000000000d2000000000280ee74ce4aa3fe4a682ec1331be709941742aa9f4a3ed1c9ccfd5e24779f4b6d441d9800279e8200053a226d9bc4a723efc8a3cdb05899ac12dc38ee80878c32af2a74e8a1fefeaa03175f4c0f2c13996b3498c11d137dbbdfc3ad5d6b1462866e48405383760b7f7dbd6b1bf79077d113eb112e7049c5ca94a101ee31de426e1583d7abd5dd1ced104259863ca1c77ddccacccc4c162b36f75485684f915da1e3bcf64771a3e036389062c12844c097b1001aa4e8dc88e4f6e301f583e8c6467a8e08bb7be1513f51d330b58afbe15917f0c69adf8e8b46f5830f2b7e1c28a2d52eb1d01cff116b30c6f899302b1805b81dc4bc49de05ca66019a8e8d3512e2af80eacdea1fed95243ebf696668b4af2d6972cdb450a1a2d53add4b1845857a9165aa53e538205c208295242f1b3cdb756881635a38d0fc22b73295feeef4f5b30c4be5d1328ae35c15a7aae6a6bcce93be28715be6a6270d37b184b8b44004396e2ef615e227f1b4ca07135acd75db37212edffdee5cd377998b26f2d4ec0ea1f52e1881f13abd7d7fa0ea9d3b57425686c7cede9e3b43c5557299b84d2b874b69685428c27ef531c753f2563650b6f494a9df5ea162fb59f46a7fda1a800ac7823c61549b6e06089655d11ac214db1d772c5523c9b4c2df9d9ec8b5b3b3285b62e14ec1c7d7c494d5422d9f1d418a9e29e6c502e97a9ad630d2bc33376c66f20114a0d653fac982b2da9e70a18ac808616fe601386c02f398e9dd641b33fcdcd74c234fcd7274bbad73680af1927ffb8da31149ff0e69eac45f161569a511ec29bc7496982c0c48aad406ba4ec2d4460bf130f6349aba22945c48c3b8881081c810389a95a183df19f90f6f529da186f18ef93f7ee1adac1bf95be23f3331695923c01c0e91f076340557993f3bba8aef14d811fb014d907f8dadb96f748ac8113b2b3d6e29f073b7a49e9d5557d45c1ea009cd89b6b198476d7dccaac06ca9d72ce4cda75442fd0788473df9c6fe78f67fb444bfe2e497315522fb2f0ad6652a2cbabdfcf2a3b363e6d3d3db9f95662eb3ac641e2f28e7c61c1cf5830c81c48c91417cf11d6af0c5fcfd0ef4fc2cce4270a525e36ea4cb14e99014701fc4c61a47aed5bd7aea1a0a76662de679c5d6d9422b7260b5f44a152a0e1290cc49ef82d041c110c0defa866cc6931529a3e3508c257b8be854789b5ae743a23ada6ee971199a8e2da360229d731fe3fb1a8a996796daa5cabc80c2b29f201fb8478f56a8a00a446d472cb8e86bd7e855809bb5c463271c9caea0fd422ddafa9028ceb452f563fed3c7bfde0d02c91e4d10c848ee85ed9fa39258fe97fc197abadab5db46b2875d35f11c7ebc5605fe37bd579d19b3c4912222e5e88168e5b1b3f817788846293f5cf8e38a76f4b0024a81e8fb5a0496d5fea7c7c6311db840c955dc2b9cbdddf044d6b3a23b7904915451f173b84b359d801c483c90c712b67aa4d4ddc65646f664c8573a51fc26caccadaa2eeda61c0425120a331690f75e30196b8f3f6cf181b2dd792ceecd7a150f84768eb9169c758133f2c37c3ebf0f91b334747d6478755a281f59e076bde14f22b4b0f9082bacaa455b8c2853b407dd9fec7470e1079e7eab764680cf93d8c3f763b8db573cc8db3e9f83250b9f193dcd09e04ae04f3179fe22cb21b8cb409847d97bbdcf96d1917c06b588fc757aa70c8a993734de942ebc97cc398fbbc1240956d0f494e0c714613b3357aaf3b4bd70667524ad062d194be5577ed8f2cc2361540fd22626798bb460c38a8e06731a81f81b8ec79963255fca1d39f34de095e0554aa25484d1d740dd59c2dcef8504a1a64f36b6aa6b6a4db843c58578acf11143891b866f2e960d6926864106f16452257c9ec636ec6298cb4eb46949051d74dcba6acb4de284cc8967e40ceebd217794744a243d3dc34b4dfc301b20825efc1d202b02d96970f9ad9f852101f9bba3d8e753f546a55543b14e7984623853b979366e97de9878ba2ea06c9262000000000000006500000000074e7cb4f018db0ab1db290d3baedfe8f32038e99ae1c2a0189ece7d04ffa76594f3a0790013cf2b00038613be7e2dce040baea24ad4f5a2ef0316e9ad56ac610a1fb4d9b9bfc53079c8a7bb0b114c1be65b33c0d0dada17d70986ee36043c48f08d0f567711ef54d1f2f2cbdcea6e7c6bd500750b3b81dd92f6752fccf8f73bd47e6a53f73b18d1c923d14d1129e105abed6e33442575a9e8ed53e66b5553230defb837196fee792adc17684d0aae1e246dbb23ef784abb0153000b866986866971d07f09f3b78973037f24c909156b29bec7a47c0319102a4b3c8f042ef5169a6157652fcf43b4e41c18c9f64dc23a50d052d713c31048a74f9a49dbe16d7c842928c4234d6652b55c9611e2bb20c25a6f77fe96c6d70b8354625fc0ea7645cc9055f8045a8b04e3411a9f5602ac10df248b51110b88e417311e4366055ed793b069d81384a379be3bb09a9280a4304fbc50e822c13e99a8164bf1a14e134eb5ca30a87097380d518d101b4361fe43add3a5aeb519f3312412f0641512516843c638b13f143ee0aead4cd370edaa53c6e31d514142be1fbcd4ad5efd81d38bdc59af9be1ef225b1428ba628642e2dc320b28114d514fa44402758f9d32c0c69a693ef0d72737220c5842048f3bb2dd8ce6834e8f9baf688ebb3a90b76bc31d6575584e3d313342e63c4a13784db66323283edb58b422d8a5c228189da93f4f5d40f8a32a4e5821e4e5fa3f4d8205385194973f4693b3e9c3f6e12b012c16a8cc691326ae997d1915d9235cffdd9ae5130e8d3d2b5e06b0baba2ed12296882d61a0668ccb8046894c898314a0069947fc6b36b37945ebc6141023b14e7e763dce0e7499e54852d01e24bdaf472993d03f178d5d6dd5c8cfdb0fe6c488f4cd3b273b873c93da08bc9189d96b33ced65cbaa468e6de7e4e7ed3239c4b4552bb15369c833352a07c626bd8381a459d5de205b2c8781e9ed97dcb0768fe4d36e4458ff769570e34e52d046d922eb089615d43ea08df87094b4222769128da4f3adfba770ef9416bc0773c4da3d791b48a6fdf94c0392a9d70cc0ffe21763bc3b0251583204f03bf7b4fa160ff76b561c2d745f7ba2d8afce6b05bfc8641fe56c3827a7da3bbe05de36c9a92a7c24f94b91f64d4dec5075b39b5d404af44cb0f77e750be9fda932ac5920fc7d96e86bb39f112ec504a993193862e1c1398bc26779f784b954fecadf80ebbb20fa0f53550549beafae04c892e61ef64453527a2e4d95d4e25a41b110db4030f776d7662413365d3afc2ac7052c6185ef9313a36d91b54543d0244d36cef1455a5ebf1ccf8442a8ab3904f659b86dd0251b0b56f68d0d1278eee22ff0ba0653f5bd78767734502de72d1dcb3489a1b67996db590c7a3f902db1894842fc5a244ea8b86cfb704aa5d1a3b0a7e24c033718d4895f385704bcd8578a6855afe40e3e7dc3795389e0c8c921cbc23625c12ddb99fa9eba99bce9afd96df35214e216f6e7c1ee728f13cae0cea5da15d82b8b3f18dcc8712ce1b7765468afceac97610abc730637b29265026450c935e4ac4b30923bfbc7363aa7f30ef34d4d5be82e5f746573c959d9b87d63292ad78dc8a9304d6f2a30a55765bc3d8a8f363fc6f8c3d65778f15e0dc108ba73b5c8ec5d1b2adb52b60c9839c4af34012123d0f43682c2090e1c77b6e6c9a92617e549bc2cf1cd3041f0339cda1c1b5d6cae3056a1826153548b340f57dddafef0d412167185b1581a65f8dcef66856761e6c906fd09b96ec8aef9069e996e75557297252b6f39c78e4b44688c4abbffe4334cb5eb6b5e23d54060edd5f73864839dd6052068ada682844ab47b7e6f2a1f022a4c778c38f28d58510c6bf047387bd7cadc70ab1dfe9e37c7a50b53a8e603b1e27120e9a6b7e22f9ca6561965c97f1ca675287e69fc1c23fec6d160a5e2db90e066b6f614da266df9d3a7a47b12235d0be783364be6a25fcddd1ec73a9faa352aaa1d8a73cc2311c29dcbc9b374bef4c3c5d1750364931000000000000004f0000000000a03b9d66e318c6046f40b6b7d029d8248a7f0268ec15808698aee226da851551c317a50009e80b800194aaef95c75af7f46c4d6233cbea0a85ad7f487f6581aa181d4e4760369c232d48fbb294f4fd13322f47914e111f90c62e196f77d7b0901eb69f632c582012c086192fe4e7a555b30d1b45d19cda55d6fc223bff191f3c6266407b5a64eb55ddc987f188a3b559d84d17da0a5022ff5f60572ac58917fd85765753f0f076da42b9aca114f8cddc162d853fd415fc0090e3df87b945ce8226f3ac24dc361a7f9aae541d61dc10144bc705ded474b7ed65bbfec89c701f54014dbbf231e1196189ff18ce9d8249b41a88e29b9bd3872d66d739025ed3e9c64b0366dbd5360ed6670b118c0e72cb22f74852c071bda46ae4bddc7a2e0c8f2dd84c3caeded266bdb0326d2c19d11071cada4ac376fdfefc05d7e0ce875ac4e48fec1567251db2e718359a8f573cb621c76944d24a56f806fc99397dc6bd51dae76fe34a7ade489a677ccf8962ff0f99388c6f7f235c8030f45ee4d79f89cb83465e1681c17d2d1490c289132c935ee65b813c85c20d4f4b90e1a4c586ae9dad79c220e1e193c885ecf59808b66ed9d5843d3c9d7c412368b2844a14e878a9b789c4814bc62e01e5f4751d081bae4196e0631a1eee982ca44b05f5118c45d08d0a8443d45d226e86619d90cdd5663a967d9b79c5d4db08fbf8268d1e085261c12caa03b360c863774777c60a0f11f37e7e1de5196592dc7627e93d4cd7063f362e0688fb283df04a900693a37913a4b9013638088cb9d0c93e741e68fd8280485b2516b20d28ba7f1703df27eef41bddcb5bbae50935923a33a1a2daa43f4461cc2ebf498f41ca47dd6fe610986721ffdc2cedf2d2438f606b3688e818d0c4ad3cd92a8810dbbea2f670aff991634a0013bf14176fa82fc7c76c813676b4300b198cb8e372b6b865f8b4ae4e8c0b86a5c1424e3b344d9c32abfba0c97795078143e1424bf57df3b9f7ba0fb6da4cda3fc40d6cec4fd50b14dbb3b0c760f2d22237a0e4b2eace03b8f90a81a88818cca73d7ff04d83bb819e5065d18857663aa016ce2f33f5d43b476368342c91fa63f5171d035f1dd62069abf18a8aa8e5fa3209ac32c9ff2e96e131caafab440708543e5b661b09c8146344a207a847bb881fb6451fcc4ee2c11276636d8686c0feb2c3a9980a178645bac8cc100e01b06e136c2890e3fbf8d885f70a0b4c9cea35c673a86cb8b032155237b46d8210b927692226c01e7d6f3b9d85de09c75b96fce3742688f56c1a3347774a7f800c37bef0de2715898d5fb9c66aea465d8013324f07df50330f92729abf275f44fa2e77b22593a610336525433dc75b0222074b4565dc86228b49f7506068916f4c7603b907a0376158fe7ee6debf0f6e589c38a298a21f7557c605119a087499cceebf6c111d25f3d21632bfcdb5e2232f5a955c4551ec83572aacd5be99a969ccd6f0b4a74c1bf88bd4425a18d2f470f711e55022cc2c78997e715cb9b4c5d70994d89494380c9e2ef0b7e1da3cc41e54f7560a712ea307a488666aab16a72e8f1ec29a3fe808ad656ff754ce5f42d403216e45344f60ac3f93723a59d253a909c579a8b0a2e550bad3b69535dee2f1c1e4caedc941e5b1f94d7dd30c94f62c830eef02657253073a11c18834dd7a6d068b3bcd72cbfeb182daee78619007de137417928ebfb185fb2a274f5bbb60d097f69855c2a616c0d59881ac22319a165116dce042063d88f35bb68969d9aa36ebdc1058d31495672d40c175c02d64d9ddb59867645780897a611ae7f3ee41d0c2f4a32f52e25bf916e46848f6e3f5ff31a2971bc8a0b664590be84b89d3bfcf12f85454521b0d0d6764fe1f23cd76f148cf83eecda18aaf4a4a6d5852c95010a6654b834a99d7a41fd1430a8cb42b3607779fbe6f564f1470d671246f1cb4e500c09ba129f7962de4c0ee7e74474262007e6ee8f639d4fd51a95550ec539e61188e14ee5e4d9ba5f7a61e2e8ba81b24988000000000000027c000000000501dcea4f082e6849eab4cd7c613679487d304c8fe37050f91e02e18e1f7258d419b11c004f405c000ced5135f6e6adc9f9e8816e3f8c8274ba23df4fdb06594dc244d872869ea3b4b0d0811a1e67aba4c9d9534b4e489a3228cdb4d53b1dcec645d2019a969e3eb8406346a9f8e9e844e30b20f118298bcf8affbff4b6bddd9c4c8d8d1535e8d6829d8ca0c175bcf7ed6c7ef99947576837ba8e44b72b796d75613805c11dbf9fdf66bc8a68ed9108d29b97114ab8f568cd7fd6b0c7d5dbfce90c5093b9b66cdf5c7c8416ff92e1a18e75cadacb088c865187cdbb8e0cdb51f8e3a8770067d3e9696a18d14dabb1bbb2d9c05b06a275c6ae2e43ea9ff28188a7f137b822063e1430dbcde1580162666cceb74140c1c8ca9e5a4bb7f3f4cf42395ca2ef6da56b82ddb793c202d39355ad290d793d3c6ee37c22b632b8c274223a6c7b2a100de769db3987c4c40fe21fed262fa87c1b7d0130fcee4e43eddb3b80c57052c403213168dc6fb26f3d9030a130eff7daf7eb726a3a80b5ec3db460d913cbeb0bfe0f57ff9441a2fe135204734dfbddfd9b8797999d41ce6eaa6d2ddda3ff1471727784095e17149f6a93ca4c09ff35c9b4793be12cead832f33ed4e40f91fef0874385afcf38fd6935d7bb0de2e2fb92a3fd3009289f00ff9e586aa9d6f2646c51e0779b60831bae998d9c4d3ce9ab77b1c30bf2872cce5aa200bda8ac28dbb68004a6fd482a8e4fae9d5b928e1edf8d4691ae0cc4997feb23344df979fbdf20b615f81eb044828b0249a11d6703a1e632c42b8484a5ad026d3adbce3d979a799aadea0d13f19b2374cd42a866b838be0774778e6683edd99130c7ed3f153f9378a642de496f7ba3a3b4258fd55fee1b7f7db979df156af4ccf9cc2350f440b3243e50a0ba00b09e89d8fcadcf8b2622c635686a4338947f25827fe6355c8bfa732f7b98327b9b467a58e02f8cb84447f9b734a383d21e3e8c4ea15e4be86761dc8ba6640328c59604ec6549b56c0cb448ba8afed4e5ca9c1ee37857830c9a65cb2bbe1c6e68dfded8f0f8a1ac731e22496d4d5e145b836277d421b327ec83a22a7e805aa8c3e8d197bf66d4c4580693195e3468d5babece48c6a60b3b75ec26f20c336627fa72cd6ebf51a3c84ea1814bd23db3251f9c8c143983ca53e2bade6c528b2703e732373e1b189d6b9a4386bc0387e6a1a8ed12d0c286881b6e16a3040ddd6db257ddbdb9158cc8001a1f63a434556a9f13f217f5b6c29893e5973d914273ffb52017f0e002cf3f602ab21167172b492fa47ed80482c9b703c40f09dd8a69cf2e3ebb095013bad8cd9234174b32944dc02d688da6bcc224ff7023eceeb5d65a2b3c345c3462c09798d5b21ee9a85b5cf38c9c1b66f0d03f58e8c323dcdcedacf6ec47c3ab287375f524f4edbd263abcec55ba1cc43b0506b5394247ff83af0a640aea5e138cfe7210e44368983af6d815f04db59735c47c8b603a1a8e26520b037256e7054439715c28f00623cd1699f6bb975c1d4258860b4a9e037e02cbf9392418d168bbbb2e76fa39109fe691cf49af818344ad76f1c460b3831521e32004992a2527f4e13e7d3d8fbba7ebbdccce18c9f26cdadc205b421b8837a00c3ae32ca923d32c98478b89a701bbe3078442b6dc7e15b182bdb5fd6281ede05f642fbfbb1a4dce9b70707346dfde871e9fb256c706fa162afc782aa2eb10f3339eb05e999701ac7bb0f26e0c578a7e30d89cab9f7ac2b0e0c3868983408c45f4765edddac96e6ffe3a50f993358dd72770ba44bfd8d97d965cf109d63376e08399bf3a8336d32fb18227fc9e2919a6109aa03ffe7f1f6ff044740e1629e3c78f846f8e0bf843472c143b027bd36cbce575eaafcc41e2fa5611a3625fbae47a5c035a7d27fd483ae5a649e752112dfcf7e38ca719c0bac094babce20b1605d1cb3f6dcb70f59eaab8ae5e3b3a25c8db0cebcbbb4f3543f37747b1cea7ea8d4aaa87629cf308c470a772f26cdd2fbd30f1745d40d924c4000000000000010a0000000005f5f168fa12cf6aaf21700eb1d6ae84fb75afbf10b0110a80e1b1107b8e26d47629f34600279f2c00052a9b5176f20ce77ca96e629568a6efc7ed1bf663c369ef88fb57f427be44d03da541223ecb31d71a093908a74a954173225125992942ea553565562711f3de87fbf4e0f8d5ffe692313f91232dc7b200f386b3d095e09654acef0c82f098f412b30346912aa778eba7172ae00a346d548f1d43dd6a92612206f32dd3f48d5e75787e98c15c273ed6f1e651eb12ce06217b80e93838b73fdd3482b8482434dec1202729049f56754e7ba6750b3378441c23ce852a1f1d1e1ae8099bc90f7664bb8f570b4cbcbaf9b6f6a93f3e057db0f68478ed3d4c42d0e297cfcaed1aaf1579bdf7a4e57779537b196b76291d5ef644391c1c50ea6d10a22c24a6d0eca264635dddfcb7f1d7072e7cf57d6a0ec92a7cb738427d1242e7fe7506be1f309c0584a0e3b90cb4070ca10df4abf91117430dfecf0938b93b0c3180d2d2247ae0f58cec7ff83b28dbe053cd0273ee97605be73e02d8c8a9c5be2110497fbab754135079ea8b1700da94155b1fdf4f12aac52001a1b1a551bffc5614d93b8dfab94ec3e1aab533f8b8f5773e3242051fa261036023277d7639cd8e2d3bba334e1622668a61372fa9629dc2c1075efcfc2f32e9df419556db5ad00b8743e0617ea031c7de2d022af8172d7281c0f2c78b9f90be68bb3696a1aa2c69b9a96468492f8c0f4093d54adfb02095573fe3686bce4de8800e725cc85940fc80e899e8c936244c47e218489b00332cdf20537280be8447e8ca613d5b95feeb9047a7178010fe3b7ce389955eabd7be55ae8c05d6fb06e2915e548a26f8e7d0b93ad0be46c0600401b763020afcfb44dcbe962fa37e7cead7fed5604956af87649fdf68e1301c1189bf58a59ff016e1665e2833ac074a2df2b4a7af04c151cdb67424eab1adfb33839e9f5426a47b24cdaaa1fa7d65887e811a9366007520c383163e01cb9a617dc3e8419d348d4d37b149c86fc82f1c686c71622fbbd9b5b6ced7b97d75c5768e43cabb584b93c723046b0b0a40d5c81b7527fe8e166e4651db40a52ea697e5bf5d78519eef061267db4c0247628b2cfb28f3c4191a3eabb14bb10f3883c09924f27a81cde81f27d4c1c142e2d24a23ad5044a6b1f86cd2ececa0d6ed5883f8d489b67b93ce5d7de926a8dd0347afeed38eb0250af8a9d549ce55504f60021288c600e797f766b96c6a635f6848828d0834483ebace3e3a90d0ab5f4849fd88b6a1ba04586253877a36ab79c14a3bd313a87ec75398de5cc5fdeeab17d8787ee552f3f7646ef15ca8b07cd167d324bf018c163df7d910f359fdd45ac184bad4383cefe961bc669b041477ccd714ea428cef73785ac2d9c2e02f39d6b5a6347326143e8f7a33b584842a8c7ef699b550c82cc8ee96a93b6c12ee9d37c05d27bb138df7f00518285db875a6da8738c3cf64186badc844e541bc57efc993f45ead325a5b0abac30c42502c1ead7e9de729e1f44c1c6d8738123e20de0153c4378daa0287634f11991b348ea03e160fb2cd0dbfd7202cc6941a8ccacbaca176bc4b8c76079ffc9b774c5119504ab852bc8082c1e3cbc81c2894c7c0b5869c049ae81cb23e977e62f725d6f176cc64296ac19bb2fc244f3a0e4cc62db140384144b4eb088c08bbddb8376ac801e66e3edeccd8f483c178fc878f7173b4863754bad647188b56e3df27bd26f395a534cf5276351013567885f6fabbf8d10757b1768683c1083fe2b09177b965ab347f3e609f87221b6b522bf8b1a7ea3799bb1f8ab74212d0a3a832b8ab721e4b5ca60d9353588c8c73d764f1c5f0e4437f4d1fb3828c1fa93e2117193e061dcbf8783dcf0992d98d00edfa20ea3a874135eb2832a5a00e3abf3d812cd3cc65571b1cdc690625ec9bb6935136ff38290673cc132c2692af48828bab18e8516914fd4f7ff84b6c7f884b25f9bba3d8e753f546a55543b14e7984623853b979366e97de9878ba2ea06c9262000000000000009100000000006ad79ec1d9b5ab5362b44b5e2a46a69fdbec29625c11f0daad2dfd7dfcb57298f5803c0013cfc200031936049dfe37de5cd8ba76624f67aa9a0100e563277484829150b7f85a9c21096b7e9f5f7eefb4542d172870f76ebdeecaa16d63edcaac1edcb600afb720083d69ecdd22a6779348c0d0270d145f91b6879a4c459125ccd378248a85e1a6f5ab19501250cdf859456b5d166fbb68a4f2624c540638852f74dc0fe26cf2082e84cf8f38770d0d4f341b9823afc8a8fb1ee361e9e9a7d928f5f78a71284b418c543d7c95fce0bd3eaa4be93fc2c06c0dcd3d47c159cc7d82ba01f23fca68961309731831f062d97d19a83ca25294ba199a024484207a189a8e5f6addba9f87ca6f4ce366d909db0e5bb48ec3683d899019ed29c6d48f89a04a245adff4266cfc66b364b01fa4c219d3c9125bca917c7bd4536a6bbabff51e8150a8f3901e166a76e63092dd65dd6eaa4eae335b4bd7abbb1d072aa0f13466b60604a8f7e0046c3bf123c5ef8c7b9dfc2fc56e1a384596413264f25841b731063033b92b146ef7596f6c85b0c1b41f87c682cf8200ffa4a04b1d8686ff357104c507925a24cc0bd0b34fff883a806c8b5fa7cdbc7df29faedc2a1b87c194ad6d5d248f134bfdc4f092feaf47afc43234114a1401045a6d550acb82278542886840058889cf3562822eba983ae98fcd48916d579d94a67881b76de9d308ec811fb177264daedf8f8f2e0135ed39a54e299636563eeb34e2b84ef11a53cc0cf17c6306765fe98a82fde7265458eb9679c7bc934602361139ec8053ed24d350ad0bcb85ed5092ed9ef9cac7158c98eb37db6ffa7cdfc166ede117bc2f4d58a5e553e52d2c211ac59bc8c36bc54148e71a28b916a3938fefb62742009ae20855806cc87d85d814dfe2548686f0fc1ab90ca47d31321bf866d2c8cd0b6cc7e5f2ad6bd9ebf6ed3c3390d25c21f78a06e3131e927668b5ed9396488e479378c244dbaa8db1ff30e61f09de1bda9b1f50fb434af823545f17e16e1daee164a75e4c748a04945c6c52e926c3414485184986e168884f9aafd38533850a1e133d051e33ddab6da662ef9eb2e3526ac41a49695160be9cceb0ff2e2bbf605851ad1c1dc04f7816238ff7da6839721af26545ef7e47b549c6b40c27bd614c4515e3ac6e9b37dd3dfea4fb3f7f6745200681e933d016600c1d80af4168d2f83829a780017fab68e1d6cc4b7662b1e4e04f9257db48ce0526cd62c302b0bfac9ac81707fff48783875a499c2704328c83ed59de68f1600094e841cb0f0223bb4b492e1fc5fd5859e1b7e990a4751c53d5227bd86dda942f1ad46caa6244d4931834867c7d65547b026c0776e0ce8bf4354fcd8913537e888dc87b6866c4b8823ac5c6c142e272b3b8f91de448182a338c6a081d920262303f464b559154c0095a6e9852992c9348caa35903c6331e83661966e9d8f2d9f5517c4e99c1cdb92a0c42808f69285aadce7c0365f2a38710719ea49c281a1ac72ba162bbb93ded733db6b6a74c51c9a35e8e6762d845c827db9a4a52f6f5823f274f22baae3cc5ea30e5c07eb4b9ec9dde893d8b79dd35aad45ae7c3f653d3040fe9e3962b8183cff7e204eac31b82f61eb3777544fce8c4baec6b2d8b2a33733928171d9b39bf03d8b8732ed247fe802d3ac28cbfa585bc9ca7623cfb48943b783a00c753bd63c2484c75b2d142d841f50bb6497597e0438b2b0c43075d0e57bba4697ffc44d0583603328103ee387d6e19c4544b9d06d43ae968993eb83eaf3fedbea6fdec601634fc031781b222dedcd4893df02696fd00f0562d43676cf15ce94955de6d600922a1b49e509e5f2d350838c0e01f0ca8f9b1463a8ce3a22e445bcc910b21e20849d637d1799c155c028812d32b5a5cc35fc5bba78a925b2320db69614b51ac0dee930103b312c2249d443637c99f4e6f9cce31d935713bcd2788862d2ea9d144bc0756fcddd1ec73a9faa352aaa1d8a73cc2311c29dcbc9b374bef4c3c5d175036493100000000000000388000000000a03b9d3392a8ff929a0bb04cc6f9c26505d0aaa7d28fb472733f57891de7d2db5107660009e7a080019d792ca6fae1d7d0ee7ec503947a56792b105520773dbe53b919b368a6a02c6e5631230dbc72a2c42e10d56940d2c8e5d73ddb7a1917dc66973094c54e7beff5b75bc1d3c668a8a5d56c8cc35d51502dec02ce8e77d9dd98d3335a84a71b88f43667e657860b433caf14528a250a752c8892facb9d84e1bf9047d35d474ea3afb06d0aa8812f7765a525dec9633ea9e2cab4c5329790de66ad4f9560a42ec08e86bc61ce1072e274463e24fa5f94ca21753c27e872e8cac8a544681e9e1ac9405a356463bd9ac505460c11307af840c53bc46e54816e0d0f6062858b98cbfdb5f0f3bc69fad36d9e3274e9a99b784cf36d2a86828f72c04ab38cc19452a698a23ba32c20577868b26890e520db8a907c0a0656976439244f8a1c17c50e3e561b473de7e0110a00d0a65767233ca79ccff3219685056edb875fe586774e8fc49d50047afbf563f77489838bf318f92eae64613cbcace67eef5179951002482048b9c8db00b09861f9911147e30958bd55c74b4f50df9c168b3f39261bea561bcf7d05628c673a83c25d7bd423115420a3f2d0ce1dec76145b03919f51ec9255b7b6d00f5656ab8bafc4ca9ca0aba75c9d26c0fc9f8d7c148313fa73642137e5a808090cb781809742c90d8a1ca1b53baa0830e1b9ce25eedc7ae3d0c639b97070f06d0789f4d81a9b1c7375233c59a505bc39de10bfca357caa18d45a320eedbc31d3e3f965c11b98c34b3b324f66a572aa93b2c1c86e244e8d71c5326272f9bf5f5b160e78fe558544ec1134846c06fac202d9aee94624d1583cc20ee71bf26fb9a42c2b2ae3ad0aa4518f1781e65f01923de1b107ec13dc171fe4a39f13066273c31ec8d1032bc29ab4bbe85f6acffadb57df54d9eb59613165460dd20050e169a7341b12c0cf6a1fe6eb56a2583a659eca6eace713736a3587a6201d9470085b1b0260cc6a08ac590f4a2cba169ac7a02853947c60607e5e1142323f199f8127cfa59728cfcf18700e1df0b1c916c136742f3a374232e470da31d4df4b0c20856da8d331e05a3194baa668d560bef875fa223685bd7790495d3789ace5d1e03199f8d6109aa710387962e5b19c33191af3333dafc2d520824d467dee8ece4b5c5b9a7412721de2232d6a3ed9684a31f5a3b6ecc6e8f89a6bb4e10067ddaec88e81bfc9b11a6919bdd8005e087227c938bc810d3bd03545d1af15821b983c838a3922eac4603985f498e3c8fc16d64d01b9c434c27373ec3651e8acee2cd7c2e729dcfa9a21d82b3d166e00b7d70c8cf0af8fe1d3b117a6b05151504dbb4a2d7dfa6837ace1af5bc791c0e39b5452f3e921170cef04e166024a4896f009759043b543eefeaafec6a83b67eb46600c5699f9a040c80c0b9dc9d3a013bc0cf26c10a77aa6f87330f7f9ad9e274272706b61582ae8b44c90ba6bf17aae6a4c7af86b114d95ec7bc034e8f30a618858d9e5e53b4658272e1730bd09eb58ba616aa10299632e4efdb6cef47357f68219907fd8203e8e6bc0629ab1fa877e41fc81bcfdf36971cb7398d410cbd8ea90d3f1b36e925b9cfb86916a70aca8b19cafb1dc66ec8328b785020faf27371912385d863b4a6d1c6c1559b0872d8d1f536bfa5186d36bf431e0153b8006b6a553004e54d8b1169d70db00cb277c0ba7e24019247192b7a580d598bdd373c75432b5d23f72ee714f29e4058be420ea6b251523557f4bf783a914f3662f677097a3e87668781a0f85b135c7792d776e71e6011789ca304903c5aec9d320183e736d2e27086522f4d259f2205b1723742985a7d50d6e77c1c976ed81c03431082f5dcc8511958d3eee2459f2e2b328be5b661d98b9e6c8dafecefdd3084e699dc35e82b3da9e6d7a0c1d330b0d578e156908e119710044613b47eed575d51ee0bb96dfbb215f0846ec8000007d00000000e304d84a000000003baa721b47a4201ea4a2993722e153d60103bdd783f4558cd53b1f2abb6e48295b2d697101ce23e0f77bca06c8a809f08bb78f4cc05f03e47b1f870cc1e1217fa002db9fbbfcddd1ec73a9faa352aaa1d8a73cc2311c29dcbc9b374bef4c3c5d175036493125a594de46a26da92286b3b08f546bc33c91eeda9d73b99a6e324ae161ba40f92e6875d473d97a12137944771b9c3b6f56945e69bcc4ea0a82e5bc8a18dd1ca30006c697e067c49745b06dff9be11947835bddf6352ee78bd66f5659c5ca2ef4af92a333510c15cecbb955f515cdbbb7f397a6f4159ab447947b803f7b9370a6e4e63612caedecae818e29c8f4c041d51a808012b3d00007be0143eafaaad1c97f7d22d47b16a187940a10a6805b0d86d554a10e09074f8ebd2a750776614d5c40380b814a2749731503b2c52afb3e079e39b6c9c00ca8ffa762173f729914a1117a9d6a4c69cdc03e2eb94cf7655ac18f17934c4e3fee40b065ca3273cc97061be2de650d4dd0230db97ee5dc67d214778d0ff3a3be623882c8ce5cb82debb602113cf914c5a85f3ec50a82dfec1112264706133fa157d2dc21dff376a7fbde5faeb84e913e377ca657ea9e829224484efbba9357e2b74e134f4ab2a261b2c6cbe3045d9a5f4ee62bb6ca19c0d506978574113484fbbef63e53c22424fec40a67f7f383b84853fb25400939bd126303068c77056bb6f6a13363afbb520952c5b1cba3a28f25a31fb1ca17f918b6c3e6af764c26409cbc92433cb04b1272a0805d45b62964a32eb167c8b694401f90cafbe6031e2f491429ee23d03b40846787164290f69d8283f650d311756f4709ec781d1a5995edd75c66d0c5af4b2c2d4b02557bcfe5c75051f7e874b780d05ac503e6df1c6762f6005af74252e21b46fdd3b68afef81eb67fa2d90cd48a5d2e1f8a7df0c3d4ef1c9403159aed79ca18e6d4c0c5683f2a1a2e66268cc76d40f74fdc28c2ab76e8dd963101698ab99a48759b62a1a4c3496c9a1f8ecb2579b76a5704e4322a5514a941ce62c0827f39c276f80c6d9aefcb2aaa2f8fa08ab6fe6d08a392b8eb6f1aab5665d401a4c76b7dd5f2e523d2cfee0a386755c9a336b21c7c55d8e24e4b77f474358590ee02e0916e834e2d46b892c4de0790e593038e91288028a9c6312764bb63ee1dfa82370f750abe0253be8804fd9c7f3452b963942781e8cdec412f936a23aa7bb85ec0b009471eecc0331d8eead9dd1505549448abc196ce041f27d6cb932d2d393ea584ae2ef1b03a86ee4424dff531c4f0618afabc8f732fa9a009bbe1557fc35a305f4d5e056a54eefac96cac8e1c1011b457814f6f0996360b9921f36636d88f121dd4a7c21c0a060d1422293fb569c0625fc65faee5bb387d58542150cfb168ae393a85d500000000000000e300093e6ee8f639d4fd51a95550ec539e61188e14ee5e4d9ba5f7a61e2e8ba81b249880000000000ae025a600000000000880083fff8ff99c311947ec9f00142b1827d3472e83790651d05ed77f12a1d7dac2ca8011d48840ba78e54cfd499eef8caa77880ef8e22eaa573a461e00fd3922c57b32561d5bc64840ff453844a4265b27da62f053d203631e50e0b0a55a8789244d4a37f885ca342414ab8001400e00003fffffffffc00821b4e4b92c88209f780afc06227d1f6e3c2556e892a01fb41f3428440853822a400e40003fffffffffa00402fdef1026976762d696e2d7366a720961e35b3d1d396cd47c02330f2d2667b9c00800001fffffffffc7600418e30dd3ba811b6944d4569444c6e4aa97756690438353e58d4bd92b2801fada2007c0001fffffffffc7801033237bdd1142d6f84a64cad08ce01e5917fdf560aa234a42e22158d5761361f0801d00007fffffffff20041de1226850e76b0f8e53077ef7bed937964749e9163e08ac6ef40451e221b9e370000fffffffffe3bf9bba3d8e753f546a55543b14e7984623853b979366e97de9878ba2ea06c92620000000081c481000000000080fcddd1ec73a9faa352aaa1d8a73cc2311c29dcbc9b374bef4c3c5d1750364931000000000031b48d4007ad0d8000000000001100103dc390113060e4eac0b0f46a8723e84c3d412fcca2d9574011a493072c589ab0210f80000000000011001052f08c3d105538556eaf8da2f8f83daec2e24a0e9d1c1ec8d7b0967f525b2c3405290000000000001100102d4e1815c9e95e31d32f4d3d120dbb6565382fba5eb3bd33c544a3bdcb8b520285290000000000001100104dc9a178c3a0ed42e98d35f13e50c07b474846da45f112bc12d8ed2c2a9fa70a852900000000000011001050a5d45e262e40c3b9db2a1eb73e05696f72d91b7c44a61a3fa4f2c8d4c3284985290000000000001100105d5e0cd2c86f8517d47d212ec99e6d12757ae2478b89fabcf4d49ddf0fc5674b05290000000000001100107a6fbeff81599abd81f77342b35613e46268863bad6e6822dfc24da3e425866605290000000000001100107a6fbeff81599abd81f77342b35613e46268863bad6e6822dfc24da3e425866605290000000000001100107a6fbeff81599abd81f77342b35613e46268863bad6e6822dfc24da3e425866605290000000000001100107a6fbeff81599abd81f77342b35613e46268863bad6e6822dfc24da3e425866609308000000000001100106478f8d43253e69ce3f89b7c6cf5e3bb4111a6780cd7a68efc115eb147996281296180000000000011001039e3d9f40769dc976b0852b07c71d14d550cc559825f6ad723cc507757911d6fec6f0080000000001100107b67450c72deaeeb5bc928fcf2499785f684a4735999eea427c0e92314d36e0242460f0000000000110010619bdc49a86a14d1893939f896ff1d3288573a2cfb0f82b00a5fc204e962afea779c3a00000000000b000a44a85599848715615619a91384af87ce90fd6ae182002418228110805dd561ecdebbb717f5739c92fe033789ea329fce0c94dcf6cfc0342e7a0becf58110124b7500f67b3db23a8e32c2916796229d3c48d219249d0410f091096bb3640c80a39822011000f25a7f6f695dfd6ffb6fba4911b14dd59bb0d4b6b970504b15b67ea6d538ff011035b8a227cf97f683cb4cd2f54687f4e3e1882368fa19fbb98d8c15c2736be55400a3a9108174f1ca99fa933ddf1954ef101df1c45d54ae748c3c01fa72458af664ac3ab78c9081fe8a7089484cb64fb4c5e0a7a406c63ca1c1614ab50f12489a946ff10b94684829570eb83f90403b4080000000004061bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c3400000002c41000000432307800000000005ea4514e9f3971721f43554b27d9c7ee5feba9d5fd57b21c0d1cc11008808a02f63fcc4043b5c38be0fea5578a33b8718ae239513a9c8b1fc91ac8fc1c2c08812947ca9fa004623b87ca8ba568337ee12c849dd84f126b5d6adfd9c7d23af1e40401358c840db76008893ef27ce38f74d1a458d73512ef9c6de60ffed31d55960f9ed29534599c0ac412c9d4840e5882e33e559c7b0d4a85c2ed7cf024c8a4785d6d5518608faf36c0577b4c3bf1a2b0000000000000003405e8080000000004061bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c00000000000000000006d4680000000000008800830cdee24d4350a68c49c9cfc4b7f8e99442b9d167d87c158052fe10274b157f50140120c1140884025ff36902c8a45ad7586eb7b7d84999630ad649f5400b30b0f97e229e2e97d16c088129d2f3e209f62532ac54ff0147ce7638abad98058a9a815e5fdc50161e984ccc051cc1100880412bb6354bf94388c4ea6915981e0e28336b1d7f70debf960710579e8836a5100880d346b1a7193cd15c4946fa331ad4045bc9914e04c99fee6be8f8c629e7d3cc90040215daa4535b119a8bdf23c80a985438c60e901f0177e3157a1d8eb19c840f586852f0d080203f185e7c724fc8459f93e1625847c3cb749d59256e316987f5f20804821d91d549f0840a3ae6e11593fe15fe5cefd785cfe00e9fa037a4434ef146a71192e8657b1ad89d4ab99ea453fb7e4ecf0df30f551c97e19f303bd7b09c6d763e22b1a1a30b3c4c0005ec080000000004061bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c0400000000000000000674780000000000008800830cdee24d4350a68c49c9cfc4b7f8e99442b9d167d87c158052fe10274b157f50140120c114088402a900fe88fa08de685ae85ad11b6f7b9693e1e6e9a42b4fd3c43711177ca0cae8088002947f6f6da772f554609d0c0ff3b39835c672db564fa20bf6c064094eeeaaac0520c11408840278395e21ae07f50567d9c9ca22475e6480ec8f0d47a4c4d2ae9116ca2110f4d40881eddfea9089c49adf421c5131d7d39722be76dc16cde01b6f4b4fd0ce599b4c88040215daa4535b119a8bdf23c80a985438c60e901f0177e3157a1d8eb19c840f586852f0d080203f185e7c724fc8459f93e1625847c3cb749d59256e316987f5f20804821d91d549f0840a3ae6e11593fe15fe5cefd785cfe00e9fa037a4434ef146a71192e8657b1ad89d4ab99ea45137249d4e1ce371b604c416f4cbce9fbd08ff60c622b1a1a30b3c4c0005ec080000000004061bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c0800000000000000000595440000000000008800830cdee24d4350a68c49c9cfc4b7f8e99442b9d167d87c158052fe10274b157f50140120c114088402a6b5976b6979540225c5cef298404d684c02225739117ef24351035501af585408819135d79729dd3bdad567febc420ea697689430f21d99b65b6bcd6d203bc3195c0520c114088403ed1399bf0a5367b38b0cd194e60887b73f615c8234ca8db7b6ae641b402cd5b408802317f3e9ed07e185b55fb4026a1852b298dbb6b851e6d845380d01ee9bf3ba9c040215daa4535b119a8bdf23c80a985438c60e901f0177e3157a1d8eb19c840f586852f0d080203f185e7c724fc8459f93e1625847c3cb749d59256e316987f5f20804821d91d549f0840a3ae6e11593fe15fe5cefd785cfe00e9fa037a4434ef146a71192e8657b1ad89d4ab99ea45074905f59500c73ed3ba65f12293dff7f8f2d28ce22b1a1a05f404c0005e4080000000004061bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c0c00000000000000000595440000000000008800830cdee24d4350a68c49c9cfc4b7f8e99442b9d167d87c158052fe10274b157f5014011cc11008818e9a0e700c7f554d35e01ea3ffe3283e69cb62aa75e3b84135bb38ad05eb1738088064f66d4869100474c74aee16f9410c46710a9ebb8f67bda2dce230f270e9e014051cc11008802793ce595b721c4a18efff8ab37e9348917d97ce288c3740d2c89b70b97f5f38088127004239130e674686d9eadbd8159bd24761a0901ab1280ea453e688465b5ae8040215daa4535b119a8bdf23c80a985438c60e901f0177e3157a1d8eb19c840f586852f0d080203f185e7c724fc8459f93e1625847c3cb749d59256e316987f5f20804821d91d549f0840a3ae6e11593fe15fe5cefd785cfe00e9fa037a4434ef146a71192e8657b1ad89d4ab99ea4513462c2d43a4c8de8ad2dbf11712f04bb42408c2222b1a1a05f404c0005e4080000000004061bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c1000000000000000000595440000000000008800830cdee24d4350a68c49c9cfc4b7f8e99442b9d167d87c158052fe10274b157f5014011cc11008817769132762abcde9bd7db265c4f4a8005a0bc964ea58332bd708c167348005ac08802646c26f337f21e46214994292fe9ed2b240a0404d898c6d7b0dd909c2104e38051cc1100881fabd0d6522e78132d9547e5fb97616b21e78834feb8fd72682203bd14b4cab240881ba717242630f8efbc0434c5095e208f1fed0ed0ff82800db5207c62805a2c83c040215daa4535b119a8bdf23c80a985438c60e901f0177e3157a1d8eb19c840f586852f0d080203f185e7c724fc8459f93e1625847c3cb749d59256e316987f5f20804821d91d549f0840a3ae6e11593fe15fe5cefd785cfe00e9fa037a4434ef146a71192e8657b1ad89d4ab99ea45373587d2ac3f83e8a316be72672e1238db7263b7a22b1a1a1633c4c0005e8080000000004061bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c1400000000000000000595440000000000008800830cdee24d4350a68c49c9cfc4b7f8e99442b9d167d87c158052fe10274b157f50140120c114088403b76a93dd9ef1c3f59cafba60ec59136ffe51e77bc3bf1f329e7c36a86eb4dc340881320408e25a2ba7a1ae674d21193a6b1e81b90089408ab9080eb8dbeb64a30890051cc110088008cb6ee751b6af195a7cfd97d3ca92df96b3defa15a1ee625c507a5e132e74dc088021ecb077691135cd465d84d6d5bcbcb288b6300f56a902b2f3a3109076bd5054040215daa4535b119a8bdf23c80a985438c60e901f0177e3157a1d8eb19c840f586852f0d080203f185e7c724fc8459f93e1625847c3cb749d59256e316987f5f20804821d91d549f0840a3ae6e11593fe15fe5cefd785cfe00e9fa037a4434ef146a71192e8657b1ad89d4ab99ea4501d28627bdbc7f65d297c1e1148515b611a7a77c222b1a1a2a73c4c0005e8080000000004061bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c1800000000000000000595440000000000008800830cdee24d4350a68c49c9cfc4b7f8e99442b9d167d87c158052fe10274b157f5014011cc1100880839b50f7b80fd515d360917b7cb8c4b42924b748d42dfe37da0e42e21db79590088155751b35517fc3b30a039af20ba5dbfd252ead7cac4a201d899d4ffee795e5b80520c1140884036a98124d36d50aa96a6f2f523ad99beb22b37ba4b33635088a3d48ddf14461280881a7e1551dec9f3b0e6c57ee7999b026db1218322e03344c574926365f673c0918040215daa4535b119a8bdf23c80a985438c60e901f0177e3157a1d8eb19c840f586852f0d080203f185e7c724fc8459f93e1625847c3cb749d59256e316987f5f20804821d91d549f0840a3ae6e11593fe15fe5cefd785cfe00e9fa037a4434ef146a71192e8657b1ad89d4ab99ea4504cea013557d5d8d7281736267b01538c13146b0622b1a1a1073c4c0005e8080000000004061bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c1c00000000000000000595440000000000008800830cdee24d4350a68c49c9cfc4b7f8e99442b9d167d87c158052fe10274b157f50140120c11408840204a8d829b771105e7e0396d094af29d2ed76f8768948c327a2bf34e228bf591c0880b916996048fc098f28455fbe6a6657b78589747b2e94ca55c617c760112a874c051cc11008811b0f3e23e7a18e725531611c9520b75b9900f7b68e0930deba576e6fe674d058088190cbddb1728f58654455706c88ff4d775089fe0f87fe6a4d0ded8d45559f48fc040215daa4535b119a8bdf23c80a985438c60e901f0177e3157a1d8eb19c840f586852f0d080203f185e7c724fc8459f93e1625847c3cb749d59256e316987f5f20804821d91d549f0840a3ae6e11593fe15fe5cefd785cfe00e9fa037a4434ef146a71192e8657b1ad89d4ab99ea4504cea013557d5d8d7281736267b01538c13146b0622b1a1a1073c4c0005e4080000000004061bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c2000000000000000000595440000000000008800830cdee24d4350a68c49c9cfc4b7f8e99442b9d167d87c158052fe10274b157f5014011cc1100881da8ce34896e25510486633964daaa990cf687d47766f2f9a7656878258305a600881f6c99d45ff1cf8d63827d3224c4eb85fb5357db610cbebb27b079e128a9f69f0051cc110088120b7c063946d3da109c5a0af0f20712074a235840029df7c849abf810b0d1c7408811dc9c83fc7a8c8e89c496408a81162b9348a397d84eb798ff28e3ad7510dd5a8040215daa4535b119a8bdf23c80a985438c60e901f0177e3157a1d8eb19c840f586852f0d080203f185e7c724fc8459f93e1625847c3cb749d59256e316987f5f20804821d91d549f0840a3ae6e11593fe15fe5cefd785cfe00e9fa037a4434ef146a71192e8657b1ad89d4ab99ea4504cea013557d5d8d7281736267b01538c13146b0622b1a1a1073c4c0005e8080000000004061bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c2400000000000000000595440000000000008800830cdee24d4350a68c49c9cfc4b7f8e99442b9d167d87c158052fe10274b157f5014011cc11008807426691929da5ac8aa3962d9714ba6946681a97127a5105575397ef4534fd6e40880f024703b16e102a02f9ce74a6d7f0b1d72e524a397aa0c4cfe13489bca3352dc0520c114088402458956d5d6359b351bd94fe2969c6c7215efeb869f3e14656d75e11db104e314088004078fb261d2a89911a4598afe2516860abc48deed871167a76ffd256e425f40040215daa4535b119a8bdf23c80a985438c60e901f0177e3157a1d8eb19c840f586852f0d080203f185e7c724fc8459f93e1625847c3cb749d59256e316987f5f20804821d91d549f0840a3ae6e11593fe15fe5cefd785cfe00e9fa037a4434ef146a71192e8657b1ad89d4ab99ea4504cea013557d5d8d7281736267b01538c13146b0622b1a1a1073c4c0005e8080000000004061bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c28000000000000000005b5800000000000008800830cdee24d4350a68c49c9cfc4b7f8e99442b9d167d87c158052fe10274b157f5014011cc1100880ebdedfa85b30063f6d053d4b34e3b55237095f2ec1b204b27707581402c852140881cb35d2df926ff604f826eb9de36a40ed12dec2b88ae1ebd30d33e2eddf7f7bc80520c11408840363aa90dbebb1ace3c55419624fc484b390d857efb126d2bdcbc58465dd2d51c80880cea8c0e64bd2a33f871df9a834aea9ceeab40c16a23685b49848b94da19d0c44040215daa4535b119a8bdf23c80a985438c60e901f0177e3157a1d8eb19c840f586852f0d080203f185e7c724fc8459f93e1625847c3cb749d59256e316987f5f20804821d91d549f0840a3ae6e11593fe15fe5cefd785cfe00e9fa037a4434ef146a71192e8657b1ad89d4ab99ea45149d6e9c52d8cff9fd703807f156df476ac33046222b1a1a05f404c0005e4080000000004061bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c2c000000000000000006b7080000000000008800830cdee24d4350a68c49c9cfc4b7f8e99442b9d167d87c158052fe10274b157f5014011cc11008810d5cfb6cdefa05746e5ee4200fd8077f2470949e7336981ab59457fb9c87f95c0880c2b0a20d022881d691ccdd10fad4e418aa2b1ad514efbecae618f5a5f8d77608051cc11008813d77410afed127148804a8b518796a5327a8ef98a702ac246c387d55a7ac4670088085047fe2bfeaeac4b34e7c3aa993c6c23e9cab69bcdf4751b1af79120f61a7ec040215daa4535b119a8bdf23c80a985438c60e901f0177e3157a1d8eb19c840f586852f0d080203f185e7c724fc8459f93e1625847c3cb749d59256e316987f5f20804821d91d549f0840a3ae6e11593fe15fe5cefd785cfe00e9fa037a4434ef146a71192e8657b1ad89d4ab99ea450306a74e9564c77db956b93679a833577c0e0ac2622b1a1a25b3c4c0005e8080000000004061bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c30000000000000000004cf780400000000008800830cdee24d4350a68c49c9cfc4b7f8e99442b9d167d87c158052fe10274b157f50140120c1140884024ff5f7e8fbdb37d6e533eb7e0c08afa6018b54f35c25e9ce51d14ec8429207ac088188b98e886de1fa9eed57c06adbf940ff2e3bd32d96b739e8992b805100d91bf8051cc1100880c3fcf5350bbc70f5d085928c3fdec084857a207649de94a85c81452911cca25408812c243dad04ba32ab2d6f2fca56eaec1d767552438fdd9e30a24819e03dbc2f24040215daa4535b119a8bdf23c80a985438c60e901f0177e3157a1d8eb19c840f586852f0d080203f185e7c724fc8459f93e1625847c3cb749d59256e316987f5f20804821d91d549f0840a3ae6e11593fe15fe5cefd785cfe00e9fa037a4434ef146a71192e8657b1ad89d4ab99ea451b717a849e11edcc1f9da13d7a7982697b42cee0222b1a1a0af3c4c00003403b4080000000004040962d0951dadda42c57a3cb36fad482714189059a23b356f8ac595c476b70e1c0000000002c410000004f4680000000000005ea4514e9f3971721f43554b27d9c7ee5feba9d5fd57b21c0d1cc1100880e3369522545ab94fbdb4b79f7bfb88c183c14d08ea891bcefbb87df0fa6c4da4088071c3e3658f3992a5c41b8d9133efc03edca07df82bff7d28cdefcc14d702e1380401358c840db76008893ef27ce38f74d1a458d73512ef9c6de60ffed31d55960f9ed29534599c0ac412c9d4840e5882e33e559c7b0d4a85c2ed7cf024c8a4785d6d5518608faf36c0577b4c3bf1a2b00000000003b408000000000405b56587ea216e35f02c34532af15a2acd5a4c4fdc35cf194e86fd3141050c672c0000000002c41000000494780000000000005ea4514e9f3971721f43554b27d9c7ee5feba9d5fd57b21c0d1cc110088150ba87cd36ec23d80ac12e7cd1164578e71e5235148cfac95640c39977db93e808802de7e704165480ae84e0d8c7bcaed5b5b755947a0b2f557e5fff7794b06734880401358c840db76008893ef27ce38f74d1a458d73512ef9c6de60ffed31d55960f9ed29534599c0ac412c9d4840e5882e33e559c7b0d4a85c2ed7cf024c8a4785d6d5518608faf36c0577b4c3bf1a2b00000000003b8080000000004041f972d10a95697bb9fd758cd126b37ce6bc0005a7420af315b0c2165268266100000000002c410000007b5400000000000005ea4514e9f3971721f43554b27d9c7ee5feba9d5fd57b21c0d20c11408840304097ac738ee7d447e0fb8664c2aa2112116132fa7504605051b82e1abd3f2d00881d7b45e635557a2ac836b2a9d813f964d21990bc16598bb07ab58582063e7879c0401358c840db76008893ef27ce38f74d1a458d73512ef9c6de60ffed31d55960f9ed29534599c0ac412c9d4840e5882e33e559c7b0d4a85c2ed7cf024c8a4785d6d5518608faf36c0577b4c3bf1a2b00000000003b408000000000404e543a6b529a60e47aedc1e465671cc7df1c027366c03f11953887a6c865be7500000000002c410000007b5400000000000005ea4514e9f3971721f43554b27d9c7ee5feba9d5fd57b21c0d1cc110088099fc4f51964e576dbd51b1cfb6ee49b3edf49a3a729b79b825424cb8e634ddb008810fedc0468d8add5c070df7ab758577c8a0f96f5975b989dbc7ca86b4d6dfe8c00401358c840db76008893ef27ce38f74d1a458d73512ef9c6de60ffed31d55960f9ed29534599c0ac412c9d4840e5882e33e559c7b0d4a85c2ed7cf024c8a4785d6d5518608faf36c0577b4c3bf1a2b00000000003b408000000000404f372d8314f4cf7a31cbf1825ba12e1c9d36ec266e7a8026c7bd9c838628c5ce00000000002c410000007b5400000000000005ea4514e9f3971721f43554b27d9c7ee5feba9d5fd57b21c0d1cc1100881c00c5af0b98ed782be0dab0ac17057476654b15a2f7dbe241401a37053c07c500881b408b91f0beb47eef2df6d91a2db83914531439e34daa7397075b813c6acd7500401358c840db76008893ef27ce38f74d1a458d73512ef9c6de60ffed31d55960f9ed29534599c0ac412c9d4840e5882e33e559c7b0d4a85c2ed7cf024c8a4785d6d5518608faf36c0577b4c3bf1a2b00000000003b4080000000004040d9a2a112060c947a639bac4f414b1cb253f2b45ea6318f0019171da574647b40000000002c410000007b5400000000000005ea4514e9f3971721f43554b27d9c7ee5feba9d5fd57b21c0d1cc1100881663fdf21905a218cf0e7aa0b457537a948e7478577f36c98d30d1bf13273041808809f2448db5f8976c730751377280b5549b8a5e257fc19864a8faa64cb4367bb540401358c840db76008893ef27ce38f74d1a458d73512ef9c6de60ffed31d55960f9ed29534599c0ac412c9d4840e5882e33e559c7b0d4a85c2ed7cf024c8a4785d6d5518608faf36c0577b4c3bf1a2b00000000003b40800000000040694ef1b4c2a3f980a12e53c70cee6bc2fe4638d29a8c5d295448be38b852a50740000000002c410000007b5400000000000005ea4514e9f3971721f43554b27d9c7ee5feba9d5fd57b21c0d1cc11008802a246bd128f60777559e8324115260cbf61a1ef837d3d46864ed4052badc21ec0881ec8eb04f1cc0a501b1ceda3848b84d13276a3f489002b649ecf8f7e6bd8006d80401358c840db76008893ef27ce38f74d1a458d73512ef9c6de60ffed31d55960f9ed29534599c0ac412c9d4840e5882e33e559c7b0d4a85c2ed7cf024c8a4785d6d5518608faf36c0577b4c3bf1a2b00000000003b40800000000040451b8cd3bc3aabad017496d72f4038f0e2e8f7ee3287115904a3be6b36fdf1d640000000002c410000007b5400000000000005ea4514e9f3971721f43554b27d9c7ee5feba9d5fd57b21c0d1cc11008805af59e3bcde88a0a4d19acfb535816e50c62fe6a9f008aedacc46898433d819c08800381c161ea8f90f2b5559b2ea5348f4cde8c10557654c059af159772a4b2dda40401358c840db76008893ef27ce38f74d1a458d73512ef9c6de60ffed31d55960f9ed29534599c0ac412c9d4840e5882e33e559c7b0d4a85c2ed7cf024c8a4785d6d5518608faf36c0577b4c3bf1a2b00000000003b4080000000004051427857372c4863df3bbcee14874e5a5b8b5ae77fe811b9720ecb87e65d45f100000000002c410000007b5400000000000005ea4514e9f3971721f43554b27d9c7ee5feba9d5fd57b21c0d1cc1100880473c75fb98e7edbea2412c00b5d3288cfd5dbcef0dc5198136f34bc4e5fdcb54088196b8df3b09ec756ce4ae1cfa59d524c92fa1379df50815b8dfc3b590a2da1dcc0401358c840db76008893ef27ce38f74d1a458d73512ef9c6de60ffed31d55960f9ed29534599c0ac412c9d4840e5882e33e559c7b0d4a85c2ed7cf024c8a4785d6d5518608faf36c0577b4c3bf1a2b00000000003b80800000000040727096b8e5ce46602d148b24b34028a18d8c0848c06b45f1333459b9cdaba9a900000000002c410000007b5400000000000005ea4514e9f3971721f43554b27d9c7ee5feba9d5fd57b21c0d20c114088403dae39436c6e1fbc4b4aaae96bfb0f014950025adb1d051340c13484fd135ad6408802b97a56b43ca527a7800f2db59276ebbe673737b3bbedebea2ef5cb48406853c0401358c840db76008893ef27ce38f74d1a458d73512ef9c6de60ffed31d55960f9ed29534599c0ac412c9d4840e5882e33e559c7b0d4a85c2ed7cf024c8a4785d6d5518608faf36c0577b4c3bf1a2b00000000003b808000000000404c16940802fee5aba6a5733dfd84aee15b9bc6cc7302c48e34d8db7765411cdfc0000000002c410000007d57c0000000000005ea4514e9f3971721f43554b27d9c7ee5feba9d5fd57b21c0d20c114088402fbc3a7c464b2131beba56d22fd557078a278346b09c76fe82115a74f28a1c08c08801672164a23bba89227ba6e01e172306e5d163ff4f82ea6ad9bf051cfd455ebd00401358c840db76008893ef27ce38f74d1a458d73512ef9c6de60ffed31d55960f9ed29534599c0ac412c9d4840e5882e33e559c7b0d4a85c2ed7cf024c8a4785d6d5518608faf36c0577b4c3bf1a2b00000000003b408000000000405a31121b88f9abcd1dd32077e8695ecffaf340583f3498aa558cddbb9e9d73d240000000002c410000004d7080000000000005ea4514e9f3971721f43554b27d9c7ee5feba9d5fd57b21c0d1cc11008812a0cb4135595b8735224cbd87042acc960beaf69b383fde8c9f5baed407914300881dec7c72220306baf2b5e4ae9a60007cf991fbca3204c6dd32d3996bef67009700401358c840db76008893ef27ce38f74d1a458d73512ef9c6de60ffed31d55960f9ed29534599c0ac412c9d4840e5882e33e559c7b0d4a85c2ed7cf024c8a4785d6d5518608faf36c0577b4c3bf1a2b00000000003b80800000000040623ab81aff972f9a0b6d87ca44230ccdd72da930e6393a7b0fe42dbb7bdd2363c0000000002c410000006ef740400000000005ea4514e9f3971721f43554b27d9c7ee5feba9d5fd57b21c0d20c1140884031e1793d5d6989a6666d4f504313c6f1a97a601945f1ac00e85e48315ab2b724c08818e4dad0879a764a2d8a1f46a9d318e3c118fda41aee6648540755f303f5d01c00401358c840db76008893ef27ce38f74d1a458d73512ef9c6de60ffed31d55960f9ed29534599c0ac412c9d4840e5882e33e559c7b0d4a85c2ed7cf024c8a4785d6d5518608faf36c0577b4c3bf1a2b000000000003000921bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c28000001fdcc125775b58f4ce0482f30c46dbdba16ec4bdbdf32566aba5bec2c814168c000921bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c14000003b6474655d9719000f31a61eb47293f25c8b014f6c5ba3ba544c861221229980c00921bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c1800000076512b878be0894695d0c5a92b8c63e42ebee4cc713ee6100b9a3c2b4f18ee9400921bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c20000003105dd6647cb8ef23951a83fe75acb5b9a4e4754ae3cfb9f23c86c7717384251400921bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c1c000001671fdf6eb3e638499114732ae37e8e2f0f8c02f5716f4816d2bbabc138cdb85000921bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c000000021f0eb475c596c7896f3639a058901b14254baf6cb23d7ac641d9ad1c96d1600800921bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c10000000e05e8c6039cbd87a6c03aae666c36dd1cae211b82718bf1fa0f74d4c32db70f000921bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c0c00000353e658866c7b89511bf0026f3425c1f07dce7256441edfae460da52ab7a540e40093e6ee8f639d4fd51a95550ec539e61188e14ee5e4d9ba5f7a61e2e8ba81b24988000000018f4382faf208162b6c23cfa83010179e3b6491b4ebe4aedff24f30de8fb7da1800921bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c2c000001273dd5ebb9d8cd5aa58b4bf1800737acfdee96877e0531dcd2bf988db92311a000921bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c24000002929ab8db9d9b4733105eb4048c84c0da1a88033648b14ad20064e65f8d6b0b2400921bdbb68cdf324ff2dcafe4e9b493663b9c141033abcc236e28160af2fb83418c300000023e35d3bfb6da40ffb3a792630e92d970dcce3040a47ed8b5a2f973f9af83aa2000000 \ No newline at end of file diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/PackageSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/PackageSpec.scala index 95364daf6..3dd302c3c 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/PackageSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/PackageSpec.scala @@ -40,7 +40,7 @@ class PackageSpec extends FunSuite { } test("decode base58 addresses") { - val priv = PrivateKey(ByteVector32(ByteVector.fill(32)(1)), compressed = true) + val priv = PrivateKey(ByteVector32(ByteVector.fill(32)(1))) val pub = priv.publicKey // p2pkh @@ -71,7 +71,7 @@ class PackageSpec extends FunSuite { } test("decode bech32 addresses") { - val priv = PrivateKey(ByteVector32(ByteVector.fill(32)(1)), compressed = true) + val priv = PrivateKey(ByteVector32(ByteVector.fill(32)(1))) val pub = priv.publicKey // p2wpkh diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/StartupSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/StartupSpec.scala index 277f3ee96..1a00b51d6 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/StartupSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/StartupSpec.scala @@ -40,7 +40,7 @@ class StartupSpec extends FunSuite { val illegalAliasConf = ConfigFactory.parseString(s"node-alias = $goUkraineGo") val conf = illegalAliasConf.withFallback(ConfigFactory.parseResources("reference.conf").getConfig("eclair")) - val keyManager = new LocalKeyManager(seed = randomKey.toBin, chainHash = Block.TestnetGenesisBlock.hash) + val keyManager = new LocalKeyManager(seed = randomBytes32, chainHash = Block.TestnetGenesisBlock.hash) // try to create a NodeParams instance with a conf that contains an illegal alias val nodeParamsAttempt = Try(NodeParams.makeNodeParams(conf, keyManager, None, TestConstants.inMemoryDb())) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala b/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala index c807936f5..b2a327223 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/TestConstants.scala @@ -101,7 +101,7 @@ object TestConstants { def channelParams = Peer.makeChannelParams( nodeParams = nodeParams, - defaultFinalScriptPubKey = Script.write(Script.pay2wpkh(PrivateKey(randomBytes32, compressed = true).publicKey)), + defaultFinalScriptPubKey = Script.write(Script.pay2wpkh(PrivateKey(randomBytes32).publicKey)), isFunder = true, fundingSatoshis).copy( channelReserveSatoshis = 10000 // Bob will need to keep that much satoshis as direct payment @@ -165,7 +165,7 @@ object TestConstants { def channelParams = Peer.makeChannelParams( nodeParams = nodeParams, - defaultFinalScriptPubKey = Script.write(Script.pay2wpkh(PrivateKey(randomBytes32, compressed = true).publicKey)), + defaultFinalScriptPubKey = Script.write(Script.pay2wpkh(PrivateKey(randomBytes32).publicKey)), isFunder = false, fundingSatoshis).copy( channelReserveSatoshis = 20000 // Alice will need to keep that much satoshis as direct payment diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala index abe4b102a..c55864946 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala @@ -214,7 +214,7 @@ class ApiServiceSpec extends FunSuite with ScalatestRouteTest with IdiomaticMock eclair.connect(any[Either[NodeURI, PublicKey]])(any[Timeout]) returns Future.successful("connected") val mockService = new MockService(eclair) - Post("/connect", FormData("nodeId" -> remoteNodeId.toHex).toEntity) ~> + Post("/connect", FormData("nodeId" -> remoteNodeId.toString()).toEntity) ~> addCredentials(BasicHttpCredentials("", mockService.password)) ~> Route.seal(mockService.route) ~> check { diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/TestWallet.scala b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/TestWallet.scala index 05136c672..825a609ef 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/TestWallet.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/TestWallet.scala @@ -55,16 +55,4 @@ object TestWallet { lockTime = 0) MakeFundingTxResponse(fundingTx, 0, Satoshi(420)) } - - def malleateTx(tx: Transaction): Transaction = { - val inputs1 = tx.txIn.map(input => Script.parse(input.signatureScript) match { - case OP_PUSHDATA(sig, _) :: OP_PUSHDATA(pub, _) :: Nil if pub.length == 33 && Try(Crypto.decodeSignature(sig)).isSuccess => - val (r, s) = Crypto.decodeSignature(sig) - val s1 = Crypto.curve.getN.subtract(s) - val sig1 = Crypto.encodeSignature(r, s1) - input.copy(signatureScript = Script.write(OP_PUSHDATA(sig1) :: OP_PUSHDATA(pub) :: Nil)) - }) - val tx1 = tx.copy(txIn = inputs1) - tx1 - } } \ No newline at end of file diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/electrum/ElectrumWalletBasicSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/electrum/ElectrumWalletBasicSpec.scala index 80696ca32..375840f74 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/electrum/ElectrumWalletBasicSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/electrum/ElectrumWalletBasicSpec.scala @@ -79,7 +79,7 @@ class ElectrumWalletBasicSpec extends FunSuite with Logging { test("compute addresses") { - val priv = PrivateKey.fromBase58("cRumXueoZHjhGXrZWeFoEBkeDHu2m8dW5qtFBCqSAt4LDR2Hnd8Q", Base58.Prefix.SecretKeyTestnet) + val priv = PrivateKey.fromBase58("cRumXueoZHjhGXrZWeFoEBkeDHu2m8dW5qtFBCqSAt4LDR2Hnd8Q", Base58.Prefix.SecretKeyTestnet)._1 assert(Base58Check.encode(Base58.Prefix.PubkeyAddressTestnet, priv.publicKey.hash160) == "ms93boMGZZjvjciujPJgDAqeR86EKBf9MC") assert(segwitAddress(priv, Block.RegtestGenesisBlock.hash) == "2MscvqgGXMTYJNAY3owdUtgWJaxPUjH38Cx") } @@ -98,7 +98,7 @@ class ElectrumWalletBasicSpec extends FunSuite with Logging { val state1 = addFunds(state, state.accountKeys.head, 1 btc) val (confirmed1, unconfirmed1) = state1.balance - val pub = PrivateKey(ByteVector32(ByteVector.fill(32)(1)), compressed = true).publicKey + val pub = PrivateKey(ByteVector32(ByteVector.fill(32)(1))).publicKey val tx = Transaction(version = 2, txIn = Nil, txOut = TxOut(0.5 btc, Script.pay2pkh(pub)) :: Nil, lockTime = 0) val (state2, tx1, fee1) = state1.completeTransaction(tx, feeRatePerKw, minimumFee, dustLimit, false) val Some((_, _, Some(fee))) = state2.computeTransactionDelta(tx1) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/electrum/ElectrumWatcherSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/electrum/ElectrumWatcherSpec.scala index 27a367ed9..066c9fb61 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/electrum/ElectrumWatcherSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/electrum/ElectrumWatcherSpec.scala @@ -86,7 +86,7 @@ class ElectrumWatcherSpec extends TestKit(ActorSystem("test")) with FunSuiteLike probe.send(bitcoincli, BitcoinReq("dumpprivkey", address)) val JString(wif) = probe.expectMsgType[JValue] - val priv = PrivateKey.fromBase58(wif, Base58.Prefix.SecretKeyTestnet) + val (priv, true) = PrivateKey.fromBase58(wif, Base58.Prefix.SecretKeyTestnet) probe.send(bitcoincli, BitcoinReq("sendtoaddress", address, 1.0)) val JString(txid) = probe.expectMsgType[JValue](30 seconds) @@ -104,7 +104,7 @@ class ElectrumWatcherSpec extends TestKit(ActorSystem("test")) with FunSuiteLike txOut = TxOut(tx.txOut(pos).amount - Satoshi(1000), publicKeyScript = Script.pay2wpkh(priv.publicKey)) :: Nil, lockTime = 0) val sig = Transaction.signInput(tmp, 0, Script.pay2pkh(priv.publicKey), SIGHASH_ALL, tx.txOut(pos).amount, SigVersion.SIGVERSION_WITNESS_V0, priv) - val signedTx = tmp.updateWitness(0, ScriptWitness(sig :: priv.publicKey.toBin :: Nil)) + val signedTx = tmp.updateWitness(0, ScriptWitness(sig :: priv.publicKey.value :: Nil)) Transaction.correctlySpends(signedTx, Seq(tx), ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS) signedTx } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForFundingSignedStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForFundingSignedStateSpec.scala index ef531d0f7..12722b2e1 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForFundingSignedStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForFundingSignedStateSpec.scala @@ -17,7 +17,7 @@ package fr.acinq.eclair.channel.states.b import akka.testkit.{TestFSMRef, TestProbe} -import fr.acinq.bitcoin.ByteVector32 +import fr.acinq.bitcoin.{ByteVector32, ByteVector64} import fr.acinq.eclair.TestConstants.{Alice, Bob} import fr.acinq.eclair.blockchain._ import fr.acinq.eclair.channel.Channel.TickChannelOpenTimeout @@ -69,7 +69,7 @@ class WaitForFundingSignedStateSpec extends TestkitBaseClass with StateTestsHelp test("recv FundingSigned with invalid signature") { f => import f._ // sending an invalid sig - alice ! FundingSigned(ByteVector32.Zeroes, ByteVector.fill(64)(0)) + alice ! FundingSigned(ByteVector32.Zeroes, ByteVector64.Zeroes) awaitCond(alice.stateName == CLOSED) alice2bob.expectMsgType[Error] } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalStateSpec.scala index 0d8613225..5e9f0696f 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/NormalStateSpec.scala @@ -21,8 +21,8 @@ import java.util.UUID import akka.actor.Status import akka.actor.Status.Failure import akka.testkit.TestProbe -import fr.acinq.bitcoin.Crypto.Scalar -import fr.acinq.bitcoin.{ByteVector32, Crypto, Satoshi, ScriptFlags, Transaction} +import fr.acinq.bitcoin.Crypto.PrivateKey +import fr.acinq.bitcoin.{ByteVector32, ByteVector64, Crypto, Satoshi, ScriptFlags, Transaction} import fr.acinq.eclair.TestConstants.{Alice, Bob} import fr.acinq.eclair.UInt64.Conversions._ import fr.acinq.eclair.blockchain._ @@ -719,7 +719,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods { val tx = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.publishableTxs.commitTx.tx val sender = TestProbe() // signature is invalid but it doesn't matter - sender.send(bob, CommitSig(ByteVector32.Zeroes, ByteVector.fill(64)(0), Nil)) + sender.send(bob, CommitSig(ByteVector32.Zeroes, ByteVector64.Zeroes, Nil)) val error = bob2alice.expectMsgType[Error] assert(new String(error.data.toArray).startsWith("cannot sign when there are no changes")) awaitCond(bob.stateName == CLOSING) @@ -737,7 +737,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods { val tx = bob.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.publishableTxs.commitTx.tx // actual test begins - sender.send(bob, CommitSig(ByteVector32.Zeroes, ByteVector.fill(64)(0), Nil)) + sender.send(bob, CommitSig(ByteVector32.Zeroes, ByteVector64.Zeroes, Nil)) val error = bob2alice.expectMsgType[Error] assert(new String(error.data.toArray).startsWith("invalid commitment signature")) awaitCond(bob.stateName == CLOSING) @@ -906,7 +906,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods { // actual test begins bob2alice.expectMsgType[RevokeAndAck] - sender.send(alice, RevokeAndAck(ByteVector32.Zeroes, Scalar(randomBytes32), Scalar(randomBytes32).toPoint)) + sender.send(alice, RevokeAndAck(ByteVector32.Zeroes, PrivateKey(randomBytes32), PrivateKey(randomBytes32).publicKey)) alice2bob.expectMsgType[Error] awaitCond(alice.stateName == CLOSING) // channel should be advertised as down @@ -921,7 +921,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods { val tx = alice.stateData.asInstanceOf[DATA_NORMAL].commitments.localCommit.publishableTxs.commitTx.tx val sender = TestProbe() awaitCond(alice.stateData.asInstanceOf[DATA_NORMAL].commitments.remoteNextCommitInfo.isRight) - sender.send(alice, RevokeAndAck(ByteVector32.Zeroes, Scalar(randomBytes32), Scalar(randomBytes32).toPoint)) + sender.send(alice, RevokeAndAck(ByteVector32.Zeroes, PrivateKey(randomBytes32), PrivateKey(randomBytes32).publicKey)) alice2bob.expectMsgType[Error] awaitCond(alice.stateName == CLOSING) // channel should be advertised as down @@ -1757,7 +1757,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods { claimHtlcTx.txOut(0).amount }).sum // at best we have a little less than 450 000 + 250 000 + 100 000 + 50 000 = 850 000 (because fees) - assert(amountClaimed == Satoshi(814840)) + assert(amountClaimed == Satoshi(814880)) assert(alice2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(bobCommitTx)) assert(alice2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(claimTxes(0))) // claim-main @@ -1819,7 +1819,7 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods { claimHtlcTx.txOut(0).amount }).sum // at best we have a little less than 500 000 + 250 000 + 100 000 = 850 000 (because fees) - assert(amountClaimed == Satoshi(822280)) + assert(amountClaimed == Satoshi(822310)) assert(alice2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(bobCommitTx)) assert(alice2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(claimTxes(0))) // claim-main @@ -1881,12 +1881,12 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods { htlcPenaltyTxs.foreach(htlcPenaltyTx => Transaction.correctlySpends(htlcPenaltyTx, Seq(revokedTx), ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)) // two main outputs are 760 000 and 200 000 - assert(mainTx.txOut(0).amount == Satoshi(741490)) - assert(mainPenaltyTx.txOut(0).amount == Satoshi(195150)) - assert(htlcPenaltyTxs(0).txOut(0).amount == Satoshi(4530)) - assert(htlcPenaltyTxs(1).txOut(0).amount == Satoshi(4530)) - assert(htlcPenaltyTxs(2).txOut(0).amount == Satoshi(4530)) - assert(htlcPenaltyTxs(3).txOut(0).amount == Satoshi(4530)) + assert(mainTx.txOut(0).amount == Satoshi(741500)) + assert(mainPenaltyTx.txOut(0).amount == Satoshi(195160)) + assert(htlcPenaltyTxs(0).txOut(0).amount == Satoshi(4540)) + assert(htlcPenaltyTxs(1).txOut(0).amount == Satoshi(4540)) + assert(htlcPenaltyTxs(2).txOut(0).amount == Satoshi(4540)) + assert(htlcPenaltyTxs(3).txOut(0).amount == Satoshi(4540)) awaitCond(alice.stateName == CLOSING) assert(alice.stateData.asInstanceOf[DATA_CLOSING].revokedCommitPublished.size == 1) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/OfflineStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/OfflineStateSpec.scala index f864e492f..e3a2c2dfc 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/OfflineStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/e/OfflineStateSpec.scala @@ -20,7 +20,7 @@ import akka.actor.Status import java.util.UUID import akka.testkit.TestProbe -import fr.acinq.bitcoin.Crypto.Scalar +import fr.acinq.bitcoin.Crypto.{PrivateKey} import fr.acinq.bitcoin.{ByteVector32, ScriptFlags, Transaction} import fr.acinq.eclair.blockchain.{PublishAsap, WatchEventSpent} import fr.acinq.eclair.channel._ @@ -85,9 +85,9 @@ class OfflineStateSpec extends TestkitBaseClass with StateTestsHelperMethods { // a didn't receive any update or sig - val ab_reestablish = alice2bob.expectMsg(ChannelReestablish(ab_add_0.channelId, 1, 0, Some(Scalar(ByteVector32.Zeroes)), Some(aliceCurrentPerCommitmentPoint))) + val ab_reestablish = alice2bob.expectMsg(ChannelReestablish(ab_add_0.channelId, 1, 0, Some(PrivateKey(ByteVector32.Zeroes)), Some(aliceCurrentPerCommitmentPoint))) // b didn't receive the sig - val ba_reestablish = bob2alice.expectMsg(ChannelReestablish(ab_add_0.channelId, 1, 0, Some(Scalar(ByteVector32.Zeroes)), Some(bobCurrentPerCommitmentPoint))) + val ba_reestablish = bob2alice.expectMsg(ChannelReestablish(ab_add_0.channelId, 1, 0, Some(PrivateKey(ByteVector32.Zeroes)), Some(bobCurrentPerCommitmentPoint))) // reestablish ->b alice2bob.forward(bob, ab_reestablish) @@ -168,9 +168,9 @@ class OfflineStateSpec extends TestkitBaseClass with StateTestsHelperMethods { val aliceCurrentPerCommitmentPoint = TestConstants.Alice.keyManager.commitmentPoint(aliceCommitments.localParams.channelKeyPath, aliceCommitments.localCommit.index) // a didn't receive the sig - val ab_reestablish = alice2bob.expectMsg(ChannelReestablish(ab_add_0.channelId, 1, 0, Some(Scalar(ByteVector32.Zeroes)), Some(aliceCurrentPerCommitmentPoint))) + val ab_reestablish = alice2bob.expectMsg(ChannelReestablish(ab_add_0.channelId, 1, 0, Some(PrivateKey(ByteVector32.Zeroes)), Some(aliceCurrentPerCommitmentPoint))) // b did receive the sig - val ba_reestablish = bob2alice.expectMsg(ChannelReestablish(ab_add_0.channelId, 2, 0, Some(Scalar(ByteVector32.Zeroes)), Some(bobCurrentPerCommitmentPoint))) + val ba_reestablish = bob2alice.expectMsg(ChannelReestablish(ab_add_0.channelId, 2, 0, Some(PrivateKey(ByteVector32.Zeroes)), Some(bobCurrentPerCommitmentPoint))) // reestablish ->b alice2bob.forward(bob, ab_reestablish) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/f/ShutdownStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/f/ShutdownStateSpec.scala index a1c80fd03..dd73f294f 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/f/ShutdownStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/f/ShutdownStateSpec.scala @@ -20,8 +20,8 @@ import java.util.UUID import akka.actor.Status.Failure import akka.testkit.TestProbe -import fr.acinq.bitcoin.Crypto.Scalar -import fr.acinq.bitcoin.{ByteVector32, Crypto, Satoshi, ScriptFlags, Transaction} +import fr.acinq.bitcoin.Crypto.{PrivateKey} +import fr.acinq.bitcoin.{ByteVector32, ByteVector64, Crypto, Satoshi, ScriptFlags, Transaction} import fr.acinq.eclair.blockchain._ import fr.acinq.eclair.blockchain.fee.FeeratesPerKw import fr.acinq.eclair.channel._ @@ -380,7 +380,7 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods { val tx = bob.stateData.asInstanceOf[DATA_SHUTDOWN].commitments.localCommit.publishableTxs.commitTx.tx val sender = TestProbe() // signature is invalid but it doesn't matter - sender.send(bob, CommitSig(ByteVector32.Zeroes, ByteVector.fill(64)(0), Nil)) + sender.send(bob, CommitSig(ByteVector32.Zeroes, ByteVector64.Zeroes, Nil)) bob2alice.expectMsgType[Error] awaitCond(bob.stateName == CLOSING) bob2blockchain.expectMsg(PublishAsap(tx)) // commit tx @@ -392,7 +392,7 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods { import f._ val tx = bob.stateData.asInstanceOf[DATA_SHUTDOWN].commitments.localCommit.publishableTxs.commitTx.tx val sender = TestProbe() - sender.send(bob, CommitSig(ByteVector32.Zeroes, ByteVector.fill(64)(0), Nil)) + sender.send(bob, CommitSig(ByteVector32.Zeroes, ByteVector64.Zeroes, Nil)) bob2alice.expectMsgType[Error] awaitCond(bob.stateName == CLOSING) bob2blockchain.expectMsg(PublishAsap(tx)) // commit tx @@ -451,7 +451,7 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods { bob2alice.forward(alice) alice2bob.expectMsgType[RevokeAndAck] awaitCond(bob.stateData.asInstanceOf[DATA_SHUTDOWN].commitments.remoteNextCommitInfo.isLeft) - sender.send(bob, RevokeAndAck(ByteVector32.Zeroes, Scalar(randomBytes32), Scalar(randomBytes32).toPoint)) + sender.send(bob, RevokeAndAck(ByteVector32.Zeroes, PrivateKey(randomBytes32), PrivateKey(randomBytes32).publicKey)) bob2alice.expectMsgType[Error] awaitCond(bob.stateName == CLOSING) bob2blockchain.expectMsg(PublishAsap(tx)) // commit tx @@ -466,7 +466,7 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods { val tx = alice.stateData.asInstanceOf[DATA_SHUTDOWN].commitments.localCommit.publishableTxs.commitTx.tx val sender = TestProbe() awaitCond(alice.stateData.asInstanceOf[DATA_SHUTDOWN].commitments.remoteNextCommitInfo.isRight) - sender.send(alice, RevokeAndAck(ByteVector32.Zeroes, Scalar(randomBytes32), Scalar(randomBytes32).toPoint)) + sender.send(alice, RevokeAndAck(ByteVector32.Zeroes, PrivateKey(randomBytes32), PrivateKey(randomBytes32).publicKey)) alice2bob.expectMsgType[Error] awaitCond(alice.stateName == CLOSING) alice2blockchain.expectMsg(PublishAsap(tx)) // commit tx @@ -695,7 +695,7 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods { claimHtlcTx.txOut(0).amount }).sum // htlc will timeout and be eventually refunded so we have a little less than fundingSatoshis - pushMsat = 1000000 - 200000 = 800000 (because fees) - assert(amountClaimed == Satoshi(774010)) + assert(amountClaimed == Satoshi(774040)) assert(alice2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(bobCommitTx)) assert(alice2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(claimTxes(0))) @@ -742,7 +742,7 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods { claimHtlcTx.txOut(0).amount }).sum // htlc will timeout and be eventually refunded so we have a little less than fundingSatoshis - pushMsat - htlc1 = 1000000 - 200000 - 300 000 = 500000 (because fees) - assert(amountClaimed == Satoshi(481190)) + assert(amountClaimed == Satoshi(481210)) assert(alice2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(bobCommitTx)) assert(alice2blockchain.expectMsgType[WatchConfirmed].event === BITCOIN_TX_CONFIRMED(claimTxes(0))) @@ -788,10 +788,10 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods { Transaction.correctlySpends(htlc2PenaltyTx, Seq(revokedTx), ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS) // two main outputs are 300 000 and 200 000, htlcs are 300 000 and 200 000 - assert(mainTx.txOut(0).amount == Satoshi(284930)) - assert(mainPenaltyTx.txOut(0).amount == Satoshi(195150)) - assert(htlc1PenaltyTx.txOut(0).amount == Satoshi(194530)) - assert(htlc2PenaltyTx.txOut(0).amount == Satoshi(294530)) + assert(mainTx.txOut(0).amount == Satoshi(284940)) + assert(mainPenaltyTx.txOut(0).amount == Satoshi(195160)) + assert(htlc1PenaltyTx.txOut(0).amount == Satoshi(194540)) + assert(htlc2PenaltyTx.txOut(0).amount == Satoshi(294540)) awaitCond(alice.stateName == CLOSING) assert(alice.stateData.asInstanceOf[DATA_CLOSING].revokedCommitPublished.size == 1) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/g/NegotiatingStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/g/NegotiatingStateSpec.scala index 87a3132cf..6ae3d6739 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/g/NegotiatingStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/g/NegotiatingStateSpec.scala @@ -21,7 +21,7 @@ import java.util.UUID import akka.actor.Status.Failure import akka.event.LoggingAdapter import akka.testkit.TestProbe -import fr.acinq.bitcoin.{ByteVector32, Satoshi} +import fr.acinq.bitcoin.{ByteVector32, ByteVector64, Satoshi} import fr.acinq.eclair.TestConstants.Bob import fr.acinq.eclair.blockchain._ import fr.acinq.eclair.blockchain.fee.FeeratesPerKw @@ -135,7 +135,7 @@ class NegotiatingStateSpec extends TestkitBaseClass with StateTestsHelperMethods val aliceCloseSig = alice2bob.expectMsgType[ClosingSigned] val sender = TestProbe() val tx = bob.stateData.asInstanceOf[DATA_NEGOTIATING].commitments.localCommit.publishableTxs.commitTx.tx - sender.send(bob, aliceCloseSig.copy(signature = ByteVector.fill(64)(0))) + sender.send(bob, aliceCloseSig.copy(signature = ByteVector64.Zeroes)) val error = bob2alice.expectMsgType[Error] assert(new String(error.data.toArray).startsWith("invalid close signature")) bob2blockchain.expectMsg(PublishAsap(tx)) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/h/ClosingStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/h/ClosingStateSpec.scala index 89b7ab4be..c2a53b9b6 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/h/ClosingStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/h/ClosingStateSpec.scala @@ -21,13 +21,13 @@ import java.util.UUID import akka.actor.Status import akka.actor.Status.Failure import akka.testkit.{TestFSMRef, TestProbe} -import fr.acinq.bitcoin.{ByteVector32, OutPoint, ScriptFlags, Transaction, TxIn} +import fr.acinq.bitcoin.{ByteVector32, ByteVector64, OutPoint, ScriptFlags, Transaction, TxIn} import fr.acinq.eclair.blockchain._ import fr.acinq.eclair.blockchain.fee.FeeratesPerKw import fr.acinq.eclair.channel.states.StateTestsHelperMethods import fr.acinq.eclair.channel.{Data, State, _} import fr.acinq.eclair.payment._ -import fr.acinq.eclair.transactions.Scripts +import fr.acinq.eclair.transactions.{Scripts, Transactions} import fr.acinq.eclair.wire._ import fr.acinq.eclair.{Globals, TestConstants, TestkitBaseClass, randomBytes32} import org.scalatest.Outcome @@ -216,12 +216,12 @@ class ClosingStateSpec extends TestkitBaseClass with StateTestsHelperMethods { channelUpdateListener.expectMsgType[LocalChannelDown] // scenario 1: bob claims the htlc output from the commit tx using its preimage - val claimHtlcSuccessFromCommitTx = Transaction(version = 0, txIn = TxIn(outPoint = OutPoint(randomBytes32, 0), signatureScript = ByteVector.empty, sequence = 0, witness = Scripts.witnessClaimHtlcSuccessFromCommitTx(ByteVector.fill(70)(11), ra1, ByteVector.fill(130)(33))) :: Nil, txOut = Nil, lockTime = 0) + val claimHtlcSuccessFromCommitTx = Transaction(version = 0, txIn = TxIn(outPoint = OutPoint(randomBytes32, 0), signatureScript = ByteVector.empty, sequence = 0, witness = Scripts.witnessClaimHtlcSuccessFromCommitTx(Transactions.PlaceHolderSig, ra1, ByteVector.fill(130)(33))) :: Nil, txOut = Nil, lockTime = 0) alice ! WatchEventSpent(BITCOIN_OUTPUT_SPENT, claimHtlcSuccessFromCommitTx) assert(relayerA.expectMsgType[ForwardFulfill].fulfill === UpdateFulfillHtlc(htlca1.channelId, htlca1.id, ra1)) // scenario 2: bob claims the htlc output from his own commit tx using its preimage (let's assume both parties had published their commitment tx) - val claimHtlcSuccessTx = Transaction(version = 0, txIn = TxIn(outPoint = OutPoint(randomBytes32, 0), signatureScript = ByteVector.empty, sequence = 0, witness = Scripts.witnessHtlcSuccess(ByteVector.fill(70)(11), ByteVector.fill(70)(22), ra1, ByteVector.fill(130)(33))) :: Nil, txOut = Nil, lockTime = 0) + val claimHtlcSuccessTx = Transaction(version = 0, txIn = TxIn(outPoint = OutPoint(randomBytes32, 0), signatureScript = ByteVector.empty, sequence = 0, witness = Scripts.witnessHtlcSuccess(Transactions.PlaceHolderSig, Transactions.PlaceHolderSig, ra1, ByteVector.fill(130)(33))) :: Nil, txOut = Nil, lockTime = 0) alice ! WatchEventSpent(BITCOIN_OUTPUT_SPENT, claimHtlcSuccessTx) assert(relayerA.expectMsgType[ForwardFulfill].fulfill === UpdateFulfillHtlc(htlca1.channelId, htlca1.id, ra1)) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/crypto/GeneratorsSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/crypto/GeneratorsSpec.scala index 705c13de4..e0a81424d 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/crypto/GeneratorsSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/crypto/GeneratorsSpec.scala @@ -16,34 +16,35 @@ package fr.acinq.eclair.crypto -import fr.acinq.bitcoin.Crypto.{Point, Scalar} +import fr.acinq.bitcoin.ByteVector32 +import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey} import org.scalatest.FunSuite import scodec.bits._ class GeneratorsSpec extends FunSuite { - val base_secret: Scalar = hex"000102030405060708090a0b0c0d0e0f101112131415161718191a1b1c1d1e1f" - val per_commitment_secret: Scalar = Scalar(hex"1f1e1d1c1b1a191817161514131211100f0e0d0c0b0a09080706050403020100") - val base_point = Point(hex"036d6caac248af96f6afa7f904f550253a0f3ef3f5aa2fe6838a95b216691468e2") - val per_commitment_point = Point(hex"025f7117a78150fe2ef97db7cfc83bd57b2e2c0d0dd25eaf467a4a1c2a45ce1486") + val base_secret: PrivateKey = PrivateKey(hex"000102030405060708090a0b0c0d0e0f101112131415161718191a1b1c1d1e1f") + val per_commitment_secret: PrivateKey = PrivateKey(hex"1f1e1d1c1b1a191817161514131211100f0e0d0c0b0a09080706050403020100") + val base_point = PublicKey(hex"036d6caac248af96f6afa7f904f550253a0f3ef3f5aa2fe6838a95b216691468e2") + val per_commitment_point = PublicKey(hex"025f7117a78150fe2ef97db7cfc83bd57b2e2c0d0dd25eaf467a4a1c2a45ce1486") test("derivation of key from basepoint and per-commitment-point") { val localKey = Generators.derivePubKey(base_point, per_commitment_point) - assert(localKey.toBin == hex"0235f2dbfaa89b57ec7b055afe29849ef7ddfeb1cefdb9ebdc43f5494984db29e5") + assert(localKey.value == hex"0235f2dbfaa89b57ec7b055afe29849ef7ddfeb1cefdb9ebdc43f5494984db29e5") } test("derivation of secret key from basepoint secret and per-commitment-secret") { val localprivkey = Generators.derivePrivKey(base_secret, per_commitment_point) - assert(localprivkey.toBin == hex"cbced912d3b21bf196a766651e436aff192362621ce317704ea2f75d87e7be0f01") + assert(localprivkey.value == ByteVector32(hex"cbced912d3b21bf196a766651e436aff192362621ce317704ea2f75d87e7be0f")) } test("derivation of revocation key from basepoint and per-commitment-point") { val revocationkey = Generators.revocationPubKey(base_point, per_commitment_point) - assert(revocationkey.toBin == hex"02916e326636d19c33f13e8c0c3a03dd157f332f3e99c317c141dd865eb01f8ff0") + assert(revocationkey.value == hex"02916e326636d19c33f13e8c0c3a03dd157f332f3e99c317c141dd865eb01f8ff0") } test("derivation of revocation secret from basepoint-secret and per-commitment-secret") { val revocationprivkey = Generators.revocationPrivKey(base_secret, per_commitment_secret) - assert(revocationprivkey.toBin == hex"d09ffff62ddb2297ab000cc85bcb4283fdeb6aa052affbc9dddcf33b6107811001") + assert(revocationprivkey.value == ByteVector32(hex"d09ffff62ddb2297ab000cc85bcb4283fdeb6aa052affbc9dddcf33b61078110")) } } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/crypto/LocalKeyManagerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/crypto/LocalKeyManagerSpec.scala index 2d59cc6cc..37a8dc367 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/crypto/LocalKeyManagerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/crypto/LocalKeyManagerSpec.scala @@ -16,7 +16,7 @@ package fr.acinq.eclair.crypto -import fr.acinq.bitcoin.Block +import fr.acinq.bitcoin.{Block, ByteVector32, DeterministicWallet} import fr.acinq.bitcoin.Crypto.PublicKey import fr.acinq.bitcoin.DeterministicWallet.KeyPath import org.scalatest.FunSuite @@ -31,6 +31,23 @@ class LocalKeyManagerSpec extends FunSuite { val keyManager = new LocalKeyManager(seed, Block.TestnetGenesisBlock.hash) assert(keyManager.nodeId == PublicKey(hex"02a051267759c3a149e3e72372f4e0c4054ba597ebfd0eda78a2273023667205ee")) } + + test("generate the same secrets from the same seed") { + // data was generated with eclair 0.3 + val seed = hex"17b086b228025fa8f4416324b6ba2ec36e68570ae2fc3d392520969f2a9d0c1501" + val keyManager = new LocalKeyManager(seed, Block.TestnetGenesisBlock.hash) + assert(keyManager.nodeId == PublicKey(hex"02a051267759c3a149e3e72372f4e0c4054ba597ebfd0eda78a2273023667205ee")) + val keyPath = KeyPath("m/1'/2'/3'/4'") + assert(keyManager.commitmentSecret(keyPath, 0L).value == ByteVector32.fromValidHex("fa7a8c2fc62642f7a9a19ea0bfad14d39a430f3c9899c185dcecc61c8077891e")) + assert(keyManager.commitmentSecret(keyPath, 1L).value == ByteVector32.fromValidHex("3e82338d3e487c760ee10448127613d196b040e86ce90d2d437db6425bb7301c")) + assert(keyManager.commitmentSecret(keyPath, 2L).value == ByteVector32.fromValidHex("102357f7a9b2d0b9147f645c98aa156d3278ddb4745caf0631773dd663e76e6f")) + assert(keyManager.commitmentPoint(keyPath, 0L).value == hex"0x0237dd5a0ea26ed84ed1249d46cc715679b542939d6943b42232e043825cde3944") + assert(DeterministicWallet.encode(keyManager.delayedPaymentPoint(keyPath), DeterministicWallet.tpub) == "tpubDMBn7xW1g1Gsok5eThkJAKJnB3ZFqZQnvsdWv8VvM3RjZkqVPZZpjPDAAmbyDHnZPdAZY8EnFBh1ibTBtiuDqb8t9wRcAZiFihma3yYRG1f") + assert(DeterministicWallet.encode(keyManager.htlcPoint(keyPath), DeterministicWallet.tpub) == "tpubDMBn7xW1g1GsqpsqaVNB1ehpjktQUX44Dycy7fJ6thp774XGzNeWFmQf5L6dVChHREgkoc8BYc2caHqwc2mZzTYCwoxsvrpchBSujsPCvGH") + assert(DeterministicWallet.encode(keyManager.paymentPoint(keyPath), DeterministicWallet.tpub) == "tpubDMBn7xW1g1Gsme9jTAEJwTvizDJtJEgE3jc9vkDqQ9azuh9Es2aM6GsioFiouwdvWPJoNw2zavCkVTMta6UJN6BWR5cMZQsSHvsFyQNfGzv") + assert(DeterministicWallet.encode(keyManager.revocationPoint(keyPath), DeterministicWallet.tpub) == "tpubDMBn7xW1g1GsizhaZ7M4co6sBtUDhRUKgUUPWRv3WfLTpTGYrSjATJy6ZVSoYFCKRnaBop5dFig3Ham1P145NQAKuUgPUbujLAooL7F2vy6") + } + test("generate different node ids from the same seed on different chains") { val seed = hex"17b086b228025fa8f4416324b6ba2ec36e68570ae2fc3d392520969f2a9d0c1501" val keyManager1 = new LocalKeyManager(seed, Block.TestnetGenesisBlock.hash) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/crypto/SphinxSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/crypto/SphinxSpec.scala index a984d7b73..f0b8c42f7 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/crypto/SphinxSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/crypto/SphinxSpec.scala @@ -174,11 +174,11 @@ class SphinxSpec extends FunSuite { object SphinxSpec { val privKeys = Seq( - PrivateKey(hex"4141414141414141414141414141414141414141414141414141414141414141", compressed = true), - PrivateKey(hex"4242424242424242424242424242424242424242424242424242424242424242", compressed = true), - PrivateKey(hex"4343434343434343434343434343434343434343434343434343434343434343", compressed = true), - PrivateKey(hex"4444444444444444444444444444444444444444444444444444444444444444", compressed = true), - PrivateKey(hex"4545454545454545454545454545454545454545454545454545454545454545", compressed = true) + PrivateKey(hex"4141414141414141414141414141414141414141414141414141414141414141"), + PrivateKey(hex"4242424242424242424242424242424242424242424242424242424242424242"), + PrivateKey(hex"4343434343434343434343434343434343434343434343434343434343434343"), + PrivateKey(hex"4444444444444444444444444444444444444444444444444444444444444444"), + PrivateKey(hex"4545454545454545454545454545454545454545454545454545454545454545") ) val publicKeys = privKeys.map(_.publicKey) assert(publicKeys == Seq( @@ -189,7 +189,7 @@ object SphinxSpec { PublicKey(hex"02edabbd16b41c8371b92ef2f04c1185b4f03b6dcd52ba9b78d9d7c89c8f221145") )) - val sessionKey: PrivateKey = PrivateKey(hex"4141414141414141414141414141414141414141414141414141414141414141", compressed = true) + val sessionKey: PrivateKey = PrivateKey(hex"4141414141414141414141414141414141414141414141414141414141414141") val payloads = Seq( hex"000000000000000000000000000000000000000000000000000000000000000000", hex"000101010101010101000000000000000100000001000000000000000000000000", diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/db/ChannelStateSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/db/ChannelStateSpec.scala index 5b1b53f83..4b1bc6427 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/db/ChannelStateSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/db/ChannelStateSpec.scala @@ -18,7 +18,7 @@ package fr.acinq.eclair.db import java.util.UUID -import fr.acinq.bitcoin.Crypto.{PrivateKey, Scalar} +import fr.acinq.bitcoin.Crypto.{PrivateKey} import fr.acinq.bitcoin.{Block, ByteVector32, Crypto, DeterministicWallet, MilliSatoshi, Satoshi, Transaction} import fr.acinq.eclair.channel.Helpers.Funding import fr.acinq.eclair.channel._ @@ -32,6 +32,8 @@ import fr.acinq.eclair.{ShortChannelId, UInt64, randomKey} import org.scalatest.FunSuite import scodec.bits._ +import scala.io.Source + /** * Created by fabrice on 07/02/17. */ @@ -47,6 +49,35 @@ class ChannelStateSpec extends FunSuite { assert(data.commitments.localCommit.spec === check.commitments.localCommit.spec) assert(data === check) } + + test("nonreg") { + val bin = ByteVector.fromValidHex(Source.fromInputStream(getClass.getResourceAsStream("/normal_data_htlcs.bin")).mkString) + val c = ChannelCodecs.stateDataCodec.decode(bin.toBitVector).require.value + + val ref = Seq( + (hex"30440220104aed8d52fe50e2313a9a456607838a0cdac75fdc37afe581c415e7a20da944022034d1ac69c64f34571251be8cc6b50116f26453813267fb9afa3e318a79f4f32401", hex"304502210097fcda40b22916b5d61badedf6126658c2b5927d5002cc2c3e5f88a78ba5f45b02204a74bcf8827d894cab153fc051f39d8e2aeb660162a6a05797f7140587a6133301"), + (hex"30450221009e0e57886b81fd4159f672728891d799203b23c351e93134aba445b288443d3502207b77faa4227126b7d087144c75f4e5c8af9db705b37806dbd2d3f4339666d32201", hex"3045022100aa403fa23e82379a16ba16b446dbdee5a4f879ba690ad3f4f10dc445df2832ba022000a51fdbdb69dcbd5518274303fcece60d719cb6d593e882fdb0190253bbaaab01"), + (hex"3045022100fb44e66fc294d9ece2c33465398221edcfd857208d32a36dedab9906d00b356d022008c5fcfa7b41f8616d57ed009a8614aca636edae1479b6114e03407ba6fceea701", hex"3045022100a9ad65dada5e5500897173bca610135a13008895ce445fbc90d440d5406bd6150220644d75e5ca774ef6b559ffaf1083a9a5da250c3c87666d96daf35b480ef0c65701"), + (hex"3044022009e4f39656dc8712863bffe2acdfa4d2245f65f38a230dd034b226dc2e5fd7ce022049c0108e44c399d1a1b67ab6f60566f491d8682406ac4a03a914f9a21196d6ba01", hex"3044022063a6839c031fd5534d7807a8fff8ca0f9a72d8aa9d78ee104d6ece2b417ac5ce0220193d9b521a44011d31d2bb85be5043119c42a7aee3d9ef68b7388c3c9c3a780501"), + (hex"304402207eaf435948b9e04cb6551f97ee5d85ac879e20d3fae3f5c9a0880ef452d32ac902206e9c5c9098c3e3bef010d3142578823c7fb43b43fe0a0036d481f18a0168b20f01", hex"304402205dda44c9d8aaf37a6f5f6c99713d2a001682f2593a960ccaf5c23059cd20016b02200991b09bccdfc87918852650a4bfa7b4ac9028101362631b5ec376427084138e01"), + (hex"304402200232dbb9d46dabc6569f3f65f4f2a4b7e5acf7be85687b9897141e9784cb9d370220087b2c1dda444d7351976135b56f2f2ca22d8c03d5aa40acbce8c4241daf541501", hex"3045022100eddaa4f767bc70fd672bee983b1644dbff9479def0efc7cca79f0daa1bad370d02204c810238968ae9e86b99d348464e9ac7a06e40225022ae4203ae36fad928c22401"), + (hex"3045022100daa604934db542aa5a9bcbd48eb666fac8acdee92ccd8d42228f52377c51184a022069f855477b27cec39b15fb9e666c09b6c4860c8b80cd1315d2498d97d9cf024601", hex"3044022020e6d43dee03f54574d8245edf2e312d0a492dd2350b7f8df68390b8876de5640220555d46cd545ff0ecc280e6bc82e976ff494bab5f2b128807626753ffb9e5796e01"), + (hex"3044022046c3cf88f9e8639c954c584725482dd6e6403deda3824c37ae95db9bf99d341602206432f76c5ca3d61951155c1b223fd35dd4227f83e1ff9a93437b63515567d23f01", hex"3045022100812a360a6ddc44179f80e5b4252bca74bb5dbe1da25230c9e8afcd388a2fd64702202e45a658123f0263ca1157ef9a9995ede1625d1ecba532957185f1d8044aa1d301"), + (hex"30440220482df018e51b4f684271682bc3c81c481d288d61000a77df2126afe042c3471d02204772720ff1ea323a271259022a0458ae4d228e5f613ade63fca38eb5d443756a01", hex"3044022076a338d225b8954412198ce5936aaa6433da1f51dd9bcbe69d95a1e0960c169802207db267517fc73e358e09f4c89313ae17ed4d5f6d8432faec9ec1e784a2a7da7c01"), + (hex"3045022100916255b5758d66cd46f653f8a5a71b1c857bfae1a7cf85195b5d78476c4138c502200101e3ec9874aa2644691662bf8945a182af1237bb61c459e9dbff495b9097d001", hex"304402201d099a464a7696b22a8e58b65c52e9a519a06a5c49e944155d4e5fbd14d3f5b902203c091c0ec5b840a80be739d29b5fc2c75cb94928e5ea83133f84d226f28cd4b701"), + (hex"3045022100d8eaa436faec6b38f155065893f1212ce43615fbec49b4af72f16119774b5472022033aa303992f4a8cfe1c77e6a0d2baa73baad0305a88da16d26122e536867431101", hex"304402203af7b7ea16cc018fdb414f52cd38ed548dc257cbb06c812c9dc1d60500b21485022072cd74b7e49bfd813e09bae778da903b44b7b0ae22b87af4c34cf8bb77dfdef201"), + (hex"304402204f5dd042bfb449c522012a2d461e5a94c9ea3be629c0ab091b0e1f5569eb119c022021411ff8affabab12cd39f0eaa64f1b08fa72ada6f37d1d46c6bde4483d869fb01", hex"3044022043573edb37be815d1b97b90803f601dfc91c25279ccda606ad6515fee721fe57022030ac2883408a2075a47337443eb539062a8ac6b5453befb2b9863d697e35dd8201"), + (hex"3044022030ff3d4d42ef1c3d742164a30ff7b021215e881d9277a52a1720514a4473289502204b090f6b412e8caacb5bcbf295babb075d9d5490e3f7678c289206780f6f0bc901", hex"304502210093fd7dfa3ef6cdf5b94cfadf83022be98062d53cd7097a73947453b210a481eb0220622e63a21b787ea7bb55f01ab6fe503fcb8ef4cb65adce7a264ae014403646fe01") + ) + + val sigs = c.commitments + .localCommit + .publishableTxs + .htlcTxsAndSigs + .map(data => (Scripts.der(data.localSig), Scripts.der(data.remoteSig))) + + assert(ref === sigs) + } } object ChannelStateSpec { @@ -74,10 +105,10 @@ object ChannelStateSpec { toSelfDelay = 144, maxAcceptedHtlcs = 50, fundingPubKey = PrivateKey(ByteVector32(ByteVector.fill(32)(1)) :+ 1.toByte).publicKey, - revocationBasepoint = Scalar(ByteVector.fill(32)(2)).toPoint, - paymentBasepoint = Scalar(ByteVector.fill(32)(3)).toPoint, - delayedPaymentBasepoint = Scalar(ByteVector.fill(32)(4)).toPoint, - htlcBasepoint = Scalar(ByteVector.fill(32)(6)).toPoint, + revocationBasepoint = PrivateKey(ByteVector.fill(32)(2)).publicKey, + paymentBasepoint = PrivateKey(ByteVector.fill(32)(3)).publicKey, + delayedPaymentBasepoint = PrivateKey(ByteVector.fill(32)(4)).publicKey, + htlcBasepoint = PrivateKey(ByteVector.fill(32)(6)).publicKey, globalFeatures = hex"dead", localFeatures = hex"beef") @@ -102,7 +133,7 @@ object ChannelStateSpec { val commitmentInput = Funding.makeFundingInputInfo(fundingTx.hash, 0, fundingAmount, keyManager.fundingPublicKey(localParams.channelKeyPath).publicKey, remoteParams.fundingPubKey) val localCommit = LocalCommit(0, CommitmentSpec(htlcs.toSet, 1500, 50000000, 70000000), PublishableTxs(CommitTx(commitmentInput, Transaction(2, Nil, Nil, 0)), Nil)) - val remoteCommit = RemoteCommit(0, CommitmentSpec(htlcs.map(htlc => htlc.copy(direction = htlc.direction.opposite)).toSet, 1500, 50000, 700000), ByteVector32(hex"0303030303030303030303030303030303030303030303030303030303030303"), Scalar(ByteVector.fill(32)(4)).toPoint) + val remoteCommit = RemoteCommit(0, CommitmentSpec(htlcs.map(htlc => htlc.copy(direction = htlc.direction.opposite)).toSet, 1500, 50000, 700000), ByteVector32(hex"0303030303030303030303030303030303030303030303030303030303030303"), PrivateKey(ByteVector.fill(32)(4)).publicKey) val commitments = Commitments(localParams, remoteParams, channelFlags = 0x01.toByte, localCommit, remoteCommit, LocalChanges(Nil, Nil, Nil), RemoteChanges(Nil, Nil, Nil), localNextHtlcId = 32L, remoteNextHtlcId = 4L, diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/db/SqliteNetworkDbSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/db/SqliteNetworkDbSpec.scala index f94dcabe6..201cff789 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/db/SqliteNetworkDbSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/db/SqliteNetworkDbSpec.scala @@ -64,7 +64,7 @@ class SqliteNetworkDbSpec extends FunSuite { val sqlite = TestConstants.sqliteInMemory() val db = new SqliteNetworkDb(sqlite) - def sig = Crypto.encodeSignature(Crypto.sign(randomBytes32, randomKey)) :+ 1.toByte + def sig = Crypto.sign(randomBytes32, randomKey) val channel_1 = Announcements.makeChannelAnnouncement(Block.RegtestGenesisBlock.hash, ShortChannelId(42), randomKey.publicKey, randomKey.publicKey, randomKey.publicKey, randomKey.publicKey, sig, sig, sig, sig) val channel_2 = Announcements.makeChannelAnnouncement(Block.RegtestGenesisBlock.hash, ShortChannelId(43), randomKey.publicKey, randomKey.publicKey, randomKey.publicKey, randomKey.publicKey, sig, sig, sig, sig) @@ -104,7 +104,7 @@ class SqliteNetworkDbSpec extends FunSuite { test("remove many channels") { val sqlite = TestConstants.sqliteInMemory() val db = new SqliteNetworkDb(sqlite) - val sig = Crypto.encodeSignature(Crypto.sign(randomBytes32, randomKey)) :+ 1.toByte + val sig = Crypto.sign(randomBytes32, randomKey) val priv = randomKey val pub = priv.publicKey val capacity = Satoshi(10000) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/integration/IntegrationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/integration/IntegrationSpec.scala index d2e56fa2c..53b1fc33a 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/integration/IntegrationSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/integration/IntegrationSpec.scala @@ -879,7 +879,7 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService sender.expectMsgType[JValue](10 seconds) logger.info(s"simulated ${channels.size} channels") - val remoteNodeId = PrivateKey(ByteVector32(ByteVector.fill(32)(1)), true).publicKey + val remoteNodeId = PrivateKey(ByteVector32(ByteVector.fill(32)(1))).publicKey // then we make the announcements val announcements = channels.map(c => AnnouncementsBatchValidationSpec.makeChannelAnnouncement(c)) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/io/PeerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/io/PeerSpec.scala index 157c6212a..87ff5c078 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/io/PeerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/io/PeerSpec.scala @@ -31,7 +31,6 @@ import fr.acinq.eclair.db.ChannelStateSpec import fr.acinq.eclair.io.Peer._ import fr.acinq.eclair.router.RoutingSyncSpec.makeFakeRoutingInfo import fr.acinq.eclair.router.{ChannelRangeQueries, ChannelRangeQueriesSpec, Rebroadcast} -import fr.acinq.eclair.wire.LightningMessageCodecsSpec.randomSignature import fr.acinq.eclair.wire.{Color, Error, IPv4, NodeAddress, NodeAnnouncement, Ping, Pong} import org.scalatest.{Outcome, Tag} import scodec.bits.ByteVector @@ -55,7 +54,7 @@ class PeerSpec extends TestkitBaseClass { val aParams = Alice.nodeParams val aliceParams = test.tags.contains("with_node_announcements") match { case true => - val aliceAnnouncement = NodeAnnouncement(randomSignature, ByteVector.empty, 1, Bob.nodeParams.nodeId, Color(100.toByte, 200.toByte, 300.toByte), "node-alias", fakeIPAddress :: Nil) + val aliceAnnouncement = NodeAnnouncement(randomBytes64, ByteVector.empty, 1, Bob.nodeParams.nodeId, Color(100.toByte, 200.toByte, 300.toByte), "node-alias", fakeIPAddress :: Nil) aParams.db.network.addNode(aliceAnnouncement) aParams case false => aParams diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/io/PeerSpecWithLogging.scala b/eclair-core/src/test/scala/fr/acinq/eclair/io/PeerSpecWithLogging.scala index 265ccf826..7a3e02a6c 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/io/PeerSpecWithLogging.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/io/PeerSpecWithLogging.scala @@ -5,11 +5,12 @@ import akka.testkit.{EventFilter, TestFSMRef, TestKit, TestProbe} import com.typesafe.config.ConfigFactory import fr.acinq.eclair.db.ChannelStateSpec import org.scalatest.{FunSuiteLike, Outcome, Tag} + import scala.concurrent.duration._ import akka.testkit.{TestFSMRef, TestProbe} import fr.acinq.eclair.TestConstants.{Alice, Bob} import fr.acinq.eclair.blockchain.EclairWallet -import fr.acinq.eclair.wire.LightningMessageCodecsSpec.randomSignature +import fr.acinq.eclair.randomBytes64 import fr.acinq.eclair.wire.{Color, IPv4, NodeAddress, NodeAnnouncement} import scodec.bits.ByteVector @@ -19,7 +20,7 @@ class PeerSpecWithLogging extends TestKit(ActorSystem("test", ConfigFactory.pars test("reconnect using the address from node_announcement") { val aliceParams = Alice.nodeParams - val aliceAnnouncement = NodeAnnouncement(randomSignature, ByteVector.empty, 1, Bob.nodeParams.nodeId, Color(100.toByte, 200.toByte, 300.toByte), "node-alias", fakeIPAddress :: Nil) + val aliceAnnouncement = NodeAnnouncement(randomBytes64, ByteVector.empty, 1, Bob.nodeParams.nodeId, Color(100.toByte, 200.toByte, 300.toByte), "node-alias", fakeIPAddress :: Nil) aliceParams.db.network.addNode(aliceAnnouncement) val authenticator = TestProbe() val watcher = TestProbe() diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/HtlcGenerationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/HtlcGenerationSpec.scala index b1e737717..fb4fb7718 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/HtlcGenerationSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/HtlcGenerationSpec.scala @@ -155,7 +155,7 @@ object HtlcGenerationSpec { val (priv_a, priv_b, priv_c, priv_d, priv_e) = (TestConstants.Alice.keyManager.nodeKey, TestConstants.Bob.keyManager.nodeKey, randomExtendedPrivateKey, randomExtendedPrivateKey, randomExtendedPrivateKey) val (a, b, c, d, e) = (priv_a.publicKey, priv_b.publicKey, priv_c.publicKey, priv_d.publicKey, priv_e.publicKey) - val sig = Crypto.encodeSignature(Crypto.sign(Crypto.sha256(ByteVector.empty), priv_a.privateKey)) :+ 1.toByte + val sig = Crypto.sign(Crypto.sha256(ByteVector.empty), priv_a.privateKey) val defaultChannelUpdate = ChannelUpdate(sig, Block.RegtestGenesisBlock.hash, ShortChannelId(0), 0, 1, 0, 0, 42000, 0, 0, Some(500000000L)) val channelUpdate_ab = defaultChannelUpdate.copy(shortChannelId = ShortChannelId(1), cltvExpiryDelta = 4, feeBaseMsat = 642000, feeProportionalMillionths = 7) val channelUpdate_bc = defaultChannelUpdate.copy(shortChannelId = ShortChannelId(2), cltvExpiryDelta = 5, feeBaseMsat = 153000, feeProportionalMillionths = 4) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentRequestSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentRequestSpec.scala index 96a633557..9acd85b61 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentRequestSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentRequestSpec.scala @@ -32,7 +32,7 @@ import scodec.bits._ class PaymentRequestSpec extends FunSuite { - val priv = PrivateKey(hex"e126f68f7eafcc8b74f54d269fe206be715000f94dac067d1c04a8ca3b2db734", compressed = true) + val priv = PrivateKey(hex"e126f68f7eafcc8b74f54d269fe206be715000f94dac067d1c04a8ca3b2db734") val pub = priv.publicKey val nodeId = pub assert(nodeId == PublicKey(hex"03e7156ae33b0a208d0744199163177e909e80176e55d97a2f221ede0f934dd9ad")) 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 a05a0aba2..fc63698bb 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 @@ -44,7 +44,7 @@ abstract class BaseRouterSpec extends TestkitBaseClass { case class FixtureParam(router: ActorRef, watcher: TestProbe) - val remoteNodeId = PrivateKey(ByteVector32(ByteVector.fill(32)(1)), compressed = true).publicKey + val remoteNodeId = PrivateKey(ByteVector32(ByteVector.fill(32)(1))).publicKey val seed = ByteVector32(ByteVector.fill(32)(2)) val testKeyManager = new LocalKeyManager(seed, Block.RegtestGenesisBlock.hash) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/router/RouteCalculationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/router/RouteCalculationSpec.scala index 7c5e1d4c9..2ff00c494 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/router/RouteCalculationSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/router/RouteCalculationSpec.scala @@ -17,11 +17,12 @@ package fr.acinq.eclair.router import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey} -import fr.acinq.bitcoin.{Block, ByteVector32, Crypto} +import fr.acinq.bitcoin.{Block, ByteVector32, ByteVector64, Crypto} import fr.acinq.eclair.payment.PaymentRequest.ExtraHop import fr.acinq.eclair.router.Graph.GraphStructure.DirectedGraph.graphEdgeToHop import fr.acinq.eclair.router.Graph.GraphStructure.{DirectedGraph, GraphEdge} import fr.acinq.eclair.router.Graph.{RichWeight, WeightRatios} +import fr.acinq.eclair.transactions.Transactions import fr.acinq.eclair.wire._ import fr.acinq.eclair.{Globals, ShortChannelId, randomKey} import org.scalatest.FunSuite @@ -333,16 +334,9 @@ class RouteCalculationSpec extends FunSuite { assert(route2.map(hops2Ids) === Failure(RouteNotFound)) } - test("compute an example sig") { - val data = ByteVector32.Zeroes - val key = PrivateKey(ByteVector32(ByteVector.fill(32)(1))) - val sig = Crypto.encodeSignature(Crypto.sign(data, key)) - assert(Crypto.isDERSignature(sig :+ 1.toByte)) - } - test("calculate route and return metadata") { - val DUMMY_SIG = hex"3045022100e0a180fdd0fe38037cc878c03832861b40a29d32bd7b40b10c9e1efc8c1468a002205ae06d1624896d0d29f4b31e32772ea3cb1b4d7ed4e077e5da28dcc33c0e781201" + val DUMMY_SIG = Transactions.PlaceHolderSig val uab = ChannelUpdate(DUMMY_SIG, Block.RegtestGenesisBlock.hash, ShortChannelId(1L), 0L, 0, 0, 1, 42, 2500, 140, None) val uba = ChannelUpdate(DUMMY_SIG, Block.RegtestGenesisBlock.hash, ShortChannelId(1L), 1L, 0, 1, 1, 43, 2501, 141, None) @@ -861,12 +855,12 @@ class RouteCalculationSpec extends FunSuite { // This test have a channel (542280x2156x0) that according to heuristics is very convenient but actually useless to reach the target, // then if the cost function is not monotonic the path-finding breaks because the result path contains a loop. val updates = List( - ChannelDesc(ShortChannelId("565643x1216x0"), PublicKey(hex"03864ef025fde8fb587d989186ce6a4a186895ee44a926bfc370e2c366597a3f8f"), PublicKey(hex"024655b768ef40951b20053a5c4b951606d4d86085d51238f2c67c7dec29c792ca")) -> ChannelUpdate(ByteVector32.Zeroes.bytes, ByteVector32.Zeroes, ShortChannelId("565643x1216x0"), 0, 1.toByte, 1.toByte, 144, htlcMinimumMsat = 0, feeBaseMsat = 1000, 100, Some(15000000000L)), - ChannelDesc(ShortChannelId("565643x1216x0"), PublicKey(hex"024655b768ef40951b20053a5c4b951606d4d86085d51238f2c67c7dec29c792ca"), PublicKey(hex"03864ef025fde8fb587d989186ce6a4a186895ee44a926bfc370e2c366597a3f8f")) -> ChannelUpdate(ByteVector32.Zeroes.bytes, ByteVector32.Zeroes, ShortChannelId("565643x1216x0"), 0, 1.toByte, 0.toByte, 14, htlcMinimumMsat = 1, 1000, 10, Some(4294967295L)), - ChannelDesc(ShortChannelId("542280x2156x0"), PublicKey(hex"03864ef025fde8fb587d989186ce6a4a186895ee44a926bfc370e2c366597a3f8f"), PublicKey(hex"03cb7983dc247f9f81a0fa2dfa3ce1c255365f7279c8dd143e086ca333df10e278")) -> ChannelUpdate(ByteVector32.Zeroes.bytes, ByteVector32.Zeroes, ShortChannelId("542280x2156x0"), 0, 1.toByte, 1.toByte, 144, htlcMinimumMsat = 1000, feeBaseMsat = 1000, 100, Some(16777000000L)), - ChannelDesc(ShortChannelId("542280x2156x0"), PublicKey(hex"03cb7983dc247f9f81a0fa2dfa3ce1c255365f7279c8dd143e086ca333df10e278"), PublicKey(hex"03864ef025fde8fb587d989186ce6a4a186895ee44a926bfc370e2c366597a3f8f")) -> ChannelUpdate(ByteVector32.Zeroes.bytes, ByteVector32.Zeroes, ShortChannelId("542280x2156x0"), 0, 1.toByte, 0.toByte, 144, htlcMinimumMsat = 1, 667, 1, Some(16777000000L)), - ChannelDesc(ShortChannelId("565779x2711x0"), PublicKey(hex"03864ef025fde8fb587d989186ce6a4a186895ee44a926bfc370e2c366597a3f8f"), PublicKey(hex"036d65409c41ab7380a43448f257809e7496b52bf92057c09c4f300cbd61c50d96")) -> ChannelUpdate(ByteVector32.Zeroes.bytes, ByteVector32.Zeroes, ShortChannelId("565779x2711x0"), 0, 1.toByte, 3.toByte, 144, htlcMinimumMsat = 1, 1000, 100, Some(230000000L)), - ChannelDesc(ShortChannelId("565779x2711x0"), PublicKey(hex"036d65409c41ab7380a43448f257809e7496b52bf92057c09c4f300cbd61c50d96"), PublicKey(hex"03864ef025fde8fb587d989186ce6a4a186895ee44a926bfc370e2c366597a3f8f")) -> ChannelUpdate(ByteVector32.Zeroes.bytes, ByteVector32.Zeroes, ShortChannelId("565779x2711x0"), 0, 1.toByte, 0.toByte, 144, htlcMinimumMsat = 1, 1000, 100, Some(230000000L)) + ChannelDesc(ShortChannelId("565643x1216x0"), PublicKey(hex"03864ef025fde8fb587d989186ce6a4a186895ee44a926bfc370e2c366597a3f8f"), PublicKey(hex"024655b768ef40951b20053a5c4b951606d4d86085d51238f2c67c7dec29c792ca")) -> ChannelUpdate(ByteVector64.Zeroes, ByteVector32.Zeroes, ShortChannelId("565643x1216x0"), 0, 1.toByte, 1.toByte, 144, htlcMinimumMsat = 0, feeBaseMsat = 1000, 100, Some(15000000000L)), + ChannelDesc(ShortChannelId("565643x1216x0"), PublicKey(hex"024655b768ef40951b20053a5c4b951606d4d86085d51238f2c67c7dec29c792ca"), PublicKey(hex"03864ef025fde8fb587d989186ce6a4a186895ee44a926bfc370e2c366597a3f8f")) -> ChannelUpdate(ByteVector64.Zeroes, ByteVector32.Zeroes, ShortChannelId("565643x1216x0"), 0, 1.toByte, 0.toByte, 14, htlcMinimumMsat = 1, 1000, 10, Some(4294967295L)), + ChannelDesc(ShortChannelId("542280x2156x0"), PublicKey(hex"03864ef025fde8fb587d989186ce6a4a186895ee44a926bfc370e2c366597a3f8f"), PublicKey(hex"03cb7983dc247f9f81a0fa2dfa3ce1c255365f7279c8dd143e086ca333df10e278")) -> ChannelUpdate(ByteVector64.Zeroes, ByteVector32.Zeroes, ShortChannelId("542280x2156x0"), 0, 1.toByte, 1.toByte, 144, htlcMinimumMsat = 1000, feeBaseMsat = 1000, 100, Some(16777000000L)), + ChannelDesc(ShortChannelId("542280x2156x0"), PublicKey(hex"03cb7983dc247f9f81a0fa2dfa3ce1c255365f7279c8dd143e086ca333df10e278"), PublicKey(hex"03864ef025fde8fb587d989186ce6a4a186895ee44a926bfc370e2c366597a3f8f")) -> ChannelUpdate(ByteVector64.Zeroes, ByteVector32.Zeroes, ShortChannelId("542280x2156x0"), 0, 1.toByte, 0.toByte, 144, htlcMinimumMsat = 1, 667, 1, Some(16777000000L)), + ChannelDesc(ShortChannelId("565779x2711x0"), PublicKey(hex"03864ef025fde8fb587d989186ce6a4a186895ee44a926bfc370e2c366597a3f8f"), PublicKey(hex"036d65409c41ab7380a43448f257809e7496b52bf92057c09c4f300cbd61c50d96")) -> ChannelUpdate(ByteVector64.Zeroes, ByteVector32.Zeroes, ShortChannelId("565779x2711x0"), 0, 1.toByte, 3.toByte, 144, htlcMinimumMsat = 1, 1000, 100, Some(230000000L)), + ChannelDesc(ShortChannelId("565779x2711x0"), PublicKey(hex"036d65409c41ab7380a43448f257809e7496b52bf92057c09c4f300cbd61c50d96"), PublicKey(hex"03864ef025fde8fb587d989186ce6a4a186895ee44a926bfc370e2c366597a3f8f")) -> ChannelUpdate(ByteVector64.Zeroes, ByteVector32.Zeroes, ShortChannelId("565779x2711x0"), 0, 1.toByte, 0.toByte, 144, htlcMinimumMsat = 1, 1000, 100, Some(230000000L)) ).toMap val g = DirectedGraph.makeGraph(updates) @@ -895,7 +889,7 @@ object RouteCalculationSpec { val DEFAULT_ROUTE_PARAMS = RouteParams(randomize = false, maxFeeBaseMsat = 21000, maxFeePct = 0.03, routeMaxCltv = 2016, routeMaxLength = 6, ratios = None) - val DUMMY_SIG = hex"3045022100e0a180fdd0fe38037cc878c03832861b40a29d32bd7b40b10c9e1efc8c1468a002205ae06d1624896d0d29f4b31e32772ea3cb1b4d7ed4e077e5da28dcc33c0e781201" + val DUMMY_SIG = Transactions.PlaceHolderSig def makeChannel(shortChannelId: Long, nodeIdA: PublicKey, nodeIdB: PublicKey) = { val (nodeId1, nodeId2) = if (Announcements.isNode1(nodeIdA, nodeIdB)) (nodeIdA, nodeIdB) else (nodeIdB, nodeIdA) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/transactions/ClaimReceivedHtlcSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/transactions/ClaimReceivedHtlcSpec.scala index 6c18c88dd..37aa3d161 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/transactions/ClaimReceivedHtlcSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/transactions/ClaimReceivedHtlcSpec.scala @@ -16,7 +16,7 @@ package fr.acinq.eclair.transactions -import fr.acinq.bitcoin.Crypto.PrivateKey +import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey, ripemd160} import fr.acinq.bitcoin._ import fr.acinq.eclair.transactions.Scripts._ import org.scalatest.FunSuite @@ -26,8 +26,8 @@ import scodec.bits.ByteVector class ClaimReceivedHtlcSpec extends FunSuite { object Alice { - val commitKey = PrivateKey.fromBase58("cVuzKWCszfvjkoJyUasvsrRdECriz8hSd1BDinRNzytwnXmX7m1g", Base58.Prefix.SecretKeyTestnet) - val finalKey = PrivateKey.fromBase58("cRUfvpbRtMSqCFD1ADdvgPn5HfRLYuHCFYAr2noWnaRDNger2AoA", Base58.Prefix.SecretKeyTestnet) + val (commitKey, true) = PrivateKey.fromBase58("cVuzKWCszfvjkoJyUasvsrRdECriz8hSd1BDinRNzytwnXmX7m1g", Base58.Prefix.SecretKeyTestnet) + val (finalKey, true) = PrivateKey.fromBase58("cRUfvpbRtMSqCFD1ADdvgPn5HfRLYuHCFYAr2noWnaRDNger2AoA", Base58.Prefix.SecretKeyTestnet) val commitPubKey = commitKey.publicKey val finalPubKey = finalKey.publicKey val R = Crypto.sha256(ByteVector.view("this is Alice's R".getBytes("UTF-8"))) @@ -38,8 +38,8 @@ class ClaimReceivedHtlcSpec extends FunSuite { } object Bob { - val commitKey = PrivateKey.fromBase58("cSupnaiBh6jgTcQf9QANCB5fZtXojxkJQczq5kwfSBeULjNd5Ypo", Base58.Prefix.SecretKeyTestnet) - val finalKey = PrivateKey.fromBase58("cQLk5fMydgVwJjygt9ta8GcUU4GXLumNiXJCQviibs2LE5vyMXey", Base58.Prefix.SecretKeyTestnet) + val (commitKey, true) = PrivateKey.fromBase58("cSupnaiBh6jgTcQf9QANCB5fZtXojxkJQczq5kwfSBeULjNd5Ypo", Base58.Prefix.SecretKeyTestnet) + val (finalKey, true) = PrivateKey.fromBase58("cQLk5fMydgVwJjygt9ta8GcUU4GXLumNiXJCQviibs2LE5vyMXey", Base58.Prefix.SecretKeyTestnet) val commitPubKey = commitKey.publicKey val finalPubKey = finalKey.publicKey val R = Crypto.sha256(ByteVector.view("this is Bob's R".getBytes("UTF-8"))) @@ -50,6 +50,26 @@ class ClaimReceivedHtlcSpec extends FunSuite { val revokeCommitH = Crypto.sha256(revokeCommit) } + def scriptPubKeyHtlcReceive(ourkey: PublicKey, theirkey: PublicKey, abstimeout: Long, reltimeout: Long, rhash: ByteVector32, commit_revoke: ByteVector): Seq[ScriptElt] = { + // values lesser than 16 should be encoded using OP_0..OP_16 instead of OP_PUSHDATA + require(abstimeout > 16, s"abstimeout=$abstimeout must be greater than 16") + // @formatter:off + OP_SIZE :: encodeNumber(32) :: OP_EQUALVERIFY :: + OP_HASH160 :: OP_DUP :: + OP_PUSHDATA(ripemd160(rhash)) :: OP_EQUAL :: + OP_IF :: + encodeNumber(reltimeout) :: OP_CHECKSEQUENCEVERIFY :: OP_2DROP :: OP_PUSHDATA(ourkey) :: + OP_ELSE :: + OP_PUSHDATA(ripemd160(commit_revoke)) :: OP_EQUAL :: + OP_NOTIF :: + encodeNumber(abstimeout) :: OP_CHECKLOCKTIMEVERIFY :: OP_DROP :: + OP_ENDIF :: + OP_PUSHDATA(theirkey) :: + OP_ENDIF :: + OP_CHECKSIG :: Nil + // @formatter:on + } + val abstimeout = 3000 val reltimeout = 2000 val htlcScript = scriptPubKeyHtlcReceive(Alice.finalPubKey, Bob.finalPubKey, abstimeout, reltimeout, Bob.Rhash, Bob.revokeCommitRHash) @@ -66,14 +86,14 @@ class ClaimReceivedHtlcSpec extends FunSuite { val tx1 = Transaction( version = 2, txIn = TxIn(OutPoint(tx, 0), ByteVector.empty, 0xffffffff) :: Nil, - txOut = TxOut(10 satoshi, OP_DUP :: OP_HASH160 :: OP_PUSHDATA(Crypto.hash160(Alice.finalPubKey.toBin)) :: OP_EQUALVERIFY :: OP_CHECKSIG :: Nil) :: Nil, + txOut = TxOut(10 satoshi, OP_DUP :: OP_HASH160 :: OP_PUSHDATA(Crypto.hash160(Alice.finalPubKey.value)) :: OP_EQUALVERIFY :: OP_CHECKSIG :: Nil) :: Nil, lockTime = 0) test("Alice can spend this HTLC after a delay if she knows the payment hash") { val tx2 = Transaction( version = 2, txIn = TxIn(OutPoint(tx, 0), ByteVector.empty, reltimeout + 1) :: Nil, - txOut = TxOut(10 satoshi, OP_DUP :: OP_HASH160 :: OP_PUSHDATA(Crypto.hash160(Alice.finalPubKey.toBin)) :: OP_EQUALVERIFY :: OP_CHECKSIG :: Nil) :: Nil, + txOut = TxOut(10 satoshi, OP_DUP :: OP_HASH160 :: OP_PUSHDATA(Crypto.hash160(Alice.finalPubKey.value)) :: OP_EQUALVERIFY :: OP_CHECKSIG :: Nil) :: Nil, lockTime = abstimeout + 1) val sig = Transaction.signInput(tx2, 0, Script.write(htlcScript), SIGHASH_ALL, tx.txOut(0).amount, 1, Alice.finalKey) @@ -87,7 +107,7 @@ class ClaimReceivedHtlcSpec extends FunSuite { val tx2 = Transaction( version = 2, txIn = TxIn(OutPoint(tx, 0), ByteVector.empty, reltimeout + 1) :: Nil, - txOut = TxOut(10 satoshi, OP_DUP :: OP_HASH160 :: OP_PUSHDATA(Crypto.hash160(Bob.finalPubKey.toBin)) :: OP_EQUALVERIFY :: OP_CHECKSIG :: Nil) :: Nil, + txOut = TxOut(10 satoshi, OP_DUP :: OP_HASH160 :: OP_PUSHDATA(Crypto.hash160(Bob.finalPubKey.value)) :: OP_EQUALVERIFY :: OP_CHECKSIG :: Nil) :: Nil, lockTime = abstimeout + 1) val sig = Transaction.signInput(tx2, 0, Script.write(htlcScript), SIGHASH_ALL, tx.txOut(0).amount, 1, Bob.finalKey) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/transactions/ClaimSentHtlcSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/transactions/ClaimSentHtlcSpec.scala index 0f9285efc..824aa1cc5 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/transactions/ClaimSentHtlcSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/transactions/ClaimSentHtlcSpec.scala @@ -16,7 +16,7 @@ package fr.acinq.eclair.transactions -import fr.acinq.bitcoin.Crypto.PrivateKey +import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey, ripemd160} import fr.acinq.bitcoin._ import fr.acinq.eclair.transactions.Scripts._ import org.scalatest.FunSuite @@ -26,8 +26,8 @@ import scodec.bits.ByteVector class ClaimSentHtlcSpec extends FunSuite { object Alice { - val commitKey = PrivateKey.fromBase58("cVuzKWCszfvjkoJyUasvsrRdECriz8hSd1BDinRNzytwnXmX7m1g", Base58.Prefix.SecretKeyTestnet) - val finalKey = PrivateKey.fromBase58("cRUfvpbRtMSqCFD1ADdvgPn5HfRLYuHCFYAr2noWnaRDNger2AoA", Base58.Prefix.SecretKeyTestnet) + val (commitKey, true) = PrivateKey.fromBase58("cVuzKWCszfvjkoJyUasvsrRdECriz8hSd1BDinRNzytwnXmX7m1g", Base58.Prefix.SecretKeyTestnet) + val (finalKey, true) = PrivateKey.fromBase58("cRUfvpbRtMSqCFD1ADdvgPn5HfRLYuHCFYAr2noWnaRDNger2AoA", Base58.Prefix.SecretKeyTestnet) val commitPubKey = commitKey.publicKey val finalPubKey = finalKey.publicKey val R = Crypto.sha256(ByteVector.view("this is Alice's R".getBytes("UTF-8"))) @@ -39,8 +39,8 @@ class ClaimSentHtlcSpec extends FunSuite { } object Bob { - val commitKey = PrivateKey.fromBase58("cSupnaiBh6jgTcQf9QANCB5fZtXojxkJQczq5kwfSBeULjNd5Ypo", Base58.Prefix.SecretKeyTestnet) - val finalKey = PrivateKey.fromBase58("cQLk5fMydgVwJjygt9ta8GcUU4GXLumNiXJCQviibs2LE5vyMXey", Base58.Prefix.SecretKeyTestnet) + val (commitKey, true) = PrivateKey.fromBase58("cSupnaiBh6jgTcQf9QANCB5fZtXojxkJQczq5kwfSBeULjNd5Ypo", Base58.Prefix.SecretKeyTestnet) + val (finalKey, true) = PrivateKey.fromBase58("cQLk5fMydgVwJjygt9ta8GcUU4GXLumNiXJCQviibs2LE5vyMXey", Base58.Prefix.SecretKeyTestnet) val commitPubKey = commitKey.publicKey val finalPubKey = finalKey.publicKey val R: ByteVector = Crypto.sha256(ByteVector.view("this is Bob's R".getBytes("UTF-8"))) @@ -51,6 +51,23 @@ class ClaimSentHtlcSpec extends FunSuite { val revokeCommitH: ByteVector = Crypto.sha256(revokeCommit) } + def scriptPubKeyHtlcSend(ourkey: PublicKey, theirkey: PublicKey, abstimeout: Long, reltimeout: Long, rhash: ByteVector32, commit_revoke: ByteVector): Seq[ScriptElt] = { + // values lesser than 16 should be encoded using OP_0..OP_16 instead of OP_PUSHDATA + require(abstimeout > 16, s"abstimeout=$abstimeout must be greater than 16") + // @formatter:off + OP_SIZE :: encodeNumber(32) :: OP_EQUALVERIFY :: + OP_HASH160 :: OP_DUP :: + OP_PUSHDATA(ripemd160(rhash)) :: OP_EQUAL :: + OP_SWAP :: OP_PUSHDATA(ripemd160(commit_revoke)) :: OP_EQUAL :: OP_ADD :: + OP_IF :: + OP_PUSHDATA(theirkey) :: + OP_ELSE :: + encodeNumber(abstimeout) :: OP_CHECKLOCKTIMEVERIFY :: encodeNumber(reltimeout) :: OP_CHECKSEQUENCEVERIFY :: OP_2DROP :: OP_PUSHDATA(ourkey) :: + OP_ENDIF :: + OP_CHECKSIG :: Nil + // @formatter:on + } + val abstimeout = 3000 val reltimeout = 2000 val htlcScript = scriptPubKeyHtlcSend(Alice.finalPubKey, Bob.finalPubKey, abstimeout, reltimeout, Alice.revokeCommitRHash, Alice.Rhash) @@ -67,14 +84,14 @@ class ClaimSentHtlcSpec extends FunSuite { val tx1 = Transaction( version = 2, txIn = TxIn(OutPoint(tx, 0), ByteVector.empty, 0xffffffff) :: Nil, - txOut = TxOut(10 satoshi, OP_DUP :: OP_HASH160 :: OP_PUSHDATA(Crypto.hash160(Alice.finalPubKey.toBin)) :: OP_EQUALVERIFY :: OP_CHECKSIG :: Nil) :: Nil, + txOut = TxOut(10 satoshi, OP_DUP :: OP_HASH160 :: OP_PUSHDATA(Crypto.hash160(Alice.finalPubKey.value)) :: OP_EQUALVERIFY :: OP_CHECKSIG :: Nil) :: Nil, lockTime = 0) test("Alice can spend this HTLC after a delay") { val tx2 = Transaction( version = 2, txIn = TxIn(OutPoint(tx, 0), ByteVector.empty, sequence = reltimeout + 1) :: Nil, - txOut = TxOut(10 satoshi, OP_DUP :: OP_HASH160 :: OP_PUSHDATA(Crypto.hash160(Alice.finalPubKey.toBin)) :: OP_EQUALVERIFY :: OP_CHECKSIG :: Nil) :: Nil, + txOut = TxOut(10 satoshi, OP_DUP :: OP_HASH160 :: OP_PUSHDATA(Crypto.hash160(Alice.finalPubKey.value)) :: OP_EQUALVERIFY :: OP_CHECKSIG :: Nil) :: Nil, lockTime = abstimeout + 1) val sig = Transaction.signInput(tx2, 0, redeemScript, SIGHASH_ALL, tx.txOut(0).amount, 1, Alice.finalKey) @@ -88,7 +105,7 @@ class ClaimSentHtlcSpec extends FunSuite { val tx2 = Transaction( version = 2, txIn = TxIn(OutPoint(tx, 0), ByteVector.empty, sequence = reltimeout + 1) :: Nil, - txOut = TxOut(10 satoshi, OP_DUP :: OP_HASH160 :: OP_PUSHDATA(Crypto.hash160(Alice.finalPubKey.toBin)) :: OP_EQUALVERIFY :: OP_CHECKSIG :: Nil) :: Nil, + txOut = TxOut(10 satoshi, OP_DUP :: OP_HASH160 :: OP_PUSHDATA(Crypto.hash160(Alice.finalPubKey.value)) :: OP_EQUALVERIFY :: OP_CHECKSIG :: Nil) :: Nil, lockTime = abstimeout - 1) val sig = Transaction.signInput(tx2, 0, redeemScript, SIGHASH_ALL, tx.txOut(0).amount, 1, Alice.finalKey) @@ -105,7 +122,7 @@ class ClaimSentHtlcSpec extends FunSuite { val tx2 = Transaction( version = 2, txIn = TxIn(OutPoint(tx, 0), ByteVector.empty, sequence = reltimeout - 1) :: Nil, - txOut = TxOut(10 satoshi, OP_DUP :: OP_HASH160 :: OP_PUSHDATA(Crypto.hash160(Alice.finalPubKey.toBin)) :: OP_EQUALVERIFY :: OP_CHECKSIG :: Nil) :: Nil, + txOut = TxOut(10 satoshi, OP_DUP :: OP_HASH160 :: OP_PUSHDATA(Crypto.hash160(Alice.finalPubKey.value)) :: OP_EQUALVERIFY :: OP_CHECKSIG :: Nil) :: Nil, lockTime = abstimeout + 1) val sig = Transaction.signInput(tx2, 0, redeemScript, SIGHASH_ALL, tx.txOut(0).amount, 1, Alice.finalKey) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/transactions/TestVectorsSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/transactions/TestVectorsSpec.scala index 8c2004156..f32d4f046 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/transactions/TestVectorsSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/transactions/TestVectorsSpec.scala @@ -16,7 +16,7 @@ package fr.acinq.eclair.transactions -import fr.acinq.bitcoin.Crypto.{Point, PrivateKey, PublicKey, Scalar} +import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey} import fr.acinq.bitcoin._ import fr.acinq.eclair.channel.Helpers.Funding import fr.acinq.eclair.crypto.Generators @@ -66,16 +66,16 @@ class TestVectorsSpec extends FunSuite with Logging { val commitTxNumber = 42 val toSelfDelay = 144 val dustLimit = Satoshi(546) - val payment_basepoint_secret = Scalar(hex"1111111111111111111111111111111111111111111111111111111111111111") - val payment_basepoint = payment_basepoint_secret.toPoint - val revocation_basepoint_secret = Scalar(hex"2222222222222222222222222222222222222222222222222222222222222222") - val revocation_basepoint = revocation_basepoint_secret.toPoint - val delayed_payment_basepoint_secret = Scalar(hex"3333333333333333333333333333333333333333333333333333333333333333") - val delayed_payment_basepoint = delayed_payment_basepoint_secret.toPoint + val payment_basepoint_secret = PrivateKey(hex"1111111111111111111111111111111111111111111111111111111111111111") + val payment_basepoint = payment_basepoint_secret.publicKey + val revocation_basepoint_secret = PrivateKey(hex"2222222222222222222222222222222222222222222222222222222222222222") + val revocation_basepoint = revocation_basepoint_secret.publicKey + val delayed_payment_basepoint_secret = PrivateKey(hex"3333333333333333333333333333333333333333333333333333333333333333") + val delayed_payment_basepoint = delayed_payment_basepoint_secret.publicKey val funding_privkey = PrivateKey(hex"30ff4956bbdd3222d44cc5e8a1261dab1e07957bdac5ae88fe3261ef321f374901") val funding_pubkey = funding_privkey.publicKey - val per_commitment_point = Point(hex"025f7117a78150fe2ef97db7cfc83bd57b2e2c0d0dd25eaf467a4a1c2a45ce1486") + val per_commitment_point = PublicKey(hex"025f7117a78150fe2ef97db7cfc83bd57b2e2c0d0dd25eaf467a4a1c2a45ce1486") val payment_privkey = Generators.derivePrivKey(payment_basepoint_secret, per_commitment_point) val delayed_payment_privkey = Generators.derivePrivKey(delayed_payment_basepoint_secret, per_commitment_point) val revocation_pubkey = PublicKey(hex"0212a140cd0c6539d07cd08dfe09984dec3251ea808b892efeac3ede9402bf2b19") @@ -108,14 +108,14 @@ class TestVectorsSpec extends FunSuite with Logging { val commitTxNumber = 42 val toSelfDelay = 144 val dustLimit = Satoshi(546) - val payment_basepoint_secret = Scalar(hex"4444444444444444444444444444444444444444444444444444444444444444") - val payment_basepoint = payment_basepoint_secret.toPoint - val revocation_basepoint_secret = Scalar(hex"2222222222222222222222222222222222222222222222222222222222222222") - val revocation_basepoint = revocation_basepoint_secret.toPoint + val payment_basepoint_secret = PrivateKey(hex"4444444444444444444444444444444444444444444444444444444444444444") + val payment_basepoint = payment_basepoint_secret.publicKey + val revocation_basepoint_secret = PrivateKey(hex"2222222222222222222222222222222222222222222222222222222222222222") + val revocation_basepoint = revocation_basepoint_secret.publicKey val funding_privkey = PrivateKey(hex"1552dfba4f6cf29a62a0af13c8d6981d36d0ef8d61ba10fb0fe90da7634d7e1301") val funding_pubkey = funding_privkey.publicKey val payment_privkey = Generators.derivePrivKey(payment_basepoint_secret, Local.per_commitment_point) - val per_commitment_point = Point(hex"022c76692fd70814a8d1ed9dedc833318afaaed8188db4d14727e2e99bc619d325") + val per_commitment_point = PublicKey(hex"022c76692fd70814a8d1ed9dedc833318afaaed8188db4d14727e2e99bc619d325") } val coinbaseTx = Transaction.read("01000000010000000000000000000000000000000000000000000000000000000000000000ffffffff03510101ffffffff0100f2052a010000001976a9143ca33c2e4446f4a305f23c80df8ad1afdcf652f988ac00000000") diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/transactions/TransactionsSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/transactions/TransactionsSpec.scala index cb3a7bc76..12ca3f87a 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/transactions/TransactionsSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/transactions/TransactionsSpec.scala @@ -22,10 +22,10 @@ import fr.acinq.bitcoin.Crypto.{PrivateKey, ripemd160, sha256} import fr.acinq.bitcoin.Script.{pay2wpkh, pay2wsh, write} import fr.acinq.bitcoin._ import fr.acinq.eclair.channel.Helpers.Funding +import fr.acinq.eclair.randomBytes32 import fr.acinq.eclair.transactions.Scripts.{htlcOffered, htlcReceived, toLocalDelayed} import fr.acinq.eclair.transactions.Transactions.{addSigs, _} import fr.acinq.eclair.wire.UpdateAddHtlc -import fr.acinq.eclair.{randomBytes, randomBytes32} import grizzled.slf4j.Logging import org.scalatest.FunSuite import scodec.bits.ByteVector @@ -75,13 +75,13 @@ class TransactionsSpec extends FunSuite with Logging { } test("check pre-computed transaction weights") { - val localRevocationPriv = PrivateKey(randomBytes32, compressed = true) - val localPaymentPriv = PrivateKey(randomBytes32, compressed = true) - val remotePaymentPriv = PrivateKey(randomBytes32, compressed = true) - val localHtlcPriv = PrivateKey(randomBytes32, compressed = true) - val remoteHtlcPriv = PrivateKey(randomBytes32, compressed = true) - val localFinalPriv = PrivateKey(randomBytes32, compressed = true) - val finalPubKeyScript = Script.write(Script.pay2wpkh(PrivateKey(randomBytes32, compressed = true).publicKey)) + val localRevocationPriv = PrivateKey(randomBytes32) + val localPaymentPriv = PrivateKey(randomBytes32) + val remotePaymentPriv = PrivateKey(randomBytes32) + val localHtlcPriv = PrivateKey(randomBytes32) + val remoteHtlcPriv = PrivateKey(randomBytes32) + val localFinalPriv = PrivateKey(randomBytes32) + val finalPubKeyScript = Script.write(Script.pay2wpkh(PrivateKey(randomBytes32).publicKey)) val localDustLimit = Satoshi(546) val toLocalDelay = 144 val feeratePerKw = fr.acinq.eclair.MinimumFeeratePerKw @@ -93,7 +93,7 @@ class TransactionsSpec extends FunSuite with Logging { val commitTx = Transaction(version = 0, txIn = Nil, txOut = TxOut(Satoshi(20000), pubKeyScript) :: Nil, lockTime = 0) val claimP2WPKHOutputTx = makeClaimP2WPKHOutputTx(commitTx, localDustLimit, localPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw) // we use dummy signatures to compute the weight - val weight = Transaction.weight(addSigs(claimP2WPKHOutputTx, localPaymentPriv.publicKey, randomBytes(73)).tx) + val weight = Transaction.weight(addSigs(claimP2WPKHOutputTx, localPaymentPriv.publicKey, PlaceHolderSig).tx) assert(claimP2WPKHOutputWeight == weight) assert(claimP2WPKHOutputTx.fee >= claimP2WPKHOutputTx.minRelayFee) } @@ -105,7 +105,7 @@ class TransactionsSpec extends FunSuite with Logging { val htlcSuccessOrTimeoutTx = Transaction(version = 0, txIn = Nil, txOut = TxOut(Satoshi(20000), pubKeyScript) :: Nil, lockTime = 0) val claimHtlcDelayedTx = makeClaimDelayedOutputTx(htlcSuccessOrTimeoutTx, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localPaymentPriv.publicKey, finalPubKeyScript, feeratePerKw) // we use dummy signatures to compute the weight - val weight = Transaction.weight(addSigs(claimHtlcDelayedTx, randomBytes(73)).tx) + val weight = Transaction.weight(addSigs(claimHtlcDelayedTx, PlaceHolderSig).tx) assert(claimHtlcDelayedWeight == weight) assert(claimHtlcDelayedTx.fee >= claimHtlcDelayedTx.minRelayFee) } @@ -117,7 +117,7 @@ class TransactionsSpec extends FunSuite with Logging { val commitTx = Transaction(version = 0, txIn = Nil, txOut = TxOut(Satoshi(20000), pubKeyScript) :: Nil, lockTime = 0) val mainPenaltyTx = makeMainPenaltyTx(commitTx, localDustLimit, localRevocationPriv.publicKey, finalPubKeyScript, toLocalDelay, localPaymentPriv.publicKey, feeratePerKw) // we use dummy signatures to compute the weight - val weight = Transaction.weight(addSigs(mainPenaltyTx, randomBytes(73)).tx) + val weight = Transaction.weight(addSigs(mainPenaltyTx, PlaceHolderSig).tx) assert(mainPenaltyWeight == weight) assert(mainPenaltyTx.fee >= mainPenaltyTx.minRelayFee) } @@ -132,7 +132,7 @@ class TransactionsSpec extends FunSuite with Logging { val commitTx = Transaction(version = 0, txIn = Nil, txOut = TxOut(Satoshi(htlc.amountMsat / 1000), pubKeyScript) :: Nil, lockTime = 0) val htlcPenaltyTx = makeHtlcPenaltyTx(commitTx, outputsAlreadyUsed = Set.empty, Script.write(redeemScript), localDustLimit, finalPubKeyScript, feeratePerKw) // we use dummy signatures to compute the weight - val weight = Transaction.weight(addSigs(htlcPenaltyTx, randomBytes(73), localRevocationPriv.publicKey).tx) + val weight = Transaction.weight(addSigs(htlcPenaltyTx, PlaceHolderSig, localRevocationPriv.publicKey).tx) assert(htlcPenaltyWeight == weight) assert(htlcPenaltyTx.fee >= htlcPenaltyTx.minRelayFee) } @@ -146,7 +146,7 @@ class TransactionsSpec extends FunSuite with Logging { val commitTx = Transaction(version = 0, txIn = Nil, txOut = TxOut(Satoshi(htlc.amountMsat / 1000), pubKeyScript) :: Nil, lockTime = 0) val claimHtlcSuccessTx = makeClaimHtlcSuccessTx(commitTx, outputsAlreadyUsed = Set.empty, localDustLimit, remoteHtlcPriv.publicKey, localHtlcPriv.publicKey, localRevocationPriv.publicKey, finalPubKeyScript, htlc, feeratePerKw) // we use dummy signatures to compute the weight - val weight = Transaction.weight(addSigs(claimHtlcSuccessTx, randomBytes(73), paymentPreimage).tx) + val weight = Transaction.weight(addSigs(claimHtlcSuccessTx, PlaceHolderSig, paymentPreimage).tx) assert(claimHtlcSuccessWeight == weight) assert(claimHtlcSuccessTx.fee >= claimHtlcSuccessTx.minRelayFee) } @@ -160,7 +160,7 @@ class TransactionsSpec extends FunSuite with Logging { val commitTx = Transaction(version = 0, txIn = Nil, txOut = TxOut(Satoshi(htlc.amountMsat / 1000), pubKeyScript) :: Nil, lockTime = 0) val claimClaimHtlcTimeoutTx = makeClaimHtlcTimeoutTx(commitTx, outputsAlreadyUsed = Set.empty, localDustLimit, remoteHtlcPriv.publicKey, localHtlcPriv.publicKey, localRevocationPriv.publicKey, finalPubKeyScript, htlc, feeratePerKw) // we use dummy signatures to compute the weight - val weight = Transaction.weight(addSigs(claimClaimHtlcTimeoutTx, randomBytes(73)).tx) + val weight = Transaction.weight(addSigs(claimClaimHtlcTimeoutTx, PlaceHolderSig).tx) assert(claimHtlcTimeoutWeight == weight) assert(claimClaimHtlcTimeoutTx.fee >= claimClaimHtlcTimeoutTx.minRelayFee) } @@ -175,7 +175,7 @@ class TransactionsSpec extends FunSuite with Logging { val remotePaymentPriv = PrivateKey(randomBytes32 :+ 1.toByte) val localHtlcPriv = PrivateKey(randomBytes32 :+ 1.toByte) val remoteHtlcPriv = PrivateKey(randomBytes32 :+ 1.toByte) - val finalPubKeyScript = Script.write(Script.pay2wpkh(PrivateKey(randomBytes32, true).publicKey)) + val finalPubKeyScript = Script.write(Script.pay2wpkh(PrivateKey(randomBytes32).publicKey)) val commitInput = Funding.makeFundingInputInfo(randomBytes32, 0, Btc(1), localFundingPriv.publicKey, remoteFundingPriv.publicKey) val toLocalDelay = 144 val localDustLimit = Satoshi(546) @@ -205,7 +205,7 @@ class TransactionsSpec extends FunSuite with Logging { val commitTxNumber = 0x404142434445L val commitTx = { - val txinfo = makeCommitTx(commitInput, commitTxNumber, localPaymentPriv.toPoint, remotePaymentPriv.toPoint, true, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, remotePaymentPriv.publicKey, localHtlcPriv.publicKey, remoteHtlcPriv.publicKey, spec) + val txinfo = makeCommitTx(commitInput, commitTxNumber, localPaymentPriv.publicKey, remotePaymentPriv.publicKey, true, localDustLimit, localRevocationPriv.publicKey, toLocalDelay, localDelayedPaymentPriv.publicKey, remotePaymentPriv.publicKey, localHtlcPriv.publicKey, remoteHtlcPriv.publicKey, spec) val localSig = Transactions.sign(txinfo, localPaymentPriv) val remoteSig = Transactions.sign(txinfo, remotePaymentPriv) Transactions.addSigs(txinfo, localFundingPriv.publicKey, remoteFundingPriv.publicKey, localSig, remoteSig) @@ -213,7 +213,7 @@ class TransactionsSpec extends FunSuite with Logging { { assert(getCommitTxNumber(commitTx.tx, true, localPaymentPriv.publicKey, remotePaymentPriv.publicKey) == commitTxNumber) - val hash = Crypto.sha256(localPaymentPriv.publicKey.toBin ++ remotePaymentPriv.publicKey.toBin) + val hash = Crypto.sha256(localPaymentPriv.publicKey.value ++ remotePaymentPriv.publicKey.value) val num = Protocol.uint64(hash.takeRight(8).toArray, ByteOrder.BIG_ENDIAN) & 0xffffffffffffL val check = ((commitTx.tx.txIn.head.sequence & 0xffffff) << 24) | (commitTx.tx.lockTime & 0xffffff) assert((check ^ num) == commitTxNumber) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/wire/ChannelCodecsSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/wire/ChannelCodecsSpec.scala index 93b5db575..e6e6b1ddd 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/wire/ChannelCodecsSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/wire/ChannelCodecsSpec.scala @@ -84,10 +84,10 @@ class ChannelCodecsSpec extends FunSuite { toSelfDelay = Random.nextInt(Short.MaxValue), maxAcceptedHtlcs = Random.nextInt(Short.MaxValue), fundingPubKey = randomKey.publicKey, - revocationBasepoint = randomKey.publicKey.value, - paymentBasepoint = randomKey.publicKey.value, - delayedPaymentBasepoint = randomKey.publicKey.value, - htlcBasepoint = randomKey.publicKey.value, + revocationBasepoint = randomKey.publicKey, + paymentBasepoint = randomKey.publicKey, + delayedPaymentBasepoint = randomKey.publicKey, + htlcBasepoint = randomKey.publicKey, globalFeatures = randomBytes(256), localFeatures = randomBytes(256)) val encoded = remoteParamsCodec.encode(o).require diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/wire/FailureMessageCodecsSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/wire/FailureMessageCodecsSpec.scala index 03e32e327..f48af32c4 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/wire/FailureMessageCodecsSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/wire/FailureMessageCodecsSpec.scala @@ -16,8 +16,8 @@ package fr.acinq.eclair.wire -import fr.acinq.bitcoin.Block -import fr.acinq.eclair.{ShortChannelId, randomBytes32} +import fr.acinq.bitcoin.{Block, ByteVector64} +import fr.acinq.eclair.{ShortChannelId, randomBytes32, randomBytes64} import org.scalatest.FunSuite import scodec.bits._ @@ -27,7 +27,7 @@ import scodec.bits._ class FailureMessageCodecsSpec extends FunSuite { val channelUpdate = ChannelUpdate( - signature = hex"3045022100c451cd65c88f55b1767941a247e849e12f5f4d4a93a07316659e22f5267d2088022009042a595c6bc8942cd9d729317b82b306edc259fb6b3a3cecb3dd1bd446e90601", + signature = randomBytes64, chainHash = Block.RegtestGenesisBlock.hash, shortChannelId = ShortChannelId(12345), timestamp = 1234567L, @@ -59,7 +59,7 @@ class FailureMessageCodecsSpec extends FunSuite { test("support encoding of channel_update with/without type in failure messages") { val tmp_channel_failure_notype = hex"10070080cc3e80149073ed487c76e48e9622bf980f78267b8a34a3f61921f2d8fce6063b08e74f34a073a13f2097337e4915bb4c001f3b5c4d81e9524ed575e1f45782196fe28c0ab6f1b372c1a6a246ae63f74f931e8365e15a089c68d619000000000008260500041300005b91b52f0003000e00000000000003e80000000100000001" val tmp_channel_failure_withtype = hex"100700820102cc3e80149073ed487c76e48e9622bf980f78267b8a34a3f61921f2d8fce6063b08e74f34a073a13f2097337e4915bb4c001f3b5c4d81e9524ed575e1f45782196fe28c0ab6f1b372c1a6a246ae63f74f931e8365e15a089c68d619000000000008260500041300005b91b52f0003000e00000000000003e80000000100000001" - val ref = TemporaryChannelFailure(ChannelUpdate(hex"3045022100cc3e80149073ed487c76e48e9622bf980f78267b8a34a3f61921f2d8fce6063b022008e74f34a073a13f2097337e4915bb4c001f3b5c4d81e9524ed575e1f457821901", Block.LivenetGenesisBlock.hash, ShortChannelId(0x826050004130000L), 1536275759, 0, 3, 14, 1000, 1, 1, None)) + val ref = TemporaryChannelFailure(ChannelUpdate(ByteVector64(hex"cc3e80149073ed487c76e48e9622bf980f78267b8a34a3f61921f2d8fce6063b08e74f34a073a13f2097337e4915bb4c001f3b5c4d81e9524ed575e1f4578219"), Block.LivenetGenesisBlock.hash, ShortChannelId(0x826050004130000L), 1536275759, 0, 3, 14, 1000, 1, 1, None)) val u = FailureMessageCodecs.failureMessageCodec.decode(tmp_channel_failure_notype.toBitVector).require.value assert(u === ref) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/wire/LightningMessageCodecsSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/wire/LightningMessageCodecsSpec.scala index 0febe2efd..057ff1fe4 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/wire/LightningMessageCodecsSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/wire/LightningMessageCodecsSpec.scala @@ -19,8 +19,8 @@ package fr.acinq.eclair.wire import java.net.{Inet4Address, Inet6Address, InetAddress} import com.google.common.net.InetAddresses -import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey, Scalar} -import fr.acinq.bitcoin.{Block, ByteVector32, Crypto} +import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey} +import fr.acinq.bitcoin.{Block, ByteVector32, ByteVector64} import fr.acinq.eclair._ import fr.acinq.eclair.crypto.Sphinx import fr.acinq.eclair.router.Announcements @@ -34,17 +34,15 @@ import scodec.bits.{BitVector, ByteVector, HexStringSyntax} class LightningMessageCodecsSpec extends FunSuite { - import LightningMessageCodecsSpec._ - def bin(len: Int, fill: Byte) = ByteVector.fill(len)(fill) def bin32(fill: Byte) = ByteVector32(bin(32, fill)) - def scalar(fill: Byte) = Scalar(ByteVector.fill(32)(fill)) + def scalar(fill: Byte) = PrivateKey(ByteVector.fill(32)(fill)) - def point(fill: Byte) = Scalar(ByteVector.fill(32)(fill)).toPoint + def point(fill: Byte) = PrivateKey(ByteVector.fill(32)(fill)).publicKey - def publicKey(fill: Byte) = PrivateKey(ByteVector.fill(32)(fill), compressed = true).publicKey + def publicKey(fill: Byte) = PrivateKey(ByteVector.fill(32)(fill)).publicKey test("encode/decode with uint64 codec") { val expected = Map( @@ -119,44 +117,16 @@ class LightningMessageCodecsSpec extends FunSuite { } } - test("encode/decode with signature codec") { - val sig = randomSignature - val wire = LightningMessageCodecs.signature.encode(sig).require - val sig1 = LightningMessageCodecs.signature.decode(wire).require.value - assert(sig1 == sig) - } - - test("encode/decode with optional signature codec") { - { - val sig = randomSignature - val wire = LightningMessageCodecs.optionalSignature.encode(Some(sig)).require - val Some(sig1) = LightningMessageCodecs.optionalSignature.decode(wire).require.value - assert(sig1 == sig) - } - { - val wire = LightningMessageCodecs.optionalSignature.encode(None).require - assert(LightningMessageCodecs.optionalSignature.decode(wire).require.value == None) - } - } - - test("encode/decode with scalar codec") { - val value = Scalar(randomBytes32) - val wire = LightningMessageCodecs.scalar.encode(value).require + test("encode/decode with private key codec") { + val value = PrivateKey(randomBytes32) + val wire = LightningMessageCodecs.privateKey.encode(value).require assert(wire.length == 256) - val value1 = LightningMessageCodecs.scalar.decode(wire).require.value - assert(value1 == value) - } - - test("encode/decode with point codec") { - val value = Scalar(randomBytes32).toPoint - val wire = LightningMessageCodecs.point.encode(value).require - assert(wire.length == 33 * 8) - val value1 = LightningMessageCodecs.point.decode(wire).require.value + val value1 = LightningMessageCodecs.privateKey.decode(wire).require.value assert(value1 == value) } test("encode/decode with public key codec") { - val value = PrivateKey(randomBytes32, true).publicKey + val value = PrivateKey(randomBytes32).publicKey val wire = LightningMessageCodecs.publicKey.encode(value).require assert(wire.length == 33 * 8) val value1 = LightningMessageCodecs.publicKey.decode(wire).require.value @@ -222,22 +192,22 @@ class LightningMessageCodecsSpec extends FunSuite { val open = OpenChannel(randomBytes32, randomBytes32, 3, 4, 5, UInt64(6), 7, 8, 9, 10, 11, publicKey(1), point(2), point(3), point(4), point(5), point(6), 0.toByte) val accept = AcceptChannel(randomBytes32, 3, UInt64(4), 5, 6, 7, 8, 9, publicKey(1), point(2), point(3), point(4), point(5), point(6)) - val funding_created = FundingCreated(randomBytes32, bin32(0), 3, randomSignature) - val funding_signed = FundingSigned(randomBytes32, randomSignature) + val funding_created = FundingCreated(randomBytes32, bin32(0), 3, randomBytes64) + val funding_signed = FundingSigned(randomBytes32, randomBytes64) val funding_locked = FundingLocked(randomBytes32, point(2)) val update_fee = UpdateFee(randomBytes32, 2) val shutdown = Shutdown(randomBytes32, bin(47, 0)) - val closing_signed = ClosingSigned(randomBytes32, 2, randomSignature) + val closing_signed = ClosingSigned(randomBytes32, 2, randomBytes64) val update_add_htlc = UpdateAddHtlc(randomBytes32, 2, 3, bin32(0), 4, bin(Sphinx.PacketLength, 0)) val update_fulfill_htlc = UpdateFulfillHtlc(randomBytes32, 2, bin32(0)) val update_fail_htlc = UpdateFailHtlc(randomBytes32, 2, bin(154, 0)) val update_fail_malformed_htlc = UpdateFailMalformedHtlc(randomBytes32, 2, randomBytes32, 1111) - val commit_sig = CommitSig(randomBytes32, randomSignature, randomSignature :: randomSignature :: randomSignature :: Nil) + val commit_sig = CommitSig(randomBytes32, randomBytes64, randomBytes64 :: randomBytes64 :: randomBytes64 :: Nil) val revoke_and_ack = RevokeAndAck(randomBytes32, scalar(0), point(1)) - val channel_announcement = ChannelAnnouncement(randomSignature, randomSignature, randomSignature, randomSignature, bin(7, 9), Block.RegtestGenesisBlock.hash, ShortChannelId(1), randomKey.publicKey, randomKey.publicKey, randomKey.publicKey, randomKey.publicKey) - val node_announcement = NodeAnnouncement(randomSignature, bin(0, 0), 1, randomKey.publicKey, Color(100.toByte, 200.toByte, 300.toByte), "node-alias", IPv4(InetAddress.getByAddress(Array[Byte](192.toByte, 168.toByte, 1.toByte, 42.toByte)).asInstanceOf[Inet4Address], 42000) :: Nil) - val channel_update = ChannelUpdate(randomSignature, Block.RegtestGenesisBlock.hash, ShortChannelId(1), 2, 42, 0, 3, 4, 5, 6, None) - val announcement_signatures = AnnouncementSignatures(randomBytes32, ShortChannelId(42), randomSignature, randomSignature) + val channel_announcement = ChannelAnnouncement(randomBytes64, randomBytes64, randomBytes64, randomBytes64, bin(7, 9), Block.RegtestGenesisBlock.hash, ShortChannelId(1), randomKey.publicKey, randomKey.publicKey, randomKey.publicKey, randomKey.publicKey) + val node_announcement = NodeAnnouncement(randomBytes64, bin(0, 0), 1, randomKey.publicKey, Color(100.toByte, 200.toByte, 300.toByte), "node-alias", IPv4(InetAddress.getByAddress(Array[Byte](192.toByte, 168.toByte, 1.toByte, 42.toByte)).asInstanceOf[Inet4Address], 42000) :: Nil) + val channel_update = ChannelUpdate(randomBytes64, Block.RegtestGenesisBlock.hash, ShortChannelId(1), 2, 42, 0, 3, 4, 5, 6, None) + val announcement_signatures = AnnouncementSignatures(randomBytes32, ShortChannelId(42), randomBytes64, randomBytes64) val gossip_timestamp_filter = GossipTimestampFilter(Block.RegtestGenesisBlock.blockId, 100000, 1500) val query_short_channel_id = QueryShortChannelIds(Block.RegtestGenesisBlock.blockId, randomBytes(7515)) val query_channel_range = QueryChannelRange(Block.RegtestGenesisBlock.blockId, 100000, 1500) @@ -278,13 +248,13 @@ class LightningMessageCodecsSpec extends FunSuite { test("encode/decode using cached codec") { val codec = cachedLightningMessageCodec - val commit_sig = CommitSig(randomBytes32, randomSignature, randomSignature :: randomSignature :: randomSignature :: Nil) + val commit_sig = CommitSig(randomBytes32, randomBytes64, randomBytes64 :: randomBytes64 :: randomBytes64 :: Nil) val revoke_and_ack = RevokeAndAck(randomBytes32, scalar(0), point(1)) - val channel_announcement = ChannelAnnouncement(randomSignature, randomSignature, randomSignature, randomSignature, bin(7, 9), Block.RegtestGenesisBlock.hash, ShortChannelId(1), randomKey.publicKey, randomKey.publicKey, randomKey.publicKey, randomKey.publicKey) - val node_announcement = NodeAnnouncement(randomSignature, bin(0, 0), 1, randomKey.publicKey, Color(100.toByte, 200.toByte, 300.toByte), "node-alias", IPv4(InetAddress.getByAddress(Array[Byte](192.toByte, 168.toByte, 1.toByte, 42.toByte)).asInstanceOf[Inet4Address], 42000) :: Nil) - val channel_update1 = ChannelUpdate(randomSignature, Block.RegtestGenesisBlock.hash, ShortChannelId(1), 2, 1, 0, 3, 4, 5, 6, Some(50000000L)) - val channel_update2 = ChannelUpdate(randomSignature, Block.RegtestGenesisBlock.hash, ShortChannelId(1), 2, 0, 0, 3, 4, 5, 6, None) - val announcement_signatures = AnnouncementSignatures(randomBytes32, ShortChannelId(42), randomSignature, randomSignature) + val channel_announcement = ChannelAnnouncement(randomBytes64, randomBytes64, randomBytes64, randomBytes64, bin(7, 9), Block.RegtestGenesisBlock.hash, ShortChannelId(1), randomKey.publicKey, randomKey.publicKey, randomKey.publicKey, randomKey.publicKey) + val node_announcement = NodeAnnouncement(randomBytes64, bin(0, 0), 1, randomKey.publicKey, Color(100.toByte, 200.toByte, 300.toByte), "node-alias", IPv4(InetAddress.getByAddress(Array[Byte](192.toByte, 168.toByte, 1.toByte, 42.toByte)).asInstanceOf[Inet4Address], 42000) :: Nil) + val channel_update1 = ChannelUpdate(randomBytes64, Block.RegtestGenesisBlock.hash, ShortChannelId(1), 2, 1, 0, 3, 4, 5, 6, Some(50000000L)) + val channel_update2 = ChannelUpdate(randomBytes64, Block.RegtestGenesisBlock.hash, ShortChannelId(1), 2, 0, 0, 3, 4, 5, 6, None) + val announcement_signatures = AnnouncementSignatures(randomBytes32, ShortChannelId(42), randomBytes64, randomBytes64) val ping = Ping(100, bin(10, 1)) val pong = Pong(bin(10, 1)) @@ -311,7 +281,7 @@ class LightningMessageCodecsSpec extends FunSuite { // this was generated by c-lightning val bin = hex"010258fff7d0e987e2cdd560e3bb5a046b4efe7b26c969c2f51da1dceec7bcb8ae1b634790503d5290c1a6c51d681cf8f4211d27ed33a257dcc1102862571bf1792306226e46111a0b59caaf126043eb5bbf28c34f3a5e332a1fc7b2b73cf188910f0005a100000200005bc75919010100060000000000000001000000010000000a000000003a699d00" val update = LightningMessageCodecs.lightningMessageCodec.decode(BitVector(bin.toArray)).require.value.asInstanceOf[ChannelUpdate] - assert(update === ChannelUpdate(hex"3044022058fff7d0e987e2cdd560e3bb5a046b4efe7b26c969c2f51da1dceec7bcb8ae1b0220634790503d5290c1a6c51d681cf8f4211d27ed33a257dcc1102862571bf1792301", ByteVector32(hex"06226e46111a0b59caaf126043eb5bbf28c34f3a5e332a1fc7b2b73cf188910f"), ShortChannelId(0x5a10000020000L), 1539791129, 1, 1, 6, 1, 1, 10, Some(980000000L))) + assert(update === ChannelUpdate(ByteVector64(hex"58fff7d0e987e2cdd560e3bb5a046b4efe7b26c969c2f51da1dceec7bcb8ae1b634790503d5290c1a6c51d681cf8f4211d27ed33a257dcc1102862571bf17923"), ByteVector32(hex"06226e46111a0b59caaf126043eb5bbf28c34f3a5e332a1fc7b2b73cf188910f"), ShortChannelId(0x5a10000020000L), 1539791129, 1, 1, 6, 1, 1, 10, Some(980000000L))) val nodeId = PublicKey(hex"03370c9bac836e557eb4f017fe8f9cc047f44db39c1c4e410ff0f7be142b817ae4") assert(Announcements.checkSig(update, nodeId)) val bin2 = ByteVector(LightningMessageCodecs.lightningMessageCodec.encode(update).require.toByteArray) @@ -319,12 +289,3 @@ class LightningMessageCodecsSpec extends FunSuite { } } - -object LightningMessageCodecsSpec { - def randomSignature: ByteVector = { - val priv = randomBytes32 - val data = randomBytes32 - val (r, s) = Crypto.sign(data, PrivateKey(priv, true)) - Crypto.encodeSignature(r, s) :+ fr.acinq.bitcoin.SIGHASH_ALL.toByte - } -} \ No newline at end of file diff --git a/pom.xml b/pom.xml index c34f96b09..aa61f0343 100644 --- a/pom.xml +++ b/pom.xml @@ -68,7 +68,7 @@ 2.4.20 10.0.11 1.3.9 - 0.11 + 0.13 24.0-android From b4adff2dc4f92c0742eb0430b8ea695d2f9b0c94 Mon Sep 17 00:00:00 2001 From: Bastien Teinturier <31281497+t-bast@users.noreply.github.com> Date: Fri, 14 Jun 2019 13:25:37 +0200 Subject: [PATCH 09/21] Electrum: add id_from_pos (#1035) This allows getting a transaction id from a short channel id. --- .../blockchain/electrum/ElectrumClient.scala | 38 ++++++++++++------- .../electrum/ElectrumClientSpec.scala | 34 ++++++++++++++++- 2 files changed, 56 insertions(+), 16 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/electrum/ElectrumClient.scala b/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/electrum/ElectrumClient.scala index 3be66f0a7..e1e1869c9 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/electrum/ElectrumClient.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/electrum/ElectrumClient.scala @@ -91,7 +91,7 @@ class ElectrumClient(serverAddress: InetSocketAddress, ssl: SSL)(implicit val ec val channelOpenFuture = b.connect(serverAddress.getHostName, serverAddress.getPort) - def errorHandler(t: Throwable) = { + def errorHandler(t: Throwable): Unit = { log.info("server={} connection error (reason={})", serverAddress, t.getMessage) self ! Close } @@ -169,6 +169,7 @@ class ElectrumClient(serverAddress: InetSocketAddress, ssl: SSL)(implicit val ec val json = ("method" -> request.method) ~ ("params" -> request.params.map { case s: String => new JString(s) case b: ByteVector32 => new JString(b.toHex) + case b: Boolean => new JBool(b) case t: Int => new JInt(t) case t: Long => new JLong(t) case t: Double => new JDouble(t) @@ -182,8 +183,6 @@ class ElectrumClient(serverAddress: InetSocketAddress, ssl: SSL)(implicit val ec /** * Forwards incoming messages to the underlying actor - * - * @param actor */ class ActorHandler(actor: ActorRef) extends ChannelInboundHandlerAdapter { @@ -220,7 +219,7 @@ class ElectrumClient(serverAddress: InetSocketAddress, ssl: SSL)(implicit val ec case PingResponse => () case Close => - statusListeners.map(_ ! ElectrumDisconnected) + statusListeners.foreach(_ ! ElectrumDisconnected) context.stop(self) case _ => log.warning("server={} unhandled message {}", serverAddress, message) @@ -282,7 +281,7 @@ class ElectrumClient(serverAddress: InetSocketAddress, ssl: SSL)(implicit val ec case Right(json: JsonRPCResponse) => val (height, header) = parseBlockHeader(json.result) log.debug("connected to server={}, tip={} height={}", serverAddress, header.hash, height) - statusListeners.map(_ ! ElectrumReady(height, header, serverAddress)) + statusListeners.foreach(_ ! ElectrumReady(height, header, serverAddress)) context become connected(ctx, height, header, Map()) case AddStatusListener(actor) => statusListeners += actor @@ -322,11 +321,11 @@ class ElectrumClient(serverAddress: InetSocketAddress, ssl: SSL)(implicit val ec } context become connected(ctx, height, tip, requests - json.id) - case Left(response: HeaderSubscriptionResponse) => headerSubscriptions.map(_ ! response) + case Left(response: HeaderSubscriptionResponse) => headerSubscriptions.foreach(_ ! response) - case Left(response: AddressSubscriptionResponse) => addressSubscriptions.get(response.address).map(listeners => listeners.map(_ ! response)) + case Left(response: AddressSubscriptionResponse) => addressSubscriptions.get(response.address).foreach(listeners => listeners.foreach(_ ! response)) - case Left(response: ScriptHashSubscriptionResponse) => scriptHashSubscriptions.get(response.scriptHash).map(listeners => listeners.map(_ ! response)) + case Left(response: ScriptHashSubscriptionResponse) => scriptHashSubscriptions.get(response.scriptHash).foreach(listeners => listeners.foreach(_ ! response)) case HeaderSubscriptionResponse(height, newtip) => log.info("server={} new tip={}", serverAddress, newtip) @@ -381,6 +380,9 @@ object ElectrumClient { case class BroadcastTransaction(tx: Transaction) extends Request case class BroadcastTransactionResponse(tx: Transaction, error: Option[Error]) extends Response + case class GetTransactionIdFromPosition(height: Int, tx_pos: Int, merkle: Boolean = false) extends Request + case class GetTransactionIdFromPositionResponse(txid: ByteVector32, merkle: Seq[ByteVector32]) extends Response + case class GetTransaction(txid: ByteVector32) extends Request case class GetTransactionResponse(tx: Transaction) extends Response @@ -533,10 +535,11 @@ object ElectrumClient { case AddressSubscription(address, _) => JsonRPCRequest(id = reqId, method = "blockchain.address.subscribe", params = address :: Nil) case ScriptHashSubscription(scriptHash, _) => JsonRPCRequest(id = reqId, method = "blockchain.scripthash.subscribe", params = scriptHash.toString() :: Nil) case BroadcastTransaction(tx) => JsonRPCRequest(id = reqId, method = "blockchain.transaction.broadcast", params = Transaction.write(tx).toHex :: Nil) + case GetTransactionIdFromPosition(height, tx_pos, merkle) => JsonRPCRequest(id = reqId, method = "blockchain.transaction.id_from_pos", params = height :: tx_pos :: merkle :: Nil) case GetTransaction(txid) => JsonRPCRequest(id = reqId, method = "blockchain.transaction.get", params = txid :: Nil) case HeaderSubscription(_) => JsonRPCRequest(id = reqId, method = "blockchain.headers.subscribe", params = Nil) case GetHeader(height) => JsonRPCRequest(id = reqId, method = "blockchain.block.header", params = height :: Nil) - case GetHeaders(start_height, count, cp_height) => JsonRPCRequest(id = reqId, method = "blockchain.block.headers", params = start_height :: count :: Nil) + case GetHeaders(start_height, count, _) => JsonRPCRequest(id = reqId, method = "blockchain.block.headers", params = start_height :: count :: Nil) case GetMerkle(txid, height) => JsonRPCRequest(id = reqId, method = "blockchain.transaction.get_merkle", params = txid :: height :: Nil) } @@ -548,7 +551,7 @@ object ElectrumClient { case _ => ServerError(request, error) } case None => (request: @unchecked) match { - case s: ServerVersion => + case _: ServerVersion => val JArray(jitems) = json.result val JString(clientName) = jitems(0) val JString(protocolVersion) = jitems(1) @@ -590,6 +593,14 @@ object ElectrumClient { UnspentItem(ByteVector32.fromValidHex(tx_hash), tx_pos, value, height) }) ScriptHashListUnspentResponse(scripthash, items) + case GetTransactionIdFromPosition(_, _, false) => + val JString(tx_hash) = json.result + GetTransactionIdFromPositionResponse(ByteVector32.fromValidHex(tx_hash), Nil) + case GetTransactionIdFromPosition(_, _, true) => + val JString(tx_hash) = json.result \ "tx_hash" + val JArray(hashes) = json.result \ "merkle" + val leaves = hashes collect { case JString(value) => ByteVector32.fromValidHex(value) } + GetTransactionIdFromPositionResponse(ByteVector32.fromValidHex(tx_hash), leaves) case GetTransaction(_) => val JString(hex) = json.result GetTransactionResponse(Transaction.read(hex)) @@ -614,16 +625,15 @@ object ElectrumClient { case GetHeader(height) => val JString(hex) = json.result GetHeaderResponse(height, BlockHeader.read(hex)) - case GetHeaders(start_height, count, cp_height) => - val count = intField(json.result, "count") + case GetHeaders(start_height, _, _) => val max = intField(json.result, "max") val JString(hex) = json.result \ "hex" val bin = ByteVector.fromValidHex(hex).toArray val blockHeaders = bin.grouped(80).map(BlockHeader.read).toList GetHeadersResponse(start_height, blockHeaders, max) - case GetMerkle(txid, height) => + case GetMerkle(txid, _) => val JArray(hashes) = json.result \ "merkle" - val leaves = hashes collect { case JString(value) => ByteVector32.fromValidHex((value)) } + val leaves = hashes collect { case JString(value) => ByteVector32.fromValidHex(value) } val blockHeight = intField(json.result, "block_height") val JInt(pos) = json.result \ "pos" GetMerkleResponse(txid, leaves, blockHeight, pos.toInt) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/electrum/ElectrumClientSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/electrum/ElectrumClientSpec.scala index d2d4606ae..fb936f955 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/electrum/ElectrumClientSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/electrum/ElectrumClientSpec.scala @@ -28,7 +28,6 @@ import scodec.bits._ import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration._ - class ElectrumClientSpec extends TestKit(ActorSystem("test")) with FunSuiteLike with Logging with BeforeAndAfterAll { import ElectrumClient._ @@ -38,6 +37,22 @@ class ElectrumClientSpec extends TestKit(ActorSystem("test")) with FunSuiteLike // this is tx #2690 of block #500000 val referenceTx = Transaction.read("0200000001983c5b32ced1de5ae97d3ce9b7436f8bb0487d15bf81e5cae97b1e238dc395c6000000006a47304402205957c75766e391350eba2c7b752f0056cb34b353648ecd0992a8a81fc9bcfe980220629c286592842d152cdde71177cd83086619744a533f262473298cacf60193500121021b8b51f74dbf0ac1e766d162c8707b5e8d89fc59da0796f3b4505e7c0fb4cf31feffffff0276bd0101000000001976a914219de672ba773aa0bc2e15cdd9d2e69b734138fa88ac3e692001000000001976a914301706dede031e9fb4b60836e073a4761855f6b188ac09a10700") val scriptHash = Crypto.sha256(referenceTx.txOut(0).publicKeyScript).reverse + val height = 500000 + val position = 2690 + val merkleProof = List( + hex"b500cd85cd6c7e0e570b82728dd516646536a477b61cc82056505d84a5820dc3", + hex"c98798c2e576566a92b23d2405f59d95c506966a6e26fecfb356d6447a199546", + hex"930d95c428546812fd11f8242904a9a1ba05d2140cd3a83be0e2ed794821c9ec", + hex"90c97965b12f4262fe9bf95bc37ff7d6362902745eaa822ecf0cf85801fa8b48", + hex"23792d51fddd6e439ed4c92ad9f19a9b73fc9d5c52bdd69039be70ad6619a1aa", + hex"4b73075f29a0abdcec2c83c2cfafc5f304d2c19dcacb50a88a023df725468760", + hex"f80225a32a5ce4ef0703822c6aa29692431a816dec77d9b1baa5b09c3ba29bfb", + hex"4858ac33f2022383d3b4dd674666a0880557d02a155073be93231a02ecbb81f4", + hex"eb5b142030ed4e0b55a8ba5a7b5b783a0a24e0c2fd67c1cfa2f7b308db00c38a", + hex"86858812c3837d209110f7ea79de485abdfd22039467a8aa15a8d85856ee7d30", + hex"de20eb85f2e9ad525a6fb5c618682b6bdce2fa83df836a698f31575c4e5b3d38", + hex"98bd1048e04ff1b0af5856d9890cd708d8d67ad6f3a01f777130fbc16810eeb3") + .map(ByteVector32(_)) override protected def beforeAll(): Unit = { client = system.actorOf(Props(new ElectrumClient(new InetSocketAddress("electrum.acinq.co", 50002), SSL.STRICT)), "electrum-client") @@ -52,6 +67,20 @@ class ElectrumClientSpec extends TestKit(ActorSystem("test")) with FunSuiteLike probe.expectMsgType[ElectrumReady](15 seconds) } + test("get transaction id from position") { + probe.send(client, GetTransactionIdFromPosition(height, position)) + val GetTransactionIdFromPositionResponse(txid, merkle) = probe.expectMsgType[GetTransactionIdFromPositionResponse] + assert(txid === referenceTx.txid) + assert(merkle === Nil) + } + + test("get transaction id from position with merkle proof") { + probe.send(client, GetTransactionIdFromPosition(height, position, merkle = true)) + val GetTransactionIdFromPositionResponse(txid, merkle) = probe.expectMsgType[GetTransactionIdFromPositionResponse] + assert(txid === referenceTx.txid) + assert(merkle === merkleProof) + } + test("get transaction") { probe.send(client, GetTransaction(referenceTx.txid)) val GetTransactionResponse(tx) = probe.expectMsgType[GetTransactionResponse] @@ -98,7 +127,7 @@ class ElectrumClientSpec extends TestKit(ActorSystem("test")) with FunSuiteLike test("get scripthash history") { probe.send(client, GetScriptHashHistory(scriptHash)) val GetScriptHashHistoryResponse(scriptHash1, history) = probe.expectMsgType[GetScriptHashHistoryResponse] - assert(history.contains((TransactionHistoryItem(500000, referenceTx.txid)))) + assert(history.contains(TransactionHistoryItem(500000, referenceTx.txid))) } test("list script unspents") { @@ -106,4 +135,5 @@ class ElectrumClientSpec extends TestKit(ActorSystem("test")) with FunSuiteLike val ScriptHashListUnspentResponse(scriptHash1, unspents) = probe.expectMsgType[ScriptHashListUnspentResponse] assert(unspents.isEmpty) } + } From a3563e38bbe5b60433daa330ab7ec2d97fbce7af Mon Sep 17 00:00:00 2001 From: Pierre-Marie Padiou Date: Fri, 14 Jun 2019 14:04:15 +0200 Subject: [PATCH 10/21] Removed cached codec (#1036) It turns out that performance gains of the cached codec are not that great, and they come at a cost of significant pressure on the GC. In other words: premature optimization. When removed, the heap usage becomes very stable, which is much better than hypothetical performance gains. Fixes #1031. --- .../fr/acinq/eclair/io/Authenticator.scala | 4 +-- .../eclair/wire/LightningMessageCodecs.scala | 25 --------------- .../wire/LightningMessageCodecsSpec.scala | 32 ------------------- 3 files changed, 2 insertions(+), 59 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/io/Authenticator.scala b/eclair-core/src/main/scala/fr/acinq/eclair/io/Authenticator.scala index 1aebf9c0f..f77b26354 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/io/Authenticator.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/io/Authenticator.scala @@ -47,9 +47,9 @@ class Authenticator(nodeParams: NodeParams) extends Actor with DiagnosticActorLo KeyPair(nodeParams.nodeId.value, nodeParams.privateKey.value), remoteNodeId_opt.map(_.value), connection = connection, - codec = LightningMessageCodecs.cachedLightningMessageCodec)) + codec = LightningMessageCodecs.lightningMessageCodec)) context watch transport - context become (ready(switchboard, authenticating + (transport -> pending))) + context become ready(switchboard, authenticating + (transport -> pending)) case HandshakeCompleted(connection, transport, remoteNodeId) if authenticating.contains(transport) => val pendingAuth = authenticating(transport) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/LightningMessageCodecs.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/LightningMessageCodecs.scala index 6b6e89ff9..e984eaf5e 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/LightningMessageCodecs.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/LightningMessageCodecs.scala @@ -321,31 +321,6 @@ object LightningMessageCodecs { .typecase(264, replyChannelRangeCodec) .typecase(265, gossipTimestampFilterCodec) - - /** - * A codec that caches serialized routing messages - */ - val cachedLightningMessageCodec = new Codec[LightningMessage] { - - override def sizeBound: SizeBound = lightningMessageCodec.sizeBound - - val cache = CacheBuilder - .newBuilder - .weakKeys() // will cleanup values when keys are garbage collected - .build(new CacheLoader[LightningMessage, Attempt[BitVector]] { - override def load(key: LightningMessage): Attempt[BitVector] = lightningMessageCodec.encode(key) - }) - - override def encode(value: LightningMessage): Attempt[BitVector] = value match { - case _: ChannelAnnouncement => cache.get(value) // we only cache serialized routing messages - case _: NodeAnnouncement => cache.get(value) // we only cache serialized routing messages - case _: ChannelUpdate => cache.get(value) // we only cache serialized routing messages - case _ => lightningMessageCodec.encode(value) - } - - override def decode(bits: BitVector): Attempt[DecodeResult[LightningMessage]] = lightningMessageCodec.decode(bits) - } - val perHopPayloadCodec: Codec[PerHopPayload] = ( ("realm" | constant(ByteVector.fromByte(0))) :: ("short_channel_id" | shortchannelid) :: diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/wire/LightningMessageCodecsSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/wire/LightningMessageCodecsSpec.scala index 057ff1fe4..93917a042 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/wire/LightningMessageCodecsSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/wire/LightningMessageCodecsSpec.scala @@ -245,38 +245,6 @@ class LightningMessageCodecsSpec extends FunSuite { } } - test("encode/decode using cached codec") { - val codec = cachedLightningMessageCodec - - val commit_sig = CommitSig(randomBytes32, randomBytes64, randomBytes64 :: randomBytes64 :: randomBytes64 :: Nil) - val revoke_and_ack = RevokeAndAck(randomBytes32, scalar(0), point(1)) - val channel_announcement = ChannelAnnouncement(randomBytes64, randomBytes64, randomBytes64, randomBytes64, bin(7, 9), Block.RegtestGenesisBlock.hash, ShortChannelId(1), randomKey.publicKey, randomKey.publicKey, randomKey.publicKey, randomKey.publicKey) - val node_announcement = NodeAnnouncement(randomBytes64, bin(0, 0), 1, randomKey.publicKey, Color(100.toByte, 200.toByte, 300.toByte), "node-alias", IPv4(InetAddress.getByAddress(Array[Byte](192.toByte, 168.toByte, 1.toByte, 42.toByte)).asInstanceOf[Inet4Address], 42000) :: Nil) - val channel_update1 = ChannelUpdate(randomBytes64, Block.RegtestGenesisBlock.hash, ShortChannelId(1), 2, 1, 0, 3, 4, 5, 6, Some(50000000L)) - val channel_update2 = ChannelUpdate(randomBytes64, Block.RegtestGenesisBlock.hash, ShortChannelId(1), 2, 0, 0, 3, 4, 5, 6, None) - val announcement_signatures = AnnouncementSignatures(randomBytes32, ShortChannelId(42), randomBytes64, randomBytes64) - val ping = Ping(100, bin(10, 1)) - val pong = Pong(bin(10, 1)) - - val cached = channel_announcement :: node_announcement :: channel_update1 :: channel_update2 :: Nil - val nonCached = commit_sig :: revoke_and_ack :: announcement_signatures :: ping :: pong :: Nil - val msgs: List[LightningMessage] = cached ::: nonCached - - msgs.foreach { - case msg => { - val encoded = codec.encode(msg).require - val decoded = codec.decode(encoded).require - assert(msg === decoded.value) - } - } - - import scala.language.reflectiveCalls - val cachedKeys = codec.cache.asMap().keySet() - assert(cached.forall(msg => cachedKeys.contains(msg))) - assert(nonCached.forall(msg => !cachedKeys.contains(msg))) - - } - test("decode channel_update with htlc_maximum_msat") { // this was generated by c-lightning val bin = hex"010258fff7d0e987e2cdd560e3bb5a046b4efe7b26c969c2f51da1dceec7bcb8ae1b634790503d5290c1a6c51d681cf8f4211d27ed33a257dcc1102862571bf1792306226e46111a0b59caaf126043eb5bbf28c34f3a5e332a1fc7b2b73cf188910f0005a100000200005bc75919010100060000000000000001000000010000000a000000003a699d00" From cb1196cf957e655395e38ec322bec93346a57a1c Mon Sep 17 00:00:00 2001 From: Anton Kumaigorodski Date: Fri, 14 Jun 2019 16:01:10 +0300 Subject: [PATCH 11/21] Correctly decode requests without multipliers (#1020) Since MAX_PAYMENT_AMOUNT has been removed we may now get payment requests without multiplier which means a whole bitcoin is requested. --- .../main/scala/fr/acinq/eclair/payment/PaymentRequest.scala | 1 + .../scala/fr/acinq/eclair/payment/PaymentRequestSpec.scala | 6 ++++++ 2 files changed, 7 insertions(+) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentRequest.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentRequest.scala index e34628560..9e8a9323c 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentRequest.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentRequest.scala @@ -399,6 +399,7 @@ object PaymentRequest { case a if a.last == 'n' => Some(MilliSatoshi(a.dropRight(1).toLong * 100L)) case a if a.last == 'u' => Some(MilliSatoshi(a.dropRight(1).toLong * 100000L)) case a if a.last == 'm' => Some(MilliSatoshi(a.dropRight(1).toLong * 100000000L)) + case a => Some(MilliSatoshi(a.toLong * 100000000000L)) } def encode(amount: Option[MilliSatoshi]): String = { diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentRequestSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentRequestSpec.scala index 9acd85b61..f67bd0e07 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentRequestSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/PaymentRequestSpec.scala @@ -262,6 +262,12 @@ class PaymentRequestSpec extends FunSuite { assert(PaymentRequest.write(PaymentRequest.read(input.toUpperCase())) == input) } + test("Pay 1 BTC without multiplier") { + val ref = "lnbc11pdkmqhupp5n2ees808r98m0rh4472yyth0c5fptzcxmexcjznrzmq8xald0cgqdqsf4ujqarfwqsxymmccqp2xvtsv5tc743wgctlza8k3zlpxucl7f3kvjnjptv7xz0nkaww307sdyrvgke2w8kmq7dgz4lkasfn0zvplc9aa4gp8fnhrwfjny0j59sq42x9gp" + val pr = PaymentRequest.read(ref) + assert(pr.amount.contains(MilliSatoshi(100000000000L))) + } + test("nonreg") { val requests = List( "lnbc40n1pw9qjvwpp5qq3w2ln6krepcslqszkrsfzwy49y0407hvks30ec6pu9s07jur3sdpstfshq5n9v9jzucm0d5s8vmm5v5s8qmmnwssyj3p6yqenwdencqzysxqrrss7ju0s4dwx6w8a95a9p2xc5vudl09gjl0w2n02sjrvffde632nxwh2l4w35nqepj4j5njhh4z65wyfc724yj6dn9wajvajfn5j7em6wsq2elakl", From a53145a6daa1e8ca260f87003b028e64b3ccf929 Mon Sep 17 00:00:00 2001 From: Bastien Teinturier <31281497+t-bast@users.noreply.github.com> Date: Mon, 17 Jun 2019 12:10:10 +0200 Subject: [PATCH 12/21] Electrum: add data to TxIdFromPosResponse. (#1038) To make the response usable, we need to copy the request input (block height and transaction position). --- .../eclair/blockchain/electrum/ElectrumClient.scala | 10 +++++----- .../blockchain/electrum/ElectrumClientSpec.scala | 8 ++------ 2 files changed, 7 insertions(+), 11 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/electrum/ElectrumClient.scala b/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/electrum/ElectrumClient.scala index e1e1869c9..271d8039d 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/electrum/ElectrumClient.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/electrum/ElectrumClient.scala @@ -381,7 +381,7 @@ object ElectrumClient { case class BroadcastTransactionResponse(tx: Transaction, error: Option[Error]) extends Response case class GetTransactionIdFromPosition(height: Int, tx_pos: Int, merkle: Boolean = false) extends Request - case class GetTransactionIdFromPositionResponse(txid: ByteVector32, merkle: Seq[ByteVector32]) extends Response + case class GetTransactionIdFromPositionResponse(txid: ByteVector32, height: Int, tx_pos: Int, merkle: Seq[ByteVector32]) extends Response case class GetTransaction(txid: ByteVector32) extends Request case class GetTransactionResponse(tx: Transaction) extends Response @@ -593,14 +593,14 @@ object ElectrumClient { UnspentItem(ByteVector32.fromValidHex(tx_hash), tx_pos, value, height) }) ScriptHashListUnspentResponse(scripthash, items) - case GetTransactionIdFromPosition(_, _, false) => + case GetTransactionIdFromPosition(height, tx_pos, false) => val JString(tx_hash) = json.result - GetTransactionIdFromPositionResponse(ByteVector32.fromValidHex(tx_hash), Nil) - case GetTransactionIdFromPosition(_, _, true) => + GetTransactionIdFromPositionResponse(ByteVector32.fromValidHex(tx_hash), height, tx_pos, Nil) + case GetTransactionIdFromPosition(height, tx_pos, true) => val JString(tx_hash) = json.result \ "tx_hash" val JArray(hashes) = json.result \ "merkle" val leaves = hashes collect { case JString(value) => ByteVector32.fromValidHex(value) } - GetTransactionIdFromPositionResponse(ByteVector32.fromValidHex(tx_hash), leaves) + GetTransactionIdFromPositionResponse(ByteVector32.fromValidHex(tx_hash), height, tx_pos, leaves) case GetTransaction(_) => val JString(hex) = json.result GetTransactionResponse(Transaction.read(hex)) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/electrum/ElectrumClientSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/electrum/ElectrumClientSpec.scala index fb936f955..ebe8e9090 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/electrum/ElectrumClientSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/blockchain/electrum/ElectrumClientSpec.scala @@ -69,16 +69,12 @@ class ElectrumClientSpec extends TestKit(ActorSystem("test")) with FunSuiteLike test("get transaction id from position") { probe.send(client, GetTransactionIdFromPosition(height, position)) - val GetTransactionIdFromPositionResponse(txid, merkle) = probe.expectMsgType[GetTransactionIdFromPositionResponse] - assert(txid === referenceTx.txid) - assert(merkle === Nil) + probe.expectMsg(GetTransactionIdFromPositionResponse(referenceTx.txid, height, position, Nil)) } test("get transaction id from position with merkle proof") { probe.send(client, GetTransactionIdFromPosition(height, position, merkle = true)) - val GetTransactionIdFromPositionResponse(txid, merkle) = probe.expectMsgType[GetTransactionIdFromPositionResponse] - assert(txid === referenceTx.txid) - assert(merkle === merkleProof) + probe.expectMsg(GetTransactionIdFromPositionResponse(referenceTx.txid, height, position, merkleProof)) } test("get transaction") { From 38696b56e49c5effa498a7e05bbcad963bb362e2 Mon Sep 17 00:00:00 2001 From: Pierre-Marie Padiou Date: Wed, 19 Jun 2019 11:36:30 +0200 Subject: [PATCH 13/21] Minor fixes (#1043) * fixed javadoc * make bitcoind only bind on localhost for tests --- .../fr/acinq/eclair/blockchain/electrum/ElectrumWallet.scala | 2 +- .../src/main/scala/fr/acinq/eclair/channel/Channel.scala | 5 ++++- eclair-core/src/test/resources/integration/bitcoin.conf | 3 ++- 3 files changed, 7 insertions(+), 3 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/electrum/ElectrumWallet.scala b/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/electrum/ElectrumWallet.scala index 912fd1206..d9f3c1215 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/electrum/ElectrumWallet.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/electrum/ElectrumWallet.scala @@ -739,7 +739,7 @@ object ElectrumWallet { } /** - * @scriptHash script hash + * * @return the ids of transactions that belong to our wallet history for this script hash but that we don't have * and have no pending requests for. */ 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 213306add..38a3609ba 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 @@ -1644,7 +1644,10 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId case Event(Status.Failure(_: CannotAffordFees), _) => stay // funding tx was confirmed in time, let's just ignore this - case Event(BITCOIN_FUNDING_TIMEOUT, d: HasCommitments) => stay + case Event(BITCOIN_FUNDING_TIMEOUT, _: HasCommitments) => stay + + // peer doesn't cancel the timer + case Event(TickChannelOpenTimeout, _) => stay case Event(WatchEventSpent(BITCOIN_FUNDING_SPENT, tx), d: HasCommitments) if tx.txid == d.commitments.localCommit.publishableTxs.commitTx.tx.txid => log.warning(s"processing local commit spent in catch-all handler") diff --git a/eclair-core/src/test/resources/integration/bitcoin.conf b/eclair-core/src/test/resources/integration/bitcoin.conf index 29775744a..8676e0b42 100644 --- a/eclair-core/src/test/resources/integration/bitcoin.conf +++ b/eclair-core/src/test/resources/integration/bitcoin.conf @@ -1,7 +1,6 @@ regtest=1 noprinttoconsole=1 server=1 -port=28333 rpcuser=foo rpcpassword=bar txindex=1 @@ -10,4 +9,6 @@ zmqpubrawtx=tcp://127.0.0.1:28335 rpcworkqueue=64 addresstype=bech32 [regtest] +bind=127.0.0.1 +port=28333 rpcport=28332 From 316ba02ff8652033a7a2a51dd58c4de58dba12a0 Mon Sep 17 00:00:00 2001 From: Pierre-Marie Padiou Date: Thu, 20 Jun 2019 11:40:01 +0200 Subject: [PATCH 14/21] Ack unhandled lightning messages (#1042) In the event when we receive an unexpected message, the `Peer` was just logging a warning and not sending an `Ack` to the `TransportHandler`. This resulted in a stuck connection, because no more data was read on the connection. Fixes #1037. --- .../src/main/scala/fr/acinq/eclair/io/Peer.scala | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/io/Peer.scala b/eclair-core/src/main/scala/fr/acinq/eclair/io/Peer.scala index 020ad791a..43dcb4903 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/io/Peer.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/io/Peer.scala @@ -191,6 +191,11 @@ class Peer(nodeParams: NodeParams, remoteNodeId: PublicKey, authenticator: Actor d.transport ! PoisonPill stay + case Event(unhandledMsg: LightningMessage, d: InitializingData) => + // we ack unhandled messages because we don't want to block further reads on the connection + d.transport ! TransportHandler.ReadAck(unhandledMsg) + log.warning(s"acking unhandled message $unhandledMsg") + stay } when(CONNECTED) { @@ -458,6 +463,12 @@ class Peer(nodeParams: NodeParams, remoteNodeId: PublicKey, authenticator: Actor d.channels.values.toSet[ActorRef].foreach(_ ! INPUT_DISCONNECTED) // we deduplicate with toSet because there might be two entries per channel (tmp id and final id) self ! h goto(DISCONNECTED) using DisconnectedData(d.address_opt, d.channels.collect { case (k: FinalChannelId, v) => (k, v) }) + + case Event(unhandledMsg: LightningMessage, d: ConnectedData) => + // we ack unhandled messages because we don't want to block further reads on the connection + d.transport ! TransportHandler.ReadAck(unhandledMsg) + log.warning(s"acking unhandled message $unhandledMsg") + stay } whenUnhandled { From de50cc437ea9ae14d3294f1d48237c697be8ee3f Mon Sep 17 00:00:00 2001 From: araspitzu Date: Mon, 24 Jun 2019 17:46:55 +0200 Subject: [PATCH 15/21] Remove old service and related docs (#1046) --- OLD-API-DOCS.md | 40 -- README.md | 2 - eclair-core/src/main/resources/reference.conf | 1 - .../main/scala/fr/acinq/eclair/Setup.scala | 22 +- .../fr/acinq/eclair/api/OldService.scala | 420 ------------------ 5 files changed, 6 insertions(+), 479 deletions(-) delete mode 100644 OLD-API-DOCS.md delete mode 100644 eclair-core/src/main/scala/fr/acinq/eclair/api/OldService.scala diff --git a/OLD-API-DOCS.md b/OLD-API-DOCS.md deleted file mode 100644 index 1c7f30abe..000000000 --- a/OLD-API-DOCS.md +++ /dev/null @@ -1,40 +0,0 @@ - ## JSON-RPC API - - :warning: Note this interface is being deprecated. - - method | params | description - ------------- |----------------------------------------------------------------------------------------|----------------------------------------------------------- - getinfo | | return basic node information (id, chain hash, current block height) - connect | nodeId, host, port | open a secure connection to a lightning node - connect | uri | open a secure connection to a lightning node - open | nodeId, fundingSatoshis, pushMsat = 0, feerateSatPerByte = ?, channelFlags = 0x01 | open a channel with another lightning node, by default push = 0, feerate for the funding tx targets 6 blocks, and channel is announced - updaterelayfee | channelId, feeBaseMsat, feeProportionalMillionths | update relay fee for payments going through this channel - peers | | list existing local peers - channels | | list existing local channels - channels | nodeId | list existing local channels opened with a particular nodeId - channel | channelId | retrieve detailed information about a given channel - channelstats | | retrieves statistics about channel usage (fees, number and average amount of payments) - allnodes | | list all known nodes - allchannels | | list all known channels - allupdates | | list all channels updates - allupdates | nodeId | list all channels updates for this nodeId - receive | description | generate a payment request without a required amount (can be useful for donations) - receive | amountMsat, description | generate a payment request for a given amount - receive | amountMsat, description, expirySeconds | generate a payment request for a given amount that expires after given number of seconds - parseinvoice | paymentRequest | returns node, amount and payment hash in a payment request - findroute | paymentRequest | returns nodes and channels of the route for this payment request if there is any - findroute | paymentRequest, amountMsat | returns nodes and channels of the route for this payment request and amount, if there is any - findroute | nodeId, amountMsat | returns nodes and channels of the route to the nodeId, if there is any - send | amountMsat, paymentHash, nodeId | send a payment to a lightning node - send | paymentRequest | send a payment to a lightning node using a BOLT11 payment request - send | paymentRequest, amountMsat | send a payment to a lightning node using a BOLT11 payment request and a custom amount - checkpayment | paymentHash | returns true if the payment has been received, false otherwise - checkpayment | paymentRequest | returns true if the payment has been received, false otherwise - close | channelId | close a channel - close | channelId, scriptPubKey | close a channel and send the funds to the given scriptPubKey - forceclose | channelId | force-close a channel by publishing the local commitment tx (careful: this is more expensive than a regular close and will incur a delay before funds are spendable)" - audit | | list all send/received/relayed payments - audit | from, to | list send/received/relayed payments in that interval (from <= timestamp < to) - networkfees | | list all network fees paid to the miners, by transaction - networkfees |from, to | list network fees paid to the miners, by transaction, in that interval (from <= timestamp < to) - help | | display available methods diff --git a/README.md b/README.md index 39d1bc593..042d2912e 100644 --- a/README.md +++ b/README.md @@ -32,8 +32,6 @@ Eclair offers a feature rich HTTP API that enables application developers to eas For more information please visit the [API documentation website](https://acinq.github.io/eclair). -:warning: You can still use the old API by setting the `eclair.api.use-old-api=true` parameter, but it is now deprecated and will soon be removed. The old documentation is still available [here](OLD-API-DOCS.md). - ## Installation ### Configuring Bitcoin Core diff --git a/eclair-core/src/main/resources/reference.conf b/eclair-core/src/main/resources/reference.conf index 07fde4664..e8764cf7c 100644 --- a/eclair-core/src/main/resources/reference.conf +++ b/eclair-core/src/main/resources/reference.conf @@ -13,7 +13,6 @@ eclair { binding-ip = "127.0.0.1" port = 8080 password = "" // password for basic auth, must be non empty if json-rpc api is enabled - use-old-api = false } // override this with a script/exe that will be called everytime a new database backup has been created diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala b/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala index 809222c1b..ef756db97 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/Setup.scala @@ -291,22 +291,12 @@ class Setup(datadir: File, case "" => throw EmptyAPIPasswordException case valid => valid } - val apiRoute = if (!config.getBoolean("api.use-old-api")) { - new Service { - override val actorSystem = kit.system - override val mat = materializer - override val password = apiPassword - override val eclairApi: Eclair = new EclairImpl(kit) - }.route - } else { - new OldService { - override val scheduler = system.scheduler - override val password = apiPassword - override val getInfoResponse: Future[GetInfoResponse] = Future.successful(getInfo) - override val appKit: Kit = kit - override val socketHandler = makeSocketHandler(system)(materializer) - }.route - } + val apiRoute = new Service { + override val actorSystem = kit.system + override val mat = materializer + override val password = apiPassword + override val eclairApi: Eclair = new EclairImpl(kit) + }.route val httpBound = Http().bindAndHandle(apiRoute, config.getString("api.binding-ip"), config.getInt("api.port")).recover { case _: BindFailedException => throw TCPBindException(config.getInt("api.port")) } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/api/OldService.scala b/eclair-core/src/main/scala/fr/acinq/eclair/api/OldService.scala deleted file mode 100644 index 89910d457..000000000 --- a/eclair-core/src/main/scala/fr/acinq/eclair/api/OldService.scala +++ /dev/null @@ -1,420 +0,0 @@ -/* - * Copyright 2019 ACINQ SAS - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package fr.acinq.eclair.api - -import akka.NotUsed -import akka.actor.{Actor, ActorRef, ActorSystem, Props, Scheduler} -import akka.http.scaladsl.model.HttpMethods._ -import akka.http.scaladsl.model._ -import akka.http.scaladsl.model.headers.CacheDirectives.{`max-age`, `no-store`, public} -import akka.http.scaladsl.model.headers._ -import akka.http.scaladsl.model.ws.{Message, TextMessage} -import akka.http.scaladsl.server.Directives._ -import akka.http.scaladsl.server._ -import akka.http.scaladsl.server.directives.Credentials -import akka.http.scaladsl.server.directives.RouteDirectives.reject -import akka.pattern.ask -import akka.stream.scaladsl.{BroadcastHub, Flow, Keep, Source} -import akka.stream.{ActorMaterializer, OverflowStrategy} -import akka.util.Timeout -import de.heikoseeberger.akkahttpjson4s.Json4sSupport -import de.heikoseeberger.akkahttpjson4s.Json4sSupport.ShouldWritePretty -import fr.acinq.bitcoin.Crypto.PublicKey -import fr.acinq.bitcoin.{ByteVector32, MilliSatoshi, Satoshi} -import fr.acinq.eclair.channel._ -import fr.acinq.eclair.io.Peer.{GetPeerInfo, PeerInfo} -import fr.acinq.eclair.io.{NodeURI, Peer} -import fr.acinq.eclair.payment.PaymentLifecycle._ -import fr.acinq.eclair.payment._ -import fr.acinq.eclair.router.{ChannelDesc, RouteRequest, RouteResponse} -import fr.acinq.eclair.wire.{ChannelAnnouncement, ChannelUpdate, NodeAnnouncement} -import fr.acinq.eclair._ -import grizzled.slf4j.Logging -import org.json4s.JsonAST.{JBool, JInt, JString} -import org.json4s.{JValue, jackson} -import scodec.bits.ByteVector - -import scala.concurrent.duration._ -import scala.concurrent.{ExecutionContext, Future} -import scala.util.{Failure, Success, Try} - -// @formatter:off -case class JsonRPCBody(jsonrpc: String = "1.0", id: String = "eclair-node", method: String, params: Seq[JValue]) -case class Error(code: Int, message: String) -case class JsonRPCRes(result: AnyRef, error: Option[Error], id: String) -case class Status(node_id: String) -trait RPCRejection extends Rejection { - def requestId: String -} -final case class UnknownMethodRejection(requestId: String) extends RPCRejection -final case class UnknownParamsRejection(requestId: String, message: String) extends RPCRejection -final case class RpcValidationRejection(requestId: String, message: String) extends RPCRejection -final case class ExceptionRejection(requestId: String, message: String) extends RPCRejection -// @formatter:on - -trait OldService extends Logging { - - implicit def ec: ExecutionContext = ExecutionContext.Implicits.global - - def scheduler: Scheduler - - implicit val serialization = jackson.Serialization - implicit val formats = org.json4s.DefaultFormats + new ByteVectorSerializer + new ByteVector32Serializer + new ByteVector64Serializer + new UInt64Serializer + new MilliSatoshiSerializer + new ShortChannelIdSerializer + new StateSerializer + new ShaChainSerializer + new PublicKeySerializer + new PrivateKeySerializer + new TransactionSerializer + new TransactionWithInputInfoSerializer + new InetSocketAddressSerializer + new OutPointSerializer + new OutPointKeySerializer + new InputInfoSerializer + new ColorSerializer + new RouteResponseSerializer + new ThrowableSerializer + new FailureMessageSerializer + new NodeAddressSerializer + new DirectionSerializer +new PaymentRequestSerializer - implicit val timeout = Timeout(60 seconds) - implicit val shouldWritePretty: ShouldWritePretty = ShouldWritePretty.True - - import Json4sSupport.{marshaller, unmarshaller} - - def password: String - - def appKit: Kit - - val socketHandler: Flow[Message, TextMessage.Strict, NotUsed] - - def userPassAuthenticator(credentials: Credentials): Future[Option[String]] = credentials match { - case p@Credentials.Provided(id) if p.verify(password) => Future.successful(Some(id)) - case _ => akka.pattern.after(1 second, using = scheduler)(Future.successful(None)) // force a 1 sec pause to deter brute force - } - - val customHeaders = `Access-Control-Allow-Headers`("Content-Type, Authorization") :: - `Access-Control-Allow-Methods`(POST) :: - `Cache-Control`(public, `no-store`, `max-age`(0)) :: Nil - - val myExceptionHandler = ExceptionHandler { - case t: Throwable => - extractRequest { _ => - logger.error(s"API call failed with cause=${t.getMessage}") - complete(StatusCodes.InternalServerError, JsonRPCRes(null, Some(Error(StatusCodes.InternalServerError.intValue, t.getMessage)), "-1")) - } - } - - def completeRpcFuture(requestId: String, future: Future[AnyRef]): Route = onComplete(future) { - case Success(s) => completeRpc(requestId, s) - case Failure(t) => reject(ExceptionRejection(requestId, t.getLocalizedMessage)) - } - - def completeRpc(requestId: String, result: AnyRef): Route = complete(JsonRPCRes(result, None, requestId)) - - val myRejectionHandler: RejectionHandler = RejectionHandler.newBuilder() - .handleNotFound { - complete(StatusCodes.NotFound, JsonRPCRes(null, Some(Error(StatusCodes.NotFound.intValue, "not found")), "-1")) - } - .handle { - case _: AuthenticationFailedRejection ⇒ complete(StatusCodes.Unauthorized, JsonRPCRes(null, Some(Error(StatusCodes.Unauthorized.intValue, "Access restricted")), "-1")) - case v: RpcValidationRejection ⇒ complete(StatusCodes.BadRequest, JsonRPCRes(null, Some(Error(StatusCodes.BadRequest.intValue, v.message)), v.requestId)) - case ukm: UnknownMethodRejection ⇒ complete(StatusCodes.BadRequest, JsonRPCRes(null, Some(Error(StatusCodes.BadRequest.intValue, "method not found")), ukm.requestId)) - case p: UnknownParamsRejection ⇒ complete(StatusCodes.BadRequest, - JsonRPCRes(null, Some(Error(StatusCodes.BadRequest.intValue, s"invalid parameters for this method, should be: ${p.message}")), p.requestId)) - case m: MalformedRequestContentRejection ⇒ complete(StatusCodes.BadRequest, - JsonRPCRes(null, Some(Error(StatusCodes.BadRequest.intValue, s"malformed parameters for this method: ${m.message}")), "-1")) - case e: ExceptionRejection ⇒ complete(StatusCodes.BadRequest, - JsonRPCRes(null, Some(Error(StatusCodes.BadRequest.intValue, s"command failed: ${e.message}")), e.requestId)) - case r ⇒ logger.error(s"API call failed with cause=$r") - complete(StatusCodes.BadRequest, JsonRPCRes(null, Some(Error(StatusCodes.BadRequest.intValue, r.toString)), "-1")) - } - .result() - - val route: Route = - respondWithDefaultHeaders(customHeaders) { - withRequestTimeoutResponse(r => HttpResponse(StatusCodes.RequestTimeout).withEntity(ContentTypes.`application/json`, """{ "result": null, "error": { "code": 408, "message": "request timed out"} } """)) { - handleExceptions(myExceptionHandler) { - handleRejections(myRejectionHandler) { - authenticateBasicAsync(realm = "Access restricted", userPassAuthenticator) { _ => - pathSingleSlash { - post { - entity(as[JsonRPCBody]) { - req => - val kit = appKit - import kit._ - - req.method match { - // utility methods - case "getinfo" => completeRpcFuture(req.id, getInfoResponse) - case "help" => completeRpc(req.id, help) - - // channel lifecycle methods - case "connect" => req.params match { - case JString(pubkey) :: JString(host) :: JInt(port) :: Nil => - completeRpcFuture(req.id, (switchboard ? Peer.Connect(NodeURI.parse(s"$pubkey@$host:$port"))).mapTo[String]) - case JString(uri) :: Nil => - completeRpcFuture(req.id, (switchboard ? Peer.Connect(NodeURI.parse(uri))).mapTo[String]) - case _ => reject(UnknownParamsRejection(req.id, "[nodeId@host:port] or [nodeId, host, port]")) - } - case "open" => req.params match { - case JString(nodeId) :: JInt(fundingSatoshis) :: Nil => - completeRpcFuture(req.id, (switchboard ? Peer.OpenChannel(PublicKey(ByteVector.fromValidHex(nodeId)), Satoshi(fundingSatoshis.toLong), MilliSatoshi(0), fundingTxFeeratePerKw_opt = None, channelFlags = None, timeout_opt = None)).mapTo[String]) - case JString(nodeId) :: JInt(fundingSatoshis) :: JInt(pushMsat) :: Nil => - completeRpcFuture(req.id, (switchboard ? Peer.OpenChannel(PublicKey(ByteVector.fromValidHex(nodeId)), Satoshi(fundingSatoshis.toLong), MilliSatoshi(pushMsat.toLong), channelFlags = None, fundingTxFeeratePerKw_opt = None, timeout_opt = None)).mapTo[String]) - case JString(nodeId) :: JInt(fundingSatoshis) :: JInt(pushMsat) :: JInt(fundingFeerateSatPerByte) :: Nil => - completeRpcFuture(req.id, (switchboard ? Peer.OpenChannel(PublicKey(ByteVector.fromValidHex(nodeId)), Satoshi(fundingSatoshis.toLong), MilliSatoshi(pushMsat.toLong), fundingTxFeeratePerKw_opt = Some(feerateByte2Kw(fundingFeerateSatPerByte.toLong)), channelFlags = None, timeout_opt = None)).mapTo[String]) - case JString(nodeId) :: JInt(fundingSatoshis) :: JInt(pushMsat) :: JInt(fundingFeerateSatPerByte) :: JInt(flags) :: Nil => - completeRpcFuture(req.id, (switchboard ? Peer.OpenChannel(PublicKey(ByteVector.fromValidHex(nodeId)), Satoshi(fundingSatoshis.toLong), MilliSatoshi(pushMsat.toLong), fundingTxFeeratePerKw_opt = Some(feerateByte2Kw(fundingFeerateSatPerByte.toLong)), channelFlags = Some(flags.toByte), timeout_opt = None)).mapTo[String]) - case _ => reject(UnknownParamsRejection(req.id, s"[nodeId, fundingSatoshis], [nodeId, fundingSatoshis, pushMsat], [nodeId, fundingSatoshis, pushMsat, feerateSatPerByte] or [nodeId, fundingSatoshis, pushMsat, feerateSatPerByte, flag]")) - } - case "close" => req.params match { - case JString(identifier) :: Nil => completeRpcFuture(req.id, sendToChannel(identifier, CMD_CLOSE(scriptPubKey = None)).mapTo[String]) - case JString(identifier) :: JString(scriptPubKey) :: Nil => completeRpcFuture(req.id, sendToChannel(identifier, CMD_CLOSE(scriptPubKey = Some(ByteVector.fromValidHex(scriptPubKey)))).mapTo[String]) - case _ => reject(UnknownParamsRejection(req.id, "[channelId] or [channelId, scriptPubKey]")) - } - case "forceclose" => req.params match { - case JString(identifier) :: Nil => completeRpcFuture(req.id, sendToChannel(identifier, CMD_FORCECLOSE).mapTo[String]) - case _ => reject(UnknownParamsRejection(req.id, "[channelId]")) - } - case "updaterelayfee" => req.params match { - case JString(identifier) :: JInt(feeBaseMsat) :: JInt(feeProportionalMillionths) :: Nil => - completeRpcFuture(req.id, sendToChannel(identifier, CMD_UPDATE_RELAY_FEE(feeBaseMsat.toLong, feeProportionalMillionths.toLong)).mapTo[String]) - case JString(identifier) :: JString(feeBaseMsat) :: JString(feeProportionalMillionths) :: Nil => - completeRpcFuture(req.id, sendToChannel(identifier, CMD_UPDATE_RELAY_FEE(feeBaseMsat.toLong, feeProportionalMillionths.toLong)).mapTo[String]) - case _ => reject(UnknownParamsRejection(req.id, "[channelId] [feeBaseMsat] [feeProportionalMillionths]")) - } - // local network methods - case "peers" => completeRpcFuture(req.id, for { - peers <- (switchboard ? 'peers).mapTo[Iterable[ActorRef]] - peerinfos <- Future.sequence(peers.map(peer => (peer ? GetPeerInfo).mapTo[PeerInfo])) - } yield peerinfos) - case "channels" => req.params match { - case Nil => - val f = for { - channels_id <- (register ? 'channels).mapTo[Map[ByteVector32, ActorRef]].map(_.keys) - channels <- Future.sequence(channels_id.map(channel_id => sendToChannel(channel_id.toString(), CMD_GETINFO).mapTo[RES_GETINFO])) - } yield channels - completeRpcFuture(req.id, f) - case JString(remoteNodeId) :: Nil => Try(PublicKey(ByteVector.fromValidHex(remoteNodeId))) match { - case Success(pk) => - val f = for { - channels_id <- (register ? 'channelsTo).mapTo[Map[ByteVector32, PublicKey]].map(_.filter(_._2 == pk).keys) - channels <- Future.sequence(channels_id.map(channel_id => sendToChannel(channel_id.toString(), CMD_GETINFO).mapTo[RES_GETINFO])) - } yield channels - completeRpcFuture(req.id, f) - case Failure(_) => reject(RpcValidationRejection(req.id, s"invalid remote node id '$remoteNodeId'")) - } - case _ => reject(UnknownParamsRejection(req.id, "no arguments or [remoteNodeId]")) - } - case "channel" => req.params match { - case JString(identifier) :: Nil => completeRpcFuture(req.id, sendToChannel(identifier, CMD_GETINFO).mapTo[RES_GETINFO]) - case _ => reject(UnknownParamsRejection(req.id, "[channelId]")) - } - - // global network methods - case "allnodes" => completeRpcFuture(req.id, (router ? 'nodes).mapTo[Iterable[NodeAnnouncement]]) - case "allchannels" => completeRpcFuture(req.id, (router ? 'channels).mapTo[Iterable[ChannelAnnouncement]].map(_.map(c => ChannelDesc(c.shortChannelId, c.nodeId1, c.nodeId2)))) - case "allupdates" => req.params match { - case JString(nodeId) :: Nil => Try(PublicKey(ByteVector.fromValidHex(nodeId))) match { - case Success(pk) => completeRpcFuture(req.id, (router ? 'updatesMap).mapTo[Map[ChannelDesc, ChannelUpdate]].map(_.filter(e => e._1.a == pk || e._1.b == pk).values)) - case Failure(_) => reject(RpcValidationRejection(req.id, s"invalid remote node id '$nodeId'")) - } - case _ => completeRpcFuture(req.id, (router ? 'updates).mapTo[Iterable[ChannelUpdate]]) - } - - // payment methods - case "receive" => req.params match { - // only the payment description is given: user may want to generate a donation payment request - case JString(description) :: Nil => - completeRpcFuture(req.id, (paymentHandler ? ReceivePayment(None, description)).mapTo[PaymentRequest].map(PaymentRequest.write)) - // the amount is now given with the description - case JInt(amountMsat) :: JString(description) :: Nil => - completeRpcFuture(req.id, (paymentHandler ? ReceivePayment(Some(MilliSatoshi(amountMsat.toLong)), description)).mapTo[PaymentRequest].map(PaymentRequest.write)) - case JInt(amountMsat) :: JString(description) :: JInt(expirySeconds) :: Nil => - completeRpcFuture(req.id, (paymentHandler ? ReceivePayment(Some(MilliSatoshi(amountMsat.toLong)), description, Some(expirySeconds.toLong))).mapTo[PaymentRequest].map(PaymentRequest.write)) - case _ => reject(UnknownParamsRejection(req.id, "[description] or [amount, description] or [amount, description, expiryDuration]")) - } - - // checkinvoice deprecated. - case "parseinvoice" | "checkinvoice" => req.params match { - case JString(paymentRequest) :: Nil => Try(PaymentRequest.read(paymentRequest)) match { - case Success(pr) => completeRpc(req.id,pr) - case Failure(t) => reject(RpcValidationRejection(req.id, s"invalid payment request ${t.getMessage}")) - } - case _ => reject(UnknownParamsRejection(req.id, "[payment_request]")) - } - - case "findroute" => req.params match { - case JString(nodeId) :: JInt(amountMsat) :: Nil if nodeId.length() == 66 => Try(PublicKey(ByteVector.fromValidHex(nodeId))) match { - case Success(pk) => completeRpcFuture(req.id, (router ? RouteRequest(appKit.nodeParams.nodeId, pk, amountMsat.toLong)).mapTo[RouteResponse]) - case Failure(_) => reject(RpcValidationRejection(req.id, s"invalid nodeId hash '$nodeId'")) - } - case JString(paymentRequest) :: Nil => Try(PaymentRequest.read(paymentRequest)) match { - case Success(PaymentRequest(_, Some(amountMsat), _, nodeId , _, _)) => completeRpcFuture(req.id, (router ? RouteRequest(appKit.nodeParams.nodeId, nodeId, amountMsat.toLong)).mapTo[RouteResponse]) - case Success(_) => reject(RpcValidationRejection(req.id, s"payment request is missing amount, please specify it")) - case Failure(t) => reject(RpcValidationRejection(req.id, s"invalid payment request ${t.getLocalizedMessage}")) - } - case JString(paymentRequest) :: JInt(amountMsat) :: Nil => Try(PaymentRequest.read(paymentRequest)) match { - case Success(PaymentRequest(_, None, _, nodeId , _, _)) => completeRpcFuture(req.id, (router ? RouteRequest(appKit.nodeParams.nodeId, nodeId, amountMsat.toLong)).mapTo[RouteResponse]) - case Success(_) => reject(RpcValidationRejection(req.id, s"amount was specified both in payment request and api call")) - case Failure(t) => reject(RpcValidationRejection(req.id, s"invalid payment request ${t.getLocalizedMessage}")) - } - case _ => reject(UnknownParamsRejection(req.id, "[payment_request] or [payment_request, amountMsat] or [nodeId, amountMsat]")) - } - - case "send" => req.params match { - // user manually sets the payment information - case JInt(amountMsat) :: JString(paymentHash) :: JString(nodeId) :: Nil => - (Try(ByteVector32.fromValidHex(paymentHash)), Try(PublicKey(ByteVector.fromValidHex(nodeId)))) match { - case (Success(ph), Success(pk)) => completeRpcFuture(req.id, (paymentInitiator ? - SendPayment(amountMsat.toLong, ph, pk, maxAttempts = appKit.nodeParams.maxPaymentAttempts)).mapTo[PaymentResult].map { - case s: PaymentSucceeded => s - case f: PaymentFailed => f.copy(failures = PaymentLifecycle.transformForUser(f.failures)) - }) - case (Failure(_), _) => reject(RpcValidationRejection(req.id, s"invalid payment hash '$paymentHash'")) - case _ => reject(RpcValidationRejection(req.id, s"invalid node id '$nodeId'")) - } - // user gives a Lightning payment request - case JString(paymentRequest) :: rest => Try(PaymentRequest.read(paymentRequest)) match { - case Success(pr) => - // setting the payment amount - val amount_msat: Long = (pr.amount, rest) match { - // optional amount always overrides the amount in the payment request - case (_, JInt(amount_msat_override) :: Nil) => amount_msat_override.toLong - case (Some(amount_msat_pr), _) => amount_msat_pr.amount - case _ => throw new RuntimeException("you must manually specify an amount for this payment request") - } - logger.debug(s"api call for sending payment with amount_msat=$amount_msat") - // optional cltv expiry - val sendPayment = pr.minFinalCltvExpiry match { - case None => SendPayment(amount_msat, pr.paymentHash, pr.nodeId, maxAttempts = appKit.nodeParams.maxPaymentAttempts) - case Some(minFinalCltvExpiry) => SendPayment(amount_msat, pr.paymentHash, pr.nodeId, assistedRoutes = Nil, minFinalCltvExpiry, maxAttempts = appKit.nodeParams.maxPaymentAttempts) - } - completeRpcFuture(req.id, (paymentInitiator ? sendPayment).mapTo[PaymentResult].map { - case s: PaymentSucceeded => s - case f: PaymentFailed => f.copy(failures = PaymentLifecycle.transformForUser(f.failures)) - }) - case _ => reject(RpcValidationRejection(req.id, s"payment request is not valid")) - } - case _ => reject(UnknownParamsRejection(req.id, "[amountMsat, paymentHash, nodeId or [paymentRequest] or [paymentRequest, amountMsat]")) - } - - // check received payments - case "checkpayment" => req.params match { - case JString(identifier) :: Nil => completeRpcFuture(req.id, for { - paymentHash <- Try(PaymentRequest.read(identifier)) match { - case Success(pr) => Future.successful(pr.paymentHash) - case _ => Try(ByteVector.fromValidHex(identifier)) match { - case Success(s) => Future.successful(s) - case _ => Future.failed(new IllegalArgumentException("payment identifier must be a payment request or a payment hash")) - } - } - found <- Future(appKit.nodeParams.db.payments.getIncomingPayment(ByteVector32.fromValidHex(identifier)).map(_ => JBool(true)).getOrElse(JBool(false))) - } yield found) - case _ => reject(UnknownParamsRejection(req.id, "[paymentHash] or [paymentRequest]")) - } - - // retrieve audit events - case "audit" => - val (from, to) = req.params match { - case JInt(from) :: JInt(to) :: Nil => (from.toLong, to.toLong) - case _ => (0L, MaxEpochSeconds) - } - completeRpcFuture(req.id, Future(AuditResponse( - sent = nodeParams.db.audit.listSent(from, to), - received = nodeParams.db.audit.listReceived(from, to), - relayed = nodeParams.db.audit.listRelayed(from, to)) - )) - - case "networkfees" => - val (from, to) = req.params match { - case JInt(from) :: JInt(to) :: Nil => (from.toLong, to.toLong) - case _ => (0L, MaxEpochSeconds) - } - completeRpcFuture(req.id, Future(nodeParams.db.audit.listNetworkFees(from, to))) - - // retrieve fee stats - case "channelstats" => completeRpcFuture(req.id, Future(nodeParams.db.audit.stats)) - - - // method name was not found - case _ => reject(UnknownMethodRejection(req.id)) - } - } - } - } - } ~ path("ws") { - handleWebSocketMessages(socketHandler) - } - } - } - } - } - - def getInfoResponse: Future[GetInfoResponse] - - def makeSocketHandler(system: ActorSystem)(implicit materializer: ActorMaterializer): Flow[Message, TextMessage.Strict, NotUsed] = { - - // create a flow transforming a queue of string -> string - val (flowInput, flowOutput) = Source.queue[String](10, OverflowStrategy.dropTail).toMat(BroadcastHub.sink[String])(Keep.both).run() - - // register an actor that feeds the queue when a payment is received - system.actorOf(Props(new Actor { - override def preStart: Unit = context.system.eventStream.subscribe(self, classOf[PaymentReceived]) - def receive: Receive = { case received: PaymentReceived => flowInput.offer(received.paymentHash.toString) } - })) - - Flow[Message] - .mapConcat(_ => Nil) // Ignore heartbeats and other data from the client - .merge(flowOutput) // Stream the data we want to the client - .map(TextMessage.apply) - } - - def help = List( - "connect (uri): open a secure connection to a lightning node", - "connect (nodeId, host, port): open a secure connection to a lightning node", - "open (nodeId, fundingSatoshis, pushMsat = 0, feerateSatPerByte = ?, channelFlags = 0x01): open a channel with another lightning node, by default push = 0, feerate for the funding tx targets 6 blocks, and channel is announced", - "updaterelayfee (channelId, feeBaseMsat, feeProportionalMillionths): update relay fee for payments going through this channel", - "peers: list existing local peers", - "channels: list existing local channels", - "channels (nodeId): list existing local channels to a particular nodeId", - "channel (channelId): retrieve detailed information about a given channel", - "channelstats: retrieves statistics about channel usage (fees, number and average amount of payments)", - "allnodes: list all known nodes", - "allchannels: list all known channels", - "allupdates: list all channels updates", - "allupdates (nodeId): list all channels updates for this nodeId", - "receive (amountMsat, description): generate a payment request for a given amount", - "receive (amountMsat, description, expirySeconds): generate a payment request for a given amount with a description and a number of seconds till it expires", - "parseinvoice (paymentRequest): returns node, amount and payment hash in a payment request", - "findroute (paymentRequest): returns nodes and channels of the route if there is any", - "findroute (paymentRequest, amountMsat): returns nodes and channels of the route if there is any", - "findroute (nodeId, amountMsat): returns nodes and channels of the route if there is any", - "send (amountMsat, paymentHash, nodeId): send a payment to a lightning node", - "send (paymentRequest): send a payment to a lightning node using a BOLT11 payment request", - "send (paymentRequest, amountMsat): send a payment to a lightning node using a BOLT11 payment request and a custom amount", - "close (channelId): close a channel", - "close (channelId, scriptPubKey): close a channel and send the funds to the given scriptPubKey", - "forceclose (channelId): force-close a channel by publishing the local commitment tx (careful: this is more expensive than a regular close and will incur a delay before funds are spendable)", - "checkpayment (paymentHash): returns true if the payment has been received, false otherwise", - "checkpayment (paymentRequest): returns true if the payment has been received, false otherwise", - "audit: list all send/received/relayed payments", - "audit (from, to): list send/received/relayed payments in that interval (from <= timestamp < to)", - "networkfees: list all network fees paid to the miners, by transaction", - "networkfees (from, to): list network fees paid to the miners, by transaction, in that interval (from <= timestamp < to)", - "getinfo: returns info about the blockchain and this node", - "help: display this message") - - /** - * Sends a request to a channel and expects a response - * - * @param channelIdentifier can be a shortChannelId (BOLT encoded) or a channelId (32-byte hex encoded) - * @param request - * @return - */ - def sendToChannel(channelIdentifier: String, request: Any): Future[Any] = - for { - fwdReq <- Future(Register.ForwardShortId(ShortChannelId(channelIdentifier), request)) - .recoverWith { case _ => Future(Register.Forward(ByteVector32.fromValidHex(channelIdentifier), request)) } - .recoverWith { case _ => Future.failed(new RuntimeException(s"invalid channel identifier '$channelIdentifier'")) } - res <- appKit.register ? fwdReq - } yield res -} From a35d50d4c8cf9909ed087fe4a0a54c2deb022d3c Mon Sep 17 00:00:00 2001 From: Anton Kumaigorodski Date: Wed, 26 Jun 2019 13:05:51 +0300 Subject: [PATCH 16/21] Add `balances` API method (#962) Currently balances can be obtained from `channels` call but this requires a lot of work on caller side and also some specific knowledge (reserves, commit tx fee, in-flight payments), so this new `balances` endpoint only returns a correct balance info for each channel. --- .../main/scala/fr/acinq/eclair/Eclair.scala | 7 +++-- .../scala/fr/acinq/eclair/api/Service.scala | 3 ++ .../fr/acinq/eclair/channel/Commitments.scala | 10 +++++-- .../fr/acinq/eclair/payment/Relayer.scala | 17 +++++++---- .../scala/fr/acinq/eclair/TestUtils.scala | 1 - .../eclair/payment/ChannelSelectionSpec.scala | 15 +++++----- .../eclair/payment/HtlcGenerationSpec.scala | 10 +++++-- .../fr/acinq/eclair/payment/RelayerSpec.scala | 28 +++++++++++++------ 8 files changed, 64 insertions(+), 27 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala b/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala index 7c425328c..e42380671 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/Eclair.scala @@ -27,13 +27,14 @@ import fr.acinq.eclair.channel.Register.{Forward, ForwardShortId} import fr.acinq.eclair.channel._ import fr.acinq.eclair.db.{IncomingPayment, NetworkFee, OutgoingPayment, Stats} import fr.acinq.eclair.io.Peer.{GetPeerInfo, PeerInfo} -import fr.acinq.eclair.io.{NodeURI, Peer, Switchboard} +import fr.acinq.eclair.io.{NodeURI, Peer} import fr.acinq.eclair.payment.PaymentLifecycle._ import fr.acinq.eclair.router.{ChannelDesc, RouteRequest, RouteResponse, Router} import scodec.bits.ByteVector + import scala.concurrent.Future import scala.concurrent.duration._ -import fr.acinq.eclair.payment.{PaymentReceived, PaymentRelayed, PaymentRequest, PaymentSent} +import fr.acinq.eclair.payment.{GetUsableBalances, PaymentReceived, PaymentRelayed, PaymentRequest, PaymentSent, UsableBalances} import fr.acinq.eclair.wire.{ChannelAnnouncement, ChannelUpdate, NodeAddress, NodeAnnouncement} import TimestampQueryFilters._ @@ -105,6 +106,7 @@ trait Eclair { def getInfoResponse()(implicit timeout: Timeout): Future[GetInfoResponse] + def usableBalances()(implicit timeout: Timeout): Future[Iterable[UsableBalances]] } class EclairImpl(appKit: Kit) extends Eclair { @@ -269,4 +271,5 @@ class EclairImpl(appKit: Kit) extends Eclair { publicAddresses = appKit.nodeParams.publicAddresses) ) + override def usableBalances()(implicit timeout: Timeout): Future[Iterable[UsableBalances]] = (appKit.relayer ? GetUsableBalances).mapTo[Iterable[UsableBalances]] } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/api/Service.scala b/eclair-core/src/main/scala/fr/acinq/eclair/api/Service.scala index f816aa8be..e7f2a26e5 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/api/Service.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/api/Service.scala @@ -281,6 +281,9 @@ trait Service extends ExtraDirectives with Logging { } ~ path("channelstats") { complete(eclairApi.channelStats()) + } ~ + path("usablebalances") { + complete(eclairApi.usableBalances()) } } ~ get { path("ws") { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Commitments.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Commitments.scala index 8d15614fb..7737648e0 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Commitments.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Commitments.scala @@ -71,13 +71,19 @@ case class Commitments(localParams: LocalParams, remoteParams: RemoteParams, def addRemoteProposal(proposal: UpdateMessage): Commitments = Commitments.addRemoteProposal(this, proposal) - def announceChannel: Boolean = (channelFlags & 0x01) != 0 + val announceChannel: Boolean = (channelFlags & 0x01) != 0 - def availableBalanceForSendMsat: Long = { + lazy val availableBalanceForSendMsat: Long = { val reduced = CommitmentSpec.reduce(remoteCommit.spec, remoteChanges.acked, localChanges.proposed) val feesMsat = if (localParams.isFunder) Transactions.commitTxFee(Satoshi(remoteParams.dustLimitSatoshis), reduced).amount * 1000 else 0 reduced.toRemoteMsat - remoteParams.channelReserveSatoshis * 1000 - feesMsat } + + lazy val availableBalanceForReceiveMsat: Long = { + val reduced = CommitmentSpec.reduce(localCommit.spec, localChanges.acked, remoteChanges.proposed) + val feesMsat = if (localParams.isFunder) 0 else Transactions.commitTxFee(Satoshi(localParams.dustLimitSatoshis), reduced).amount * 1000 + reduced.toRemoteMsat - localParams.channelReserveSatoshis * 1000 - feesMsat + } } object Commitments { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/Relayer.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/Relayer.scala index 0d0b55fe4..6703be7ec 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/Relayer.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/Relayer.scala @@ -47,6 +47,9 @@ case class ForwardFulfill(fulfill: UpdateFulfillHtlc, to: Origin, htlc: UpdateAd case class ForwardFail(fail: UpdateFailHtlc, to: Origin, htlc: UpdateAddHtlc) extends ForwardMessage case class ForwardFailMalformed(fail: UpdateFailMalformedHtlc, to: Origin, htlc: UpdateAddHtlc) extends ForwardMessage +case object GetUsableBalances +case class UsableBalances(canSendMsat: Long, canReceiveMsat: Long, isPublic: Boolean) + // @formatter:on @@ -70,9 +73,13 @@ class Relayer(nodeParams: NodeParams, register: ActorRef, paymentHandler: ActorR def main(channelUpdates: Map[ShortChannelId, OutgoingChannel], node2channels: mutable.HashMap[PublicKey, mutable.Set[ShortChannelId]] with mutable.MultiMap[PublicKey, ShortChannelId]): Receive = { + case GetUsableBalances => + sender ! channelUpdates.values.map(o => UsableBalances(o.commitments.availableBalanceForSendMsat, o.commitments.availableBalanceForReceiveMsat, o.commitments.announceChannel)) + case LocalChannelUpdate(_, channelId, shortChannelId, remoteNodeId, _, channelUpdate, commitments) => log.debug(s"updating local channel info for channelId=$channelId shortChannelId=$shortChannelId remoteNodeId=$remoteNodeId channelUpdate={} commitments={}", channelUpdate, commitments) - context become main(channelUpdates + (channelUpdate.shortChannelId -> OutgoingChannel(remoteNodeId, channelUpdate, commitments.availableBalanceForSendMsat)), node2channels.addBinding(remoteNodeId, channelUpdate.shortChannelId)) + val channelUpdates1 = channelUpdates + (channelUpdate.shortChannelId -> OutgoingChannel(remoteNodeId, channelUpdate, commitments)) + context become main(channelUpdates1, node2channels.addBinding(remoteNodeId, channelUpdate.shortChannelId)) case LocalChannelDown(_, channelId, shortChannelId, remoteNodeId) => log.debug(s"removed local channel info for channelId=$channelId shortChannelId=$shortChannelId") @@ -80,7 +87,7 @@ class Relayer(nodeParams: NodeParams, register: ActorRef, paymentHandler: ActorR case AvailableBalanceChanged(_, _, shortChannelId, _, commitments) => val channelUpdates1 = channelUpdates.get(shortChannelId) match { - case Some(c: OutgoingChannel) => channelUpdates + (shortChannelId -> c.copy(availableBalanceMsat = commitments.availableBalanceForSendMsat)) + case Some(c: OutgoingChannel) => channelUpdates + (shortChannelId -> c.copy(commitments = commitments)) case None => channelUpdates // we only consider the balance if we have the channel_update } context become main(channelUpdates1, node2channels) @@ -197,7 +204,7 @@ class Relayer(nodeParams: NodeParams, register: ActorRef, paymentHandler: ActorR object Relayer { def props(nodeParams: NodeParams, register: ActorRef, paymentHandler: ActorRef) = Props(classOf[Relayer], nodeParams, register, paymentHandler) - case class OutgoingChannel(nextNodeId: PublicKey, channelUpdate: ChannelUpdate, availableBalanceMsat: Long) + case class OutgoingChannel(nextNodeId: PublicKey, channelUpdate: ChannelUpdate, commitments: Commitments) // @formatter:off sealed trait NextPayload @@ -302,10 +309,10 @@ object Relayer { val channelInfo_opt = channelUpdates.get(shortChannelId) val channelUpdate_opt = channelInfo_opt.map(_.channelUpdate) val relayResult = relayOrFail(relayPayload, channelUpdate_opt) - log.debug(s"candidate channel for htlc #${add.id} paymentHash=${add.paymentHash}: shortChannelId={} balanceMsat={} channelUpdate={} relayResult={}", shortChannelId, channelInfo_opt.map(_.availableBalanceMsat).getOrElse(""), channelUpdate_opt.getOrElse(""), relayResult) + log.debug(s"candidate channel for htlc #${add.id} paymentHash=${add.paymentHash}: shortChannelId={} balanceMsat={} channelUpdate={} relayResult={}", shortChannelId, channelInfo_opt.map(_.commitments.availableBalanceForSendMsat).getOrElse(""), channelUpdate_opt.getOrElse(""), relayResult) (shortChannelId, channelInfo_opt, relayResult) } - .collect { case (shortChannelId, Some(channelInfo), Right(_)) => (shortChannelId, channelInfo.availableBalanceMsat) } + .collect { case (shortChannelId, Some(channelInfo), Right(_)) => (shortChannelId, channelInfo.commitments.availableBalanceForSendMsat) } .filter(_._2 > relayPayload.payload.amtToForward) // we only keep channels that have enough balance to handle this payment .toList // needed for ordering .sortBy(_._2) // we want to use the channel with the lowest available balance that can process the payment diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/TestUtils.scala b/eclair-core/src/test/scala/fr/acinq/eclair/TestUtils.scala index e42ec10ce..4f5ecdcb6 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/TestUtils.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/TestUtils.scala @@ -27,5 +27,4 @@ object TestUtils { .props .get("buildDirectory") // this is defined if we run from maven .getOrElse(new File(sys.props("user.dir"), "target").getAbsolutePath) // otherwise we probably are in intellij, so we build it manually assuming that user.dir == path to the module - } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/ChannelSelectionSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/ChannelSelectionSpec.scala index 7a7508ef0..41f83245b 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/ChannelSelectionSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/ChannelSelectionSpec.scala @@ -16,14 +16,15 @@ package fr.acinq.eclair.payment -import fr.acinq.bitcoin.Block +import fr.acinq.bitcoin.{Block, ByteVector32} import fr.acinq.bitcoin.Crypto.PublicKey -import fr.acinq.eclair.channel.{AddHtlcFailed, CMD_ADD_HTLC, CMD_FAIL_HTLC} +import fr.acinq.eclair.channel.{CMD_ADD_HTLC, CMD_FAIL_HTLC} import fr.acinq.eclair.crypto.Sphinx import fr.acinq.eclair.payment.Relayer.{OutgoingChannel, RelayPayload} import fr.acinq.eclair.router.Announcements import fr.acinq.eclair.wire._ import fr.acinq.eclair.{ShortChannelId, randomBytes32, randomKey} +import fr.acinq.eclair.payment.HtlcGenerationSpec.makeCommitments import org.scalatest.FunSuite import scodec.bits.ByteVector @@ -81,11 +82,11 @@ class ChannelSelectionSpec extends FunSuite { val channelUpdate = dummyUpdate(ShortChannelId(12345), 10, 100, 1000, 100, 10000000, true) val channelUpdates = Map( - ShortChannelId(11111) -> OutgoingChannel(a, channelUpdate, 100000000), - ShortChannelId(12345) -> OutgoingChannel(a, channelUpdate, 20000000), - ShortChannelId(22222) -> OutgoingChannel(a, channelUpdate, 10000000), - ShortChannelId(33333) -> OutgoingChannel(a, channelUpdate, 100000), - ShortChannelId(44444) -> OutgoingChannel(b, channelUpdate, 1000000) + ShortChannelId(11111) -> OutgoingChannel(a, channelUpdate, makeCommitments(ByteVector32.Zeroes, 100000000)), + ShortChannelId(12345) -> OutgoingChannel(a, channelUpdate, makeCommitments(ByteVector32.Zeroes, 20000000)), + ShortChannelId(22222) -> OutgoingChannel(a, channelUpdate, makeCommitments(ByteVector32.Zeroes, 10000000)), + ShortChannelId(33333) -> OutgoingChannel(a, channelUpdate, makeCommitments(ByteVector32.Zeroes, 100000)), + ShortChannelId(44444) -> OutgoingChannel(b, channelUpdate, makeCommitments(ByteVector32.Zeroes, 1000000)) ) val node2channels = new mutable.HashMap[PublicKey, mutable.Set[ShortChannelId]] with mutable.MultiMap[PublicKey, ShortChannelId] diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/HtlcGenerationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/HtlcGenerationSpec.scala index fb4fb7718..ef4f4cb8c 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/HtlcGenerationSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/HtlcGenerationSpec.scala @@ -19,8 +19,8 @@ package fr.acinq.eclair.payment import java.util.UUID import fr.acinq.bitcoin.DeterministicWallet.ExtendedPrivateKey -import fr.acinq.bitcoin.{Block, Crypto, DeterministicWallet} -import fr.acinq.eclair.channel.Channel +import fr.acinq.bitcoin.{Block, ByteVector32, Crypto, DeterministicWallet} +import fr.acinq.eclair.channel.{Channel, Commitments} import fr.acinq.eclair.crypto.Sphinx import fr.acinq.eclair.crypto.Sphinx.{PacketAndSecrets, ParsedPacket} import fr.acinq.eclair.payment.PaymentLifecycle._ @@ -151,6 +151,12 @@ class HtlcGenerationSpec extends FunSuite { object HtlcGenerationSpec { + def makeCommitments(channelId: ByteVector32, availableBalanceForSend: Long = 50000000L, availableBalanceForReceive: Long = 50000000L) = + new Commitments(null, null, 0.toByte, null, null, null, null, 0, 0, Map.empty, null, null, null, channelId) { + override lazy val availableBalanceForSendMsat: Long = availableBalanceForSend + override lazy val availableBalanceForReceiveMsat: Long = availableBalanceForReceive + } + def randomExtendedPrivateKey: ExtendedPrivateKey = DeterministicWallet.generate(randomBytes32) val (priv_a, priv_b, priv_c, priv_d, priv_e) = (TestConstants.Alice.keyManager.nodeKey, TestConstants.Bob.keyManager.nodeKey, randomExtendedPrivateKey, randomExtendedPrivateKey, randomExtendedPrivateKey) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/RelayerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/RelayerSpec.scala index a71efb14e..27db2350b 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/RelayerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/RelayerSpec.scala @@ -25,9 +25,8 @@ import fr.acinq.eclair.channel._ import fr.acinq.eclair.crypto.Sphinx import fr.acinq.eclair.payment.PaymentLifecycle.buildCommand import fr.acinq.eclair.router.Announcements -import fr.acinq.eclair.transactions.CommitmentSpec import fr.acinq.eclair.wire._ -import fr.acinq.eclair.{ShortChannelId, TestConstants, TestkitBaseClass, UInt64, randomBytes32, randomKey} +import fr.acinq.eclair.{ShortChannelId, TestConstants, TestkitBaseClass, UInt64, randomBytes32} import org.scalatest.Outcome import scodec.bits.ByteVector @@ -58,11 +57,6 @@ class RelayerSpec extends TestkitBaseClass { val channelId_ab = randomBytes32 val channelId_bc = randomBytes32 - def makeCommitments(channelId: ByteVector32, availableBalanceMsat: Long = 50000000L) = new Commitments(null, null, 0.toByte, null, null, - null, null, 0, 0, Map.empty, null, null, null, channelId) { - override def availableBalanceForSendMsat: Long = availableBalanceMsat - } - test("relay an htlc-add") { f => import f._ val sender = TestProbe() @@ -97,7 +91,7 @@ class RelayerSpec extends TestkitBaseClass { // this is another channel B-C, with less balance (it will be preferred) val (channelId_bc_1, channelUpdate_bc_1) = (randomBytes32, channelUpdate_bc.copy(shortChannelId = ShortChannelId("500000x1x1"))) - relayer ! LocalChannelUpdate(null, channelId_bc_1, channelUpdate_bc_1.shortChannelId, c, None, channelUpdate_bc_1, makeCommitments(channelId_bc_1, availableBalanceMsat = 49000000L)) + relayer ! LocalChannelUpdate(null, channelId_bc_1, channelUpdate_bc_1.shortChannelId, c, None, channelUpdate_bc_1, makeCommitments(channelId_bc_1, 49000000L)) sender.send(relayer, ForwardAdd(add_ab)) @@ -416,4 +410,22 @@ class RelayerSpec extends TestkitBaseClass { assert(fwd.channelId === origin.originChannelId) assert(fwd.message.id === origin.originHtlcId) } + + test("get usable balances") { f => + import f._ + val sender = TestProbe() + relayer ! LocalChannelUpdate(null, channelId_ab, channelUpdate_ab.shortChannelId, a , None, channelUpdate_ab, makeCommitments(channelId_ab, 100000, 200000)) + relayer ! LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, makeCommitments(channelId_bc, 300000, 400000)) + sender.send(relayer, GetUsableBalances) + assert(sender.expectMsgType[Iterable[UsableBalances]].size === 2) + + relayer ! AvailableBalanceChanged(null, channelId_bc, channelUpdate_bc.shortChannelId, 0, makeCommitments(channelId_bc, 200000, 500000)) + sender.send(relayer, GetUsableBalances) + assert(sender.expectMsgType[Iterable[UsableBalances]].last.canReceiveMsat === 500000) + + relayer ! LocalChannelDown(null, channelId_bc, channelUpdate_bc.shortChannelId, c) + sender.send(relayer, GetUsableBalances) + val usableBalances = sender.expectMsgType[Iterable[UsableBalances]] + assert(usableBalances.size === 1 && usableBalances.head.canSendMsat === 100000) + } } From 94a526cc83296239845f49390df720c6a204349c Mon Sep 17 00:00:00 2001 From: araspitzu Date: Mon, 1 Jul 2019 09:31:37 +0200 Subject: [PATCH 17/21] Update link for recommended JDK (#1054) --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 042d2912e..e5c086d72 100644 --- a/README.md +++ b/README.md @@ -58,7 +58,7 @@ Eclair is developed in [Scala](https://www.scala-lang.org/), a powerful function * eclair-node, which is a headless application that you can run on servers and desktops, and control from the command line * eclair-node-gui, which also includes a JavaFX GUI -To run Eclair, you first need to install Java, we recommend that you use [OpenJDK 11](https://jdk.java.net/11/). Eclair will also run on Oracle JDK 1.8, Oracle JDK 11, and other versions of OpenJDK but we don't recommend using them. +To run Eclair, you first need to install Java, we recommend that you use [OpenJDK 11](https://adoptopenjdk.net/?variant=openjdk11&jvmVariant=hotspot). Eclair will also run on Oracle JDK 1.8, Oracle JDK 11, and other versions of OpenJDK but we don't recommend using them. Then download our latest [release](https://github.com/ACINQ/eclair/releases) and depending on whether or not you want a GUI run the following command: * with GUI: From 48cb8644faccb214d6f4ab786e51ba65824faf61 Mon Sep 17 00:00:00 2001 From: Pierre-Marie Padiou Date: Tue, 2 Jul 2019 10:06:57 +0200 Subject: [PATCH 18/21] Added more memory for scoverage (#1050) In order to fix the stack overflow errors related to scoverage. --- pom.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/pom.xml b/pom.xml index aa61f0343..eadb3b89d 100644 --- a/pom.xml +++ b/pom.xml @@ -130,6 +130,11 @@ -nobootcp + + -Xmx1024m + -Xms1024m + -Xss32m + ${scala.version.short} From 5e923b68570e5e01f27feeb987f77c55c28b1d5f Mon Sep 17 00:00:00 2001 From: Pierre-Marie Padiou Date: Tue, 2 Jul 2019 10:22:34 +0200 Subject: [PATCH 19/21] Make Bitcoin Core bind on localhost in tests (#1056) * minor fixes * fixed javadoc * make bitcoind only bind on localhost for tests From c4f07e8742a417db6d1d24bb030d58751809c5bb Mon Sep 17 00:00:00 2001 From: Anton Kumaigorodski Date: Tue, 2 Jul 2019 12:20:02 +0300 Subject: [PATCH 20/21] Add more data to `usablebalances` API method (#1053) Now it additionally returns remote `nodeId` and `shortChannelId` for each channel. Also negative balances are represented as `0`s. --- .../fr/acinq/eclair/channel/Commitments.scala | 4 +-- .../fr/acinq/eclair/payment/Relayer.scala | 12 ++++++--- .../src/test/resources/api/usablebalances | 1 + .../fr/acinq/eclair/api/ApiServiceSpec.scala | 27 ++++++++++++++++--- .../states/StateTestsHelperMethods.scala | 2 +- .../eclair/payment/HtlcGenerationSpec.scala | 4 +-- .../fr/acinq/eclair/payment/RelayerSpec.scala | 27 ++++++++++++++----- 7 files changed, 59 insertions(+), 18 deletions(-) create mode 100644 eclair-core/src/test/resources/api/usablebalances diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Commitments.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Commitments.scala index 7737648e0..fd044ee1c 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Commitments.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Commitments.scala @@ -76,13 +76,13 @@ case class Commitments(localParams: LocalParams, remoteParams: RemoteParams, lazy val availableBalanceForSendMsat: Long = { val reduced = CommitmentSpec.reduce(remoteCommit.spec, remoteChanges.acked, localChanges.proposed) val feesMsat = if (localParams.isFunder) Transactions.commitTxFee(Satoshi(remoteParams.dustLimitSatoshis), reduced).amount * 1000 else 0 - reduced.toRemoteMsat - remoteParams.channelReserveSatoshis * 1000 - feesMsat + math.max(reduced.toRemoteMsat - remoteParams.channelReserveSatoshis * 1000 - feesMsat, 0) } lazy val availableBalanceForReceiveMsat: Long = { val reduced = CommitmentSpec.reduce(localCommit.spec, localChanges.acked, remoteChanges.proposed) val feesMsat = if (localParams.isFunder) 0 else Transactions.commitTxFee(Satoshi(localParams.dustLimitSatoshis), reduced).amount * 1000 - reduced.toRemoteMsat - localParams.channelReserveSatoshis * 1000 - feesMsat + math.max(reduced.toRemoteMsat - localParams.channelReserveSatoshis * 1000 - feesMsat, 0) } } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/Relayer.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/Relayer.scala index 6703be7ec..dd27dc096 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/Relayer.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/Relayer.scala @@ -48,7 +48,7 @@ case class ForwardFail(fail: UpdateFailHtlc, to: Origin, htlc: UpdateAddHtlc) ex case class ForwardFailMalformed(fail: UpdateFailMalformedHtlc, to: Origin, htlc: UpdateAddHtlc) extends ForwardMessage case object GetUsableBalances -case class UsableBalances(canSendMsat: Long, canReceiveMsat: Long, isPublic: Boolean) +case class UsableBalances(remoteNodeId: PublicKey, shortChannelId: ShortChannelId, canSendMsat: Long, canReceiveMsat: Long, isPublic: Boolean) // @formatter:on @@ -72,9 +72,15 @@ class Relayer(nodeParams: NodeParams, register: ActorRef, paymentHandler: ActorR override def receive: Receive = main(Map.empty, new mutable.HashMap[PublicKey, mutable.Set[ShortChannelId]] with mutable.MultiMap[PublicKey, ShortChannelId]) def main(channelUpdates: Map[ShortChannelId, OutgoingChannel], node2channels: mutable.HashMap[PublicKey, mutable.Set[ShortChannelId]] with mutable.MultiMap[PublicKey, ShortChannelId]): Receive = { - case GetUsableBalances => - sender ! channelUpdates.values.map(o => UsableBalances(o.commitments.availableBalanceForSendMsat, o.commitments.availableBalanceForReceiveMsat, o.commitments.announceChannel)) + sender ! channelUpdates.values + .filter(o => Announcements.isEnabled(o.channelUpdate.channelFlags)) + .map(o => UsableBalances( + remoteNodeId = o.nextNodeId, + shortChannelId = o.channelUpdate.shortChannelId, + canSendMsat = o.commitments.availableBalanceForSendMsat, + canReceiveMsat = o.commitments.availableBalanceForReceiveMsat, + isPublic = o.commitments.announceChannel)) case LocalChannelUpdate(_, channelId, shortChannelId, remoteNodeId, _, channelUpdate, commitments) => log.debug(s"updating local channel info for channelId=$channelId shortChannelId=$shortChannelId remoteNodeId=$remoteNodeId channelUpdate={} commitments={}", channelUpdate, commitments) diff --git a/eclair-core/src/test/resources/api/usablebalances b/eclair-core/src/test/resources/api/usablebalances new file mode 100644 index 000000000..edbd4e5a9 --- /dev/null +++ b/eclair-core/src/test/resources/api/usablebalances @@ -0,0 +1 @@ +[{"remoteNodeId":"03af0ed6052cf28d670665549bc86f4b721c9fdb309d40c58f5811f63966e005d0","shortChannelId":"0x0x1","canSendMsat":100000000,"canReceiveMsat":20000000,"isPublic":true},{"remoteNodeId":"03af0ed6052cf28d670665549bc86f4b721c9fdb309d40c58f5811f63966e005d0","shortChannelId":"0x0x2","canSendMsat":400000000,"canReceiveMsat":30000000,"isPublic":false}] \ No newline at end of file diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala index c55864946..4c3f5d9e9 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/api/ApiServiceSpec.scala @@ -27,16 +27,14 @@ import akka.http.scaladsl.testkit.{RouteTestTimeout, ScalatestRouteTest, WSProbe import akka.stream.ActorMaterializer import akka.util.Timeout import fr.acinq.bitcoin.Crypto.PublicKey -import fr.acinq.bitcoin.{ByteVector32, Crypto, MilliSatoshi} +import fr.acinq.bitcoin.{ByteVector32, MilliSatoshi} import fr.acinq.eclair.TestConstants._ import fr.acinq.eclair._ -import fr.acinq.eclair.channel.RES_GETINFO -import fr.acinq.eclair.db.{IncomingPayment, NetworkFee, OutgoingPayment, Stats} import fr.acinq.eclair.io.NodeURI import fr.acinq.eclair.io.Peer.PeerInfo import fr.acinq.eclair.payment.PaymentLifecycle.PaymentFailed import fr.acinq.eclair.payment._ -import fr.acinq.eclair.wire.{ChannelUpdate, NodeAddress, NodeAnnouncement} +import fr.acinq.eclair.wire.NodeAddress import org.json4s.jackson.Serialization import org.mockito.scalatest.IdiomaticMockito import org.scalatest.{FunSuite, Matchers} @@ -144,6 +142,27 @@ class ApiServiceSpec extends FunSuite with ScalatestRouteTest with IdiomaticMock } } + test("'usablebalances' asks router for current usable balances") { + + val eclair = mock[Eclair] + val mockService = new MockService(eclair) + eclair.usableBalances()(any[Timeout]) returns Future.successful(List( + UsableBalances(canSendMsat = 100000000, canReceiveMsat = 20000000, shortChannelId = ShortChannelId(1), remoteNodeId = TestConstants.Alice.keyManager.nodeKey.publicKey, isPublic = true), + UsableBalances(canSendMsat = 400000000, canReceiveMsat = 30000000, shortChannelId = ShortChannelId(2), remoteNodeId = TestConstants.Alice.keyManager.nodeKey.publicKey, isPublic = false) + )) + + Post("/usablebalances") ~> + addCredentials(BasicHttpCredentials("", mockService.password)) ~> + Route.seal(mockService.route) ~> + check { + assert(handled) + assert(status == OK) + val response = entityAs[String] + eclair.usableBalances()(any[Timeout]).wasCalled(once) + matchTestJson("usablebalances", response) + } + } + test("'getinfo' response should include this node ID") { val eclair = mock[Eclair] diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/StateTestsHelperMethods.scala b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/StateTestsHelperMethods.scala index bcab510f4..28e5a3920 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/StateTestsHelperMethods.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/channel/states/StateTestsHelperMethods.scala @@ -103,7 +103,7 @@ trait StateTestsHelperMethods extends TestKitBase { bob2blockchain.expectMsgType[WatchConfirmed] // deeply buried awaitCond(alice.stateName == NORMAL) awaitCond(bob.stateName == NORMAL) - assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.availableBalanceForSendMsat == pushMsat - TestConstants.Alice.channelParams.channelReserveSatoshis * 1000) + assert(bob.stateData.asInstanceOf[DATA_NORMAL].commitments.availableBalanceForSendMsat == math.max(pushMsat - TestConstants.Alice.channelParams.channelReserveSatoshis * 1000, 0)) // x2 because alice and bob share the same relayer channelUpdateListener.expectMsgType[LocalChannelUpdate] channelUpdateListener.expectMsgType[LocalChannelUpdate] diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/HtlcGenerationSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/HtlcGenerationSpec.scala index ef4f4cb8c..2110bbc83 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/HtlcGenerationSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/HtlcGenerationSpec.scala @@ -153,8 +153,8 @@ object HtlcGenerationSpec { def makeCommitments(channelId: ByteVector32, availableBalanceForSend: Long = 50000000L, availableBalanceForReceive: Long = 50000000L) = new Commitments(null, null, 0.toByte, null, null, null, null, 0, 0, Map.empty, null, null, null, channelId) { - override lazy val availableBalanceForSendMsat: Long = availableBalanceForSend - override lazy val availableBalanceForReceiveMsat: Long = availableBalanceForReceive + override lazy val availableBalanceForSendMsat: Long = availableBalanceForSend.max(0) + override lazy val availableBalanceForReceiveMsat: Long = availableBalanceForReceive.max(0) } def randomExtendedPrivateKey: ExtendedPrivateKey = DeterministicWallet.generate(randomBytes32) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/payment/RelayerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/payment/RelayerSpec.scala index 27db2350b..fd481b5ba 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/payment/RelayerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/payment/RelayerSpec.scala @@ -414,18 +414,33 @@ class RelayerSpec extends TestkitBaseClass { test("get usable balances") { f => import f._ val sender = TestProbe() - relayer ! LocalChannelUpdate(null, channelId_ab, channelUpdate_ab.shortChannelId, a , None, channelUpdate_ab, makeCommitments(channelId_ab, 100000, 200000)) - relayer ! LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, makeCommitments(channelId_bc, 300000, 400000)) + relayer ! LocalChannelUpdate(null, channelId_ab, channelUpdate_ab.shortChannelId, a, None, channelUpdate_ab, makeCommitments(channelId_ab, -2000, 300000)) + relayer ! LocalChannelUpdate(null, channelId_bc, channelUpdate_bc.shortChannelId, c, None, channelUpdate_bc, makeCommitments(channelId_bc, 400000, -5000)) sender.send(relayer, GetUsableBalances) - assert(sender.expectMsgType[Iterable[UsableBalances]].size === 2) + val usableBalances1 = sender.expectMsgType[Iterable[UsableBalances]] + assert(usableBalances1.size === 2) + assert(usableBalances1.head.canSendMsat === 0 && usableBalances1.head.canReceiveMsat === 300000 && usableBalances1.head.shortChannelId == channelUpdate_ab.shortChannelId) + assert(usableBalances1.last.canReceiveMsat === 0 && usableBalances1.last.canSendMsat === 400000 && usableBalances1.last.shortChannelId == channelUpdate_bc.shortChannelId) relayer ! AvailableBalanceChanged(null, channelId_bc, channelUpdate_bc.shortChannelId, 0, makeCommitments(channelId_bc, 200000, 500000)) sender.send(relayer, GetUsableBalances) - assert(sender.expectMsgType[Iterable[UsableBalances]].last.canReceiveMsat === 500000) + val usableBalances2 = sender.expectMsgType[Iterable[UsableBalances]] + assert(usableBalances2.last.canReceiveMsat === 500000 && usableBalances2.last.canSendMsat === 200000) + relayer ! AvailableBalanceChanged(null, channelId_ab, channelUpdate_ab.shortChannelId, 0, makeCommitments(channelId_ab, 100000, 200000)) relayer ! LocalChannelDown(null, channelId_bc, channelUpdate_bc.shortChannelId, c) sender.send(relayer, GetUsableBalances) - val usableBalances = sender.expectMsgType[Iterable[UsableBalances]] - assert(usableBalances.size === 1 && usableBalances.head.canSendMsat === 100000) + val usableBalances3 = sender.expectMsgType[Iterable[UsableBalances]] + assert(usableBalances3.size === 1 && usableBalances3.head.canSendMsat === 100000) + + relayer ! LocalChannelUpdate(null, channelId_ab, channelUpdate_ab.shortChannelId, a, None, channelUpdate_ab.copy(channelFlags = 2), makeCommitments(channelId_ab, 100000, 200000)) + sender.send(relayer, GetUsableBalances) + val usableBalances4 = sender.expectMsgType[Iterable[UsableBalances]] + assert(usableBalances4.isEmpty) + + relayer ! LocalChannelUpdate(null, channelId_ab, channelUpdate_ab.shortChannelId, a, None, channelUpdate_ab, makeCommitments(channelId_ab, 100000, 200000)) + sender.send(relayer, GetUsableBalances) + val usableBalances5 = sender.expectMsgType[Iterable[UsableBalances]] + assert(usableBalances5.size === 1) } } From 1cc14aeda3337d0c659060ba69026bf7708f9265 Mon Sep 17 00:00:00 2001 From: Bastien Teinturier <31281497+t-bast@users.noreply.github.com> Date: Tue, 2 Jul 2019 11:53:14 +0200 Subject: [PATCH 21/21] Add TLV and TLV stream codec support (#1045) TLV (tag-length-value) types and TLV streams have been defined in the following spec PR: https://github.com/lightningnetwork/lightning-rfc/pull/607 New Lightning Messages should use TLV extensively instead of ad-hoc per-message encoding. This also allows ignoring unknown odd TLV types, which lets implementers safely test new features on mainnet without impacting legacy nodes. It also allows type re-use which speeds up new features development. Also cleaned-up and refactored common codecs. --- .../electrum/db/sqlite/SqliteWalletDb.scala | 2 +- .../fr/acinq/eclair/crypto/ShaChain.scala | 4 +- .../eclair/db/sqlite/SqlitePeersDb.scala | 4 +- .../acinq/eclair/payment/PaymentRequest.scala | 4 +- .../fr/acinq/eclair/wire/ChannelCodecs.scala | 42 +-- .../fr/acinq/eclair/wire/CommandCodecs.scala | 2 +- .../fr/acinq/eclair/wire/CommonCodecs.scala | 128 +++++++++ .../fr/acinq/eclair/wire/FailureMessage.scala | 27 +- .../eclair/wire/FixedSizeStrictCodec.scala | 75 ------ .../eclair/wire/LightningMessageCodecs.scala | 140 +++------- .../eclair/wire/LightningMessageTypes.scala | 4 - .../fr/acinq/eclair/wire/TlvCodecs.scala | 52 ++++ .../scala/fr/acinq/eclair/wire/TlvTypes.scala | 63 +++++ .../eclair/crypto/TransportHandlerSpec.scala | 14 +- .../acinq/eclair/wire/CommonCodecsSpec.scala | 253 ++++++++++++++++++ .../wire/LightningMessageCodecsSpec.scala | 132 +-------- .../fr/acinq/eclair/wire/TlvCodecsSpec.scala | 157 +++++++++++ 17 files changed, 741 insertions(+), 362 deletions(-) create mode 100644 eclair-core/src/main/scala/fr/acinq/eclair/wire/CommonCodecs.scala delete mode 100644 eclair-core/src/main/scala/fr/acinq/eclair/wire/FixedSizeStrictCodec.scala create mode 100644 eclair-core/src/main/scala/fr/acinq/eclair/wire/TlvCodecs.scala create mode 100644 eclair-core/src/main/scala/fr/acinq/eclair/wire/TlvTypes.scala create mode 100644 eclair-core/src/test/scala/fr/acinq/eclair/wire/CommonCodecsSpec.scala create mode 100644 eclair-core/src/test/scala/fr/acinq/eclair/wire/TlvCodecsSpec.scala diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/electrum/db/sqlite/SqliteWalletDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/electrum/db/sqlite/SqliteWalletDb.scala index ca7b4fbbb..907cd45f7 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/electrum/db/sqlite/SqliteWalletDb.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/blockchain/electrum/db/sqlite/SqliteWalletDb.scala @@ -136,7 +136,7 @@ class SqliteWalletDb(sqlite: Connection) extends WalletDb { object SqliteWalletDb { import fr.acinq.eclair.wire.ChannelCodecs._ - import fr.acinq.eclair.wire.LightningMessageCodecs._ + import fr.acinq.eclair.wire.CommonCodecs._ import scodec.Codec import scodec.bits.BitVector import scodec.codecs._ diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/crypto/ShaChain.scala b/eclair-core/src/main/scala/fr/acinq/eclair/crypto/ShaChain.scala index 985136ae8..94438b65e 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/crypto/ShaChain.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/crypto/ShaChain.scala @@ -17,7 +17,7 @@ package fr.acinq.eclair.crypto import fr.acinq.bitcoin._ -import fr.acinq.eclair.wire.LightningMessageCodecs +import fr.acinq.eclair.wire.CommonCodecs import scodec.Codec import scala.annotation.tailrec @@ -117,7 +117,7 @@ object ShaChain { import scodec.codecs._ // codec for a single map entry (i.e. Vector[Boolean] -> ByteVector - val entryCodec = vectorOfN(uint16, bool) ~ variableSizeBytes(uint16, LightningMessageCodecs.bytes32) + val entryCodec = vectorOfN(uint16, bool) ~ variableSizeBytes(uint16, CommonCodecs.bytes32) // codec for a Map[Vector[Boolean], ByteVector]: write all k -> v pairs using the codec defined above val mapCodec: Codec[Map[Vector[Boolean], ByteVector32]] = Codec[Map[Vector[Boolean], ByteVector32]]( diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqlitePeersDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqlitePeersDb.scala index f34d98d50..8d9e828ba 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqlitePeersDb.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqlitePeersDb.scala @@ -38,7 +38,7 @@ import SqliteUtils.ExtendedResultSet._ } override def addOrUpdatePeer(nodeId: Crypto.PublicKey, nodeaddress: NodeAddress): Unit = { - val data = LightningMessageCodecs.nodeaddress.encode(nodeaddress).require.toByteArray + val data = CommonCodecs.nodeaddress.encode(nodeaddress).require.toByteArray using(sqlite.prepareStatement("UPDATE peers SET data=? WHERE node_id=?")) { update => update.setBytes(1, data) update.setBytes(2, nodeId.value.toArray) @@ -65,7 +65,7 @@ import SqliteUtils.ExtendedResultSet._ var m: Map[PublicKey, NodeAddress] = Map() while (rs.next()) { val nodeid = PublicKey(rs.getByteVector("node_id")) - val nodeaddress = LightningMessageCodecs.nodeaddress.decode(BitVector(rs.getBytes("data"))).require.value + val nodeaddress = CommonCodecs.nodeaddress.decode(BitVector(rs.getBytes("data"))).require.value m += (nodeid -> nodeaddress) } m diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentRequest.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentRequest.scala index 9e8a9323c..b35b0d7b1 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentRequest.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentRequest.scala @@ -16,8 +16,6 @@ package fr.acinq.eclair.payment -import java.math.BigInteger - import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey} import fr.acinq.bitcoin.{MilliSatoshi, _} import fr.acinq.eclair.ShortChannelId @@ -303,7 +301,7 @@ object PaymentRequest { object Codecs { - import fr.acinq.eclair.wire.LightningMessageCodecs._ + import fr.acinq.eclair.wire.CommonCodecs._ import scodec.bits.BitVector import scodec.codecs._ import scodec.{Attempt, Codec, DecodeResult} diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/ChannelCodecs.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/ChannelCodecs.scala index 66569bc30..b5164b485 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/ChannelCodecs.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/ChannelCodecs.scala @@ -26,6 +26,7 @@ import fr.acinq.eclair.crypto.ShaChain import fr.acinq.eclair.payment.{Local, Origin, Relayed} import fr.acinq.eclair.transactions.Transactions._ import fr.acinq.eclair.transactions._ +import fr.acinq.eclair.wire.CommonCodecs._ import fr.acinq.eclair.wire.LightningMessageCodecs._ import grizzled.slf4j.Logging import scodec.bits.BitVector @@ -35,7 +36,6 @@ import scodec.{Attempt, Codec} import scala.compat.Platform import scala.concurrent.duration._ - /** * Created by PM on 02/06/2017. */ @@ -53,10 +53,10 @@ object ChannelCodecs extends Logging { val localParamsCodec: Codec[LocalParams] = ( ("nodeId" | publicKey) :: ("channelPath" | keyPathCodec) :: - ("dustLimitSatoshis" | uint64) :: - ("maxHtlcValueInFlightMsat" | uint64ex) :: - ("channelReserveSatoshis" | uint64) :: - ("htlcMinimumMsat" | uint64) :: + ("dustLimitSatoshis" | uint64overflow) :: + ("maxHtlcValueInFlightMsat" | uint64) :: + ("channelReserveSatoshis" | uint64overflow) :: + ("htlcMinimumMsat" | uint64overflow) :: ("toSelfDelay" | uint16) :: ("maxAcceptedHtlcs" | uint16) :: ("isFunder" | bool) :: @@ -66,10 +66,10 @@ object ChannelCodecs extends Logging { val remoteParamsCodec: Codec[RemoteParams] = ( ("nodeId" | publicKey) :: - ("dustLimitSatoshis" | uint64) :: - ("maxHtlcValueInFlightMsat" | uint64ex) :: - ("channelReserveSatoshis" | uint64) :: - ("htlcMinimumMsat" | uint64) :: + ("dustLimitSatoshis" | uint64overflow) :: + ("maxHtlcValueInFlightMsat" | uint64) :: + ("channelReserveSatoshis" | uint64overflow) :: + ("htlcMinimumMsat" | uint64overflow) :: ("toSelfDelay" | uint16) :: ("maxAcceptedHtlcs" | uint16) :: ("fundingPubKey" | publicKey) :: @@ -97,14 +97,14 @@ object ChannelCodecs extends Logging { val commitmentSpecCodec: Codec[CommitmentSpec] = ( ("htlcs" | setCodec(htlcCodec)) :: ("feeratePerKw" | uint32) :: - ("toLocalMsat" | uint64) :: - ("toRemoteMsat" | uint64)).as[CommitmentSpec] + ("toLocalMsat" | uint64overflow) :: + ("toRemoteMsat" | uint64overflow)).as[CommitmentSpec] - def outPointCodec: Codec[OutPoint] = variableSizeBytes(uint16, bytes.xmap(d => OutPoint.read(d.toArray), d => OutPoint.write(d))) + val outPointCodec: Codec[OutPoint] = variableSizeBytes(uint16, bytes.xmap(d => OutPoint.read(d.toArray), d => OutPoint.write(d))) - def txOutCodec: Codec[TxOut] = variableSizeBytes(uint16, bytes.xmap(d => TxOut.read(d.toArray), d => TxOut.write(d))) + val txOutCodec: Codec[TxOut] = variableSizeBytes(uint16, bytes.xmap(d => TxOut.read(d.toArray), d => TxOut.write(d))) - def txCodec: Codec[Transaction] = variableSizeBytes(uint16, bytes.xmap(d => Transaction.read(d.toArray), d => Transaction.write(d))) + val txCodec: Codec[Transaction] = variableSizeBytes(uint16, bytes.xmap(d => Transaction.read(d.toArray), d => Transaction.write(d))) val inputInfoCodec: Codec[InputInfo] = ( ("outPoint" | outPointCodec) :: @@ -142,12 +142,12 @@ object ChannelCodecs extends Logging { ("htlcTxsAndSigs" | listOfN(uint16, htlcTxAndSigsCodec))).as[PublishableTxs] val localCommitCodec: Codec[LocalCommit] = ( - ("index" | uint64) :: + ("index" | uint64overflow) :: ("spec" | commitmentSpecCodec) :: ("publishableTxs" | publishableTxsCodec)).as[LocalCommit] val remoteCommitCodec: Codec[RemoteCommit] = ( - ("index" | uint64) :: + ("index" | uint64overflow) :: ("spec" | commitmentSpecCodec) :: ("txid" | bytes32) :: ("remotePerCommitmentPoint" | publicKey)).as[RemoteCommit] @@ -167,7 +167,7 @@ object ChannelCodecs extends Logging { val waitingForRevocationCodec: Codec[WaitingForRevocation] = ( ("nextRemoteCommit" | remoteCommitCodec) :: ("sent" | commitSigCodec) :: - ("sentAfterLocalCommitIndex" | uint64) :: + ("sentAfterLocalCommitIndex" | uint64overflow) :: ("reSignAsap" | bool)).as[WaitingForRevocation] val localCodec: Codec[Local] = ( @@ -178,8 +178,8 @@ object ChannelCodecs extends Logging { val relayedCodec: Codec[Relayed] = ( ("originChannelId" | bytes32) :: ("originHtlcId" | int64) :: - ("amountMsatIn" | uint64) :: - ("amountMsatOut" | uint64)).as[Relayed] + ("amountMsatIn" | uint64overflow) :: + ("amountMsatOut" | uint64overflow)).as[Relayed] // this is for backward compatibility to handle legacy payments that didn't have identifiers val UNKNOWN_UUID = UUID.fromString("00000000-0000-0000-0000-000000000000") @@ -211,8 +211,8 @@ object ChannelCodecs extends Logging { ("remoteCommit" | remoteCommitCodec) :: ("localChanges" | localChangesCodec) :: ("remoteChanges" | remoteChangesCodec) :: - ("localNextHtlcId" | uint64) :: - ("remoteNextHtlcId" | uint64) :: + ("localNextHtlcId" | uint64overflow) :: + ("remoteNextHtlcId" | uint64overflow) :: ("originChannels" | originsMapCodec) :: ("remoteNextCommitInfo" | either(bool, waitingForRevocationCodec, publicKey)) :: ("commitInput" | inputInfoCodec) :: diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/CommandCodecs.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/CommandCodecs.scala index e70677aa5..dac5191ae 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/CommandCodecs.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/CommandCodecs.scala @@ -17,8 +17,8 @@ package fr.acinq.eclair.wire import fr.acinq.eclair.channel.{CMD_FAIL_HTLC, CMD_FAIL_MALFORMED_HTLC, CMD_FULFILL_HTLC, Command} +import fr.acinq.eclair.wire.CommonCodecs._ import fr.acinq.eclair.wire.FailureMessageCodecs.failureMessageCodec -import fr.acinq.eclair.wire.LightningMessageCodecs._ import scodec.Codec import scodec.codecs._ diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/CommonCodecs.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/CommonCodecs.scala new file mode 100644 index 000000000..143a51666 --- /dev/null +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/CommonCodecs.scala @@ -0,0 +1,128 @@ +/* + * Copyright 2019 ACINQ SAS + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package fr.acinq.eclair.wire + +import java.net.{Inet4Address, Inet6Address, InetAddress} + +import fr.acinq.bitcoin.{ByteVector32, ByteVector64} +import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey} +import fr.acinq.eclair.{ShortChannelId, UInt64} +import org.apache.commons.codec.binary.Base32 +import scodec.{Attempt, Codec, DecodeResult, Err, SizeBound} +import scodec.bits.{BitVector, ByteVector} +import scodec.codecs._ + +import scala.util.Try + +/** + * Created by t-bast on 20/06/2019. + */ + +object CommonCodecs { + + /** + * Discriminator codec with a default fallback codec (of the same type). + */ + def discriminatorWithDefault[A](discriminator: Codec[A], fallback: Codec[A]): Codec[A] = new Codec[A] { + def sizeBound: SizeBound = discriminator.sizeBound | fallback.sizeBound + + def encode(e: A): Attempt[BitVector] = discriminator.encode(e).recoverWith { case _ => fallback.encode(e) } + + def decode(b: BitVector): Attempt[DecodeResult[A]] = discriminator.decode(b).recoverWith { + case _: KnownDiscriminatorType[_]#UnknownDiscriminator => fallback.decode(b) + } + } + + // this codec can be safely used for values < 2^63 and will fail otherwise + // (for something smarter see https://github.com/yzernik/bitcoin-scodec/blob/master/src/main/scala/io/github/yzernik/bitcoinscodec/structures/UInt64.scala) + val uint64overflow: Codec[Long] = int64.narrow(l => if (l >= 0) Attempt.Successful(l) else Attempt.failure(Err(s"overflow for value $l")), l => l) + + val uint64: Codec[UInt64] = bytes(8).xmap(b => UInt64(b), a => a.toByteVector.padLeft(8)) + + val uint64L: Codec[UInt64] = bytes(8).xmap(b => UInt64(b.reverse), a => a.toByteVector.padLeft(8).reverse) + + /** + * We impose a minimal encoding on varint values to ensure that signed hashes can be reproduced easily. + * If a value could be encoded with less bytes, it's considered invalid and results in a failed decoding attempt. + * + * @param codec the integer codec (depends on the value). + * @param min the minimal value that should be encoded. + */ + def uint64min(codec: Codec[UInt64], min: UInt64): Codec[UInt64] = codec.exmap({ + case i if i < min => Attempt.failure(Err("varint was not minimally encoded")) + case i => Attempt.successful(i) + }, Attempt.successful) + + // Bitcoin-style varint codec (CompactSize). + // See https://bitcoin.org/en/developer-reference#compactsize-unsigned-integers for reference. + val varint: Codec[UInt64] = discriminatorWithDefault( + discriminated[UInt64].by(uint8L) + .\(0xff) { case i if i >= UInt64(0x100000000L) => i }(uint64min(uint64L, UInt64(0x100000000L))) + .\(0xfe) { case i if i >= UInt64(0x10000) => i }(uint64min(uint32L.xmap(UInt64(_), _.toBigInt.toLong), UInt64(0x10000))) + .\(0xfd) { case i if i >= UInt64(0xfd) => i }(uint64min(uint16L.xmap(UInt64(_), _.toBigInt.toInt), UInt64(0xfd))), + uint8L.xmap(UInt64(_), _.toBigInt.toInt) + ) + + // This codec can be safely used for values < 2^63 and will fail otherwise. + // It is useful in combination with variableSizeBytesLong to encode/decode TLV lengths because those will always be < 2^63. + val varintoverflow: Codec[Long] = varint.narrow(l => if (l <= UInt64(Long.MaxValue)) Attempt.successful(l.toBigInt.toLong) else Attempt.failure(Err(s"overflow for value $l")), l => UInt64(l)) + + val bytes32: Codec[ByteVector32] = limitedSizeBytes(32, bytesStrict(32).xmap(d => ByteVector32(d), d => d.bytes)) + + val bytes64: Codec[ByteVector64] = limitedSizeBytes(64, bytesStrict(64).xmap(d => ByteVector64(d), d => d.bytes)) + + val sha256: Codec[ByteVector32] = bytes32 + + val varsizebinarydata: Codec[ByteVector] = variableSizeBytes(uint16, bytes) + + val listofsignatures: Codec[List[ByteVector64]] = listOfN(uint16, bytes64) + + val ipv4address: Codec[Inet4Address] = bytes(4).xmap(b => InetAddress.getByAddress(b.toArray).asInstanceOf[Inet4Address], a => ByteVector(a.getAddress)) + + val ipv6address: Codec[Inet6Address] = bytes(16).exmap(b => Attempt.fromTry(Try(Inet6Address.getByAddress(null, b.toArray, null))), a => Attempt.fromTry(Try(ByteVector(a.getAddress)))) + + def base32(size: Int): Codec[String] = bytes(size).xmap(b => new Base32().encodeAsString(b.toArray).toLowerCase, a => ByteVector(new Base32().decode(a.toUpperCase()))) + + val nodeaddress: Codec[NodeAddress] = + discriminated[NodeAddress].by(uint8) + .typecase(1, (ipv4address :: uint16).as[IPv4]) + .typecase(2, (ipv6address :: uint16).as[IPv6]) + .typecase(3, (base32(10) :: uint16).as[Tor2]) + .typecase(4, (base32(35) :: uint16).as[Tor3]) + + // this one is a bit different from most other codecs: the first 'len' element is *not* the number of items + // in the list but rather the number of bytes of the encoded list. The rationale is once we've read this + // number of bytes we can just skip to the next field + val listofnodeaddresses: Codec[List[NodeAddress]] = variableSizeBytes(uint16, list(nodeaddress)) + + val shortchannelid: Codec[ShortChannelId] = int64.xmap(l => ShortChannelId(l), s => s.toLong) + + val privateKey: Codec[PrivateKey] = Codec[PrivateKey]( + (priv: PrivateKey) => bytes(32).encode(priv.value), + (wire: BitVector) => bytes(32).decode(wire).map(_.map(b => PrivateKey(b))) + ) + + val publicKey: Codec[PublicKey] = Codec[PublicKey]( + (pub: PublicKey) => bytes(33).encode(pub.value), + (wire: BitVector) => bytes(33).decode(wire).map(_.map(b => PublicKey(b))) + ) + + val rgb: Codec[Color] = bytes(3).xmap(buf => Color(buf(0), buf(1), buf(2)), t => ByteVector(t.r, t.g, t.b)) + + def zeropaddedstring(size: Int): Codec[String] = fixedSizeBytes(32, utf8).xmap(s => s.takeWhile(_ != '\u0000'), s => s) + +} diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/FailureMessage.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/FailureMessage.scala index 7039f0036..fc1233283 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/FailureMessage.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/FailureMessage.scala @@ -17,9 +17,10 @@ package fr.acinq.eclair.wire import fr.acinq.bitcoin.ByteVector32 -import fr.acinq.eclair.wire.LightningMessageCodecs.{bytes32, channelUpdateCodec, uint64} +import fr.acinq.eclair.wire.CommonCodecs.{sha256, uint64overflow} +import fr.acinq.eclair.wire.LightningMessageCodecs.channelUpdateCodec import scodec.codecs._ -import scodec.{Attempt, Codec} +import scodec.Attempt /** * see https://github.com/lightningnetwork/lightning-rfc/blob/master/04-onion-routing.md @@ -63,8 +64,6 @@ object FailureMessageCodecs { val NODE = 0x2000 val UPDATE = 0x1000 - val sha256Codec: Codec[ByteVector32] = ("sha256Codec" | bytes32) - val channelUpdateCodecWithType = LightningMessageCodecs.lightningMessageCodec.narrow[ChannelUpdate](f => Attempt.successful(f.asInstanceOf[ChannelUpdate]), g => g) // NB: for historical reasons some implementations were including/ommitting the message type (258 for ChannelUpdate) @@ -76,22 +75,22 @@ object FailureMessageCodecs { .typecase(NODE | 2, provide(TemporaryNodeFailure)) .typecase(PERM | 2, provide(PermanentNodeFailure)) .typecase(PERM | NODE | 3, provide(RequiredNodeFeatureMissing)) - .typecase(BADONION | PERM | 4, sha256Codec.as[InvalidOnionVersion]) - .typecase(BADONION | PERM | 5, sha256Codec.as[InvalidOnionHmac]) - .typecase(BADONION | PERM | 6, sha256Codec.as[InvalidOnionKey]) - .typecase(UPDATE | 7, (("channelUpdate" | channelUpdateWithLengthCodec)).as[TemporaryChannelFailure]) + .typecase(BADONION | PERM | 4, sha256.as[InvalidOnionVersion]) + .typecase(BADONION | PERM | 5, sha256.as[InvalidOnionHmac]) + .typecase(BADONION | PERM | 6, sha256.as[InvalidOnionKey]) + .typecase(UPDATE | 7, ("channelUpdate" | channelUpdateWithLengthCodec).as[TemporaryChannelFailure]) .typecase(PERM | 8, provide(PermanentChannelFailure)) .typecase(PERM | 9, provide(RequiredChannelFeatureMissing)) .typecase(PERM | 10, provide(UnknownNextPeer)) - .typecase(UPDATE | 11, (("amountMsat" | uint64) :: ("channelUpdate" | channelUpdateWithLengthCodec)).as[AmountBelowMinimum]) - .typecase(UPDATE | 12, (("amountMsat" | uint64) :: ("channelUpdate" | channelUpdateWithLengthCodec)).as[FeeInsufficient]) + .typecase(UPDATE | 11, (("amountMsat" | uint64overflow) :: ("channelUpdate" | channelUpdateWithLengthCodec)).as[AmountBelowMinimum]) + .typecase(UPDATE | 12, (("amountMsat" | uint64overflow) :: ("channelUpdate" | channelUpdateWithLengthCodec)).as[FeeInsufficient]) .typecase(UPDATE | 13, (("expiry" | uint32) :: ("channelUpdate" | channelUpdateWithLengthCodec)).as[IncorrectCltvExpiry]) - .typecase(UPDATE | 14, (("channelUpdate" | channelUpdateWithLengthCodec)).as[ExpiryTooSoon]) + .typecase(UPDATE | 14, ("channelUpdate" | channelUpdateWithLengthCodec).as[ExpiryTooSoon]) .typecase(UPDATE | 20, (("messageFlags" | byte) :: ("channelFlags" | byte) :: ("channelUpdate" | channelUpdateWithLengthCodec)).as[ChannelDisabled]) - .typecase(PERM | 15, (("amountMsat" | withDefaultValue(optional(bitsRemaining, uint64), 0L))).as[IncorrectOrUnknownPaymentDetails]) + .typecase(PERM | 15, ("amountMsat" | withDefaultValue(optional(bitsRemaining, uint64overflow), 0L)).as[IncorrectOrUnknownPaymentDetails]) .typecase(PERM | 16, provide(IncorrectPaymentAmount)) .typecase(17, provide(FinalExpiryTooSoon)) - .typecase(18, (("expiry" | uint32)).as[FinalIncorrectCltvExpiry]) - .typecase(19, (("amountMsat" | uint64)).as[FinalIncorrectHtlcAmount]) + .typecase(18, ("expiry" | uint32).as[FinalIncorrectCltvExpiry]) + .typecase(19, ("amountMsat" | uint64overflow).as[FinalIncorrectHtlcAmount]) .typecase(21, provide(ExpiryTooFar)) } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/FixedSizeStrictCodec.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/FixedSizeStrictCodec.scala deleted file mode 100644 index d5afa984c..000000000 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/FixedSizeStrictCodec.scala +++ /dev/null @@ -1,75 +0,0 @@ -/* - * Copyright 2019 ACINQ SAS - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package fr.acinq.eclair.wire - -import scodec.bits.{BitVector, ByteVector} -import scodec.{Attempt, Codec, DecodeResult, Err, SizeBound, codecs} - -/** - * - * REMOVE THIS A NEW VERSION OF SCODEC IS RELEASED THAT INCLUDES CHANGES MADE IN - * https://github.com/scodec/scodec/pull/99/files - * - * Created by PM on 02/06/2017. - */ -final class FixedSizeStrictCodec[A](size: Long, codec: Codec[A]) extends Codec[A] { - - override def sizeBound = SizeBound.exact(size) - - override def encode(a: A) = for { - encoded <- codec.encode(a) - result <- { - if (encoded.size != size) - Attempt.failure(Err(s"[$a] requires ${encoded.size} bits but field is fixed size of exactly $size bits")) - else - Attempt.successful(encoded.padTo(size)) - } - } yield result - - override def decode(buffer: BitVector) = { - if (buffer.size == size) { - codec.decode(buffer.take(size)) map { res => - DecodeResult(res.value, buffer.drop(size)) - } - } else { - Attempt.failure(Err(s"expected exactly $size bits but got ${buffer.size} bits")) - } - } - - override def toString = s"fixedSizeBitsStrict($size, $codec)" -} - -object FixedSizeStrictCodec { - /** - * Encodes by returning the supplied byte vector if its length is `size` bytes, otherwise returning error; - * decodes by taking `size * 8` bits from the supplied bit vector and converting to a byte vector. - * - * @param size number of bits to encode/decode - * @group bits - */ - def bytesStrict(size: Int): Codec[ByteVector] = new Codec[ByteVector] { - private val codec = new FixedSizeStrictCodec(size * 8L, codecs.bits).xmap[ByteVector](_.toByteVector, _.toBitVector) - - def sizeBound = codec.sizeBound - - def encode(b: ByteVector) = codec.encode(b) - - def decode(b: BitVector) = codec.decode(b) - - override def toString = s"bytesStrict($size)" - } -} \ No newline at end of file diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/LightningMessageCodecs.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/LightningMessageCodecs.scala index e984eaf5e..468573788 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/LightningMessageCodecs.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/LightningMessageCodecs.scala @@ -16,80 +16,18 @@ package fr.acinq.eclair.wire -import java.net.{Inet4Address, Inet6Address, InetAddress} - -import com.google.common.cache.{CacheBuilder, CacheLoader} -import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey} -import fr.acinq.bitcoin.{ByteVector32, ByteVector64} import fr.acinq.eclair.crypto.Sphinx -import fr.acinq.eclair.wire.FixedSizeStrictCodec.bytesStrict -import fr.acinq.eclair.{ShortChannelId, UInt64, wire} -import org.apache.commons.codec.binary.Base32 -import scodec.bits.{BitVector, ByteVector} +import fr.acinq.eclair.wire +import fr.acinq.eclair.wire.CommonCodecs._ +import scodec.bits.ByteVector import scodec.codecs._ -import scodec.{Attempt, Codec, DecodeResult, Err, SizeBound} - -import scala.util.{Failure, Success, Try} - +import scodec.Codec /** * Created by PM on 15/11/2016. */ object LightningMessageCodecs { - def attemptFromTry[T](f: => T): Attempt[T] = Try(f) match { - case Success(t) => Attempt.successful(t) - case Failure(t) => Attempt.failure(Err(s"deserialization error: ${t.getMessage}")) - } - - // this codec can be safely used for values < 2^63 and will fail otherwise - // (for something smarter see https://github.com/yzernik/bitcoin-scodec/blob/master/src/main/scala/io/github/yzernik/bitcoinscodec/structures/UInt64.scala) - val uint64: Codec[Long] = int64.narrow(l => if (l >= 0) Attempt.Successful(l) else Attempt.failure(Err(s"overflow for value $l")), l => l) - - val uint64ex: Codec[UInt64] = bytes(8).xmap(b => UInt64(b), a => a.toByteVector.padLeft(8)) - - def bytes32: Codec[ByteVector32] = limitedSizeBytes(32, bytesStrict(32).xmap(d => ByteVector32(d), d => d.bytes)) - - def bytes64: Codec[ByteVector64] = limitedSizeBytes(64, bytesStrict(64).xmap(d => ByteVector64(d), d => d.bytes)) - - def varsizebinarydata: Codec[ByteVector] = variableSizeBytes(uint16, bytes) - - def listofsignatures: Codec[List[ByteVector64]] = listOfN(uint16, bytes64) - - def ipv4address: Codec[Inet4Address] = bytes(4).xmap(b => InetAddress.getByAddress(b.toArray).asInstanceOf[Inet4Address], a => ByteVector(a.getAddress)) - - def ipv6address: Codec[Inet6Address] = bytes(16).exmap(b => attemptFromTry(Inet6Address.getByAddress(null, b.toArray, null)), a => attemptFromTry(ByteVector(a.getAddress))) - - def base32(size: Int): Codec[String] = bytes(size).xmap(b => new Base32().encodeAsString(b.toArray).toLowerCase, a => ByteVector(new Base32().decode(a.toUpperCase()))) - - def nodeaddress: Codec[NodeAddress] = - discriminated[NodeAddress].by(uint8) - .typecase(1, (ipv4address :: uint16).as[IPv4]) - .typecase(2, (ipv6address :: uint16).as[IPv6]) - .typecase(3, (base32(10) :: uint16).as[Tor2]) - .typecase(4, (base32(35) :: uint16).as[Tor3]) - - // this one is a bit different from most other codecs: the first 'len' element is *not* the number of items - // in the list but rather the number of bytes of the encoded list. The rationale is once we've read this - // number of bytes we can just skip to the next field - def listofnodeaddresses: Codec[List[NodeAddress]] = variableSizeBytes(uint16, list(nodeaddress)) - - def shortchannelid: Codec[ShortChannelId] = int64.xmap(l => ShortChannelId(l), s => s.toLong) - - def privateKey: Codec[PrivateKey] = Codec[PrivateKey]( - (priv: PrivateKey) => bytes(32).encode(priv.value), - (wire: BitVector) => bytes(32).decode(wire).map(_.map(b => PrivateKey(b))) - ) - - def publicKey: Codec[PublicKey] = Codec[PublicKey]( - (pub: PublicKey) => bytes(33).encode(pub.value), - (wire: BitVector) => bytes(33).decode(wire).map(_.map(b => PublicKey(b))) - ) - - def rgb: Codec[Color] = bytes(3).xmap(buf => Color(buf(0), buf(1), buf(2)), t => ByteVector(t.r, t.g, t.b)) - - def zeropaddedstring(size: Int): Codec[String] = fixedSizeBytes(32, utf8).xmap(s => s.takeWhile(_ != '\u0000'), s => s) - val initCodec: Codec[Init] = ( ("globalFeatures" | varsizebinarydata) :: ("localFeatures" | varsizebinarydata)).as[Init] @@ -107,20 +45,20 @@ object LightningMessageCodecs { val channelReestablishCodec: Codec[ChannelReestablish] = ( ("channelId" | bytes32) :: - ("nextLocalCommitmentNumber" | uint64) :: - ("nextRemoteRevocationNumber" | uint64) :: + ("nextLocalCommitmentNumber" | uint64overflow) :: + ("nextRemoteRevocationNumber" | uint64overflow) :: ("yourLastPerCommitmentSecret" | optional(bitsRemaining, privateKey)) :: ("myCurrentPerCommitmentPoint" | optional(bitsRemaining, publicKey))).as[ChannelReestablish] val openChannelCodec: Codec[OpenChannel] = ( ("chainHash" | bytes32) :: ("temporaryChannelId" | bytes32) :: - ("fundingSatoshis" | uint64) :: - ("pushMsat" | uint64) :: - ("dustLimitSatoshis" | uint64) :: - ("maxHtlcValueInFlightMsat" | uint64ex) :: - ("channelReserveSatoshis" | uint64) :: - ("htlcMinimumMsat" | uint64) :: + ("fundingSatoshis" | uint64overflow) :: + ("pushMsat" | uint64overflow) :: + ("dustLimitSatoshis" | uint64overflow) :: + ("maxHtlcValueInFlightMsat" | uint64) :: + ("channelReserveSatoshis" | uint64overflow) :: + ("htlcMinimumMsat" | uint64overflow) :: ("feeratePerKw" | uint32) :: ("toSelfDelay" | uint16) :: ("maxAcceptedHtlcs" | uint16) :: @@ -134,10 +72,10 @@ object LightningMessageCodecs { val acceptChannelCodec: Codec[AcceptChannel] = ( ("temporaryChannelId" | bytes32) :: - ("dustLimitSatoshis" | uint64) :: - ("maxHtlcValueInFlightMsat" | uint64ex) :: - ("channelReserveSatoshis" | uint64) :: - ("htlcMinimumMsat" | uint64) :: + ("dustLimitSatoshis" | uint64overflow) :: + ("maxHtlcValueInFlightMsat" | uint64) :: + ("channelReserveSatoshis" | uint64overflow) :: + ("htlcMinimumMsat" | uint64overflow) :: ("minimumDepth" | uint32) :: ("toSelfDelay" | uint16) :: ("maxAcceptedHtlcs" | uint16) :: @@ -168,30 +106,30 @@ object LightningMessageCodecs { val closingSignedCodec: Codec[ClosingSigned] = ( ("channelId" | bytes32) :: - ("feeSatoshis" | uint64) :: + ("feeSatoshis" | uint64overflow) :: ("signature" | bytes64)).as[ClosingSigned] val updateAddHtlcCodec: Codec[UpdateAddHtlc] = ( ("channelId" | bytes32) :: - ("id" | uint64) :: - ("amountMsat" | uint64) :: + ("id" | uint64overflow) :: + ("amountMsat" | uint64overflow) :: ("paymentHash" | bytes32) :: ("expiry" | uint32) :: ("onionRoutingPacket" | bytes(Sphinx.PacketLength))).as[UpdateAddHtlc] val updateFulfillHtlcCodec: Codec[UpdateFulfillHtlc] = ( ("channelId" | bytes32) :: - ("id" | uint64) :: + ("id" | uint64overflow) :: ("paymentPreimage" | bytes32)).as[UpdateFulfillHtlc] val updateFailHtlcCodec: Codec[UpdateFailHtlc] = ( ("channelId" | bytes32) :: - ("id" | uint64) :: + ("id" | uint64overflow) :: ("reason" | varsizebinarydata)).as[UpdateFailHtlc] val updateFailMalformedHtlcCodec: Codec[UpdateFailMalformedHtlc] = ( ("channelId" | bytes32) :: - ("id" | uint64) :: + ("id" | uint64overflow) :: ("onionHash" | bytes32) :: ("failureCode" | uint16)).as[UpdateFailMalformedHtlc] @@ -216,14 +154,13 @@ object LightningMessageCodecs { ("nodeSignature" | bytes64) :: ("bitcoinSignature" | bytes64)).as[AnnouncementSignatures] - val channelAnnouncementWitnessCodec = ( - ("features" | varsizebinarydata) :: - ("chainHash" | bytes32) :: - ("shortChannelId" | shortchannelid) :: - ("nodeId1" | publicKey) :: - ("nodeId2" | publicKey) :: - ("bitcoinKey1" | publicKey) :: - ("bitcoinKey2" | publicKey)) + val channelAnnouncementWitnessCodec = ("features" | varsizebinarydata) :: + ("chainHash" | bytes32) :: + ("shortChannelId" | shortchannelid) :: + ("nodeId1" | publicKey) :: + ("nodeId2" | publicKey) :: + ("bitcoinKey1" | publicKey) :: + ("bitcoinKey2" | publicKey) val channelAnnouncementCodec: Codec[ChannelAnnouncement] = ( ("nodeSignature1" | bytes64) :: @@ -232,13 +169,12 @@ object LightningMessageCodecs { ("bitcoinSignature2" | bytes64) :: channelAnnouncementWitnessCodec).as[ChannelAnnouncement] - val nodeAnnouncementWitnessCodec = ( - ("features" | varsizebinarydata) :: - ("timestamp" | uint32) :: - ("nodeId" | publicKey) :: - ("rgbColor" | rgb) :: - ("alias" | zeropaddedstring(32)) :: - ("addresses" | listofnodeaddresses)) + val nodeAnnouncementWitnessCodec = ("features" | varsizebinarydata) :: + ("timestamp" | uint32) :: + ("nodeId" | publicKey) :: + ("rgbColor" | rgb) :: + ("alias" | zeropaddedstring(32)) :: + ("addresses" | listofnodeaddresses) val nodeAnnouncementCodec: Codec[NodeAnnouncement] = ( ("signature" | bytes64) :: @@ -251,10 +187,10 @@ object LightningMessageCodecs { (("messageFlags" | byte) >>:~ { messageFlags => ("channelFlags" | byte) :: ("cltvExpiryDelta" | uint16) :: - ("htlcMinimumMsat" | uint64) :: + ("htlcMinimumMsat" | uint64overflow) :: ("feeBaseMsat" | uint32) :: ("feeProportionalMillionths" | uint32) :: - ("htlcMaximumMsat" | conditional((messageFlags & 1) != 0, uint64)) + ("htlcMaximumMsat" | conditional((messageFlags & 1) != 0, uint64overflow)) }) val channelUpdateCodec: Codec[ChannelUpdate] = ( @@ -324,7 +260,7 @@ object LightningMessageCodecs { val perHopPayloadCodec: Codec[PerHopPayload] = ( ("realm" | constant(ByteVector.fromByte(0))) :: ("short_channel_id" | shortchannelid) :: - ("amt_to_forward" | uint64) :: + ("amt_to_forward" | uint64overflow) :: ("outgoing_cltv_value" | uint32) :: ("unused_with_v0_version_on_header" | ignore(8 * 12))).as[PerHopPayload] diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/LightningMessageTypes.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/LightningMessageTypes.scala index e6019d7d9..9cf827eb5 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/LightningMessageTypes.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/LightningMessageTypes.scala @@ -181,10 +181,6 @@ object NodeAddress { * * We don't attempt to resolve onion addresses (it will be done by the tor proxy), so we just recognize them based on * the .onion TLD and rely on their length to separate v2/v3. - * - * @param host - * @param port - * @return */ def fromParts(host: String, port: Int): Try[NodeAddress] = Try { host match { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/TlvCodecs.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/TlvCodecs.scala new file mode 100644 index 000000000..986f7c1d3 --- /dev/null +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/TlvCodecs.scala @@ -0,0 +1,52 @@ +/* + * Copyright 2019 ACINQ SAS + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package fr.acinq.eclair.wire + +import fr.acinq.eclair.wire.CommonCodecs._ +import scodec.{Attempt, Codec} +import scodec.codecs._ + +import scala.util.Try + +/** + * Created by t-bast on 20/06/2019. + */ + +object TlvCodecs { + + val genericTlv: Codec[GenericTlv] = (("type" | varint) :: variableSizeBytesLong(varintoverflow, bytes)).as[GenericTlv] + + def tlvFallback(codec: Codec[Tlv]): Codec[Tlv] = discriminatorFallback(genericTlv, codec).xmap({ + case Left(l) => l + case Right(r) => r + }, { + case g: GenericTlv => Left(g) + case o => Right(o) + }) + + /** + * A tlv stream codec relies on an underlying tlv codec. + * This allows tlv streams to have different namespaces, increasing the total number of tlv types available. + * + * @param codec codec used for the tlv records contained in the stream. + */ + def tlvStream(codec: Codec[Tlv]): Codec[TlvStream] = list(codec).exmap( + records => Attempt.fromTry(Try(TlvStream(records))), + stream => Attempt.successful(stream.records.toList) + ) + +} diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/TlvTypes.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/TlvTypes.scala new file mode 100644 index 000000000..84d38a090 --- /dev/null +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/TlvTypes.scala @@ -0,0 +1,63 @@ +/* + * Copyright 2019 ACINQ SAS + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package fr.acinq.eclair.wire + +import fr.acinq.eclair.UInt64 +import scodec.bits.ByteVector + +import scala.annotation.tailrec + +/** + * Created by t-bast on 20/06/2019. + */ + +// @formatter:off +trait Tlv { + val `type`: UInt64 +} +sealed trait OnionTlv extends Tlv +// @formatter:on + +/** + * Generic tlv type we fallback to if we don't understand the incoming type. + * + * @param `type` tlv type. + * @param value tlv value (length is implicit, and encoded as a varint). + */ +case class GenericTlv(`type`: UInt64, value: ByteVector) extends Tlv + +/** + * A tlv stream is a collection of tlv records. + * A tlv stream is part of a given namespace that dictates how to parse the tlv records. + * That namespace is indicated by a trait extending the top-level tlv trait. + * + * @param records tlv records. + */ +case class TlvStream(records: Seq[Tlv]) { + + records.foldLeft(Option.empty[Tlv]) { + case (None, record) => + require(!record.isInstanceOf[GenericTlv] || record.`type`.toBigInt % 2 != 0, "tlv streams must not contain unknown even tlv types") + Some(record) + case (Some(previousRecord), record) => + require(record.`type` != previousRecord.`type`, "tlv streams must not contain duplicate records") + require(record.`type` > previousRecord.`type`, "tlv records must be ordered by monotonically-increasing types") + require(!record.isInstanceOf[GenericTlv] || record.`type`.toBigInt % 2 != 0, "tlv streams must not contain unknown even tlv types") + Some(record) + } + +} \ No newline at end of file diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/crypto/TransportHandlerSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/crypto/TransportHandlerSpec.scala index 4f137245b..37f2e58fe 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/crypto/TransportHandlerSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/crypto/TransportHandlerSpec.scala @@ -23,7 +23,7 @@ import akka.io.Tcp import akka.testkit.{TestActorRef, TestFSMRef, TestKit, TestProbe} import fr.acinq.eclair.crypto.Noise.{Chacha20Poly1305CipherFunctions, CipherState} import fr.acinq.eclair.crypto.TransportHandler.{Encryptor, ExtendedCipherState, Listener} -import fr.acinq.eclair.wire.LightningMessageCodecs +import fr.acinq.eclair.wire.CommonCodecs import org.scalatest.{BeforeAndAfterAll, FunSuiteLike} import scodec.Codec import scodec.bits._ @@ -49,8 +49,8 @@ class TransportHandlerSpec extends TestKit(ActorSystem("test")) with FunSuiteLik val pipe = system.actorOf(Props[MyPipe]) val probe1 = TestProbe() val probe2 = TestProbe() - val initiator = TestFSMRef(new TransportHandler(Initiator.s, Some(Responder.s.pub), pipe, LightningMessageCodecs.varsizebinarydata)) - val responder = TestFSMRef(new TransportHandler(Responder.s, None, pipe, LightningMessageCodecs.varsizebinarydata)) + val initiator = TestFSMRef(new TransportHandler(Initiator.s, Some(Responder.s.pub), pipe, CommonCodecs.varsizebinarydata)) + val responder = TestFSMRef(new TransportHandler(Responder.s, None, pipe, CommonCodecs.varsizebinarydata)) pipe ! (initiator, responder) awaitCond(initiator.stateName == TransportHandler.WaitingForListener) @@ -111,8 +111,8 @@ class TransportHandlerSpec extends TestKit(ActorSystem("test")) with FunSuiteLik val pipe = system.actorOf(Props[MyPipeSplitter]) val probe1 = TestProbe() val probe2 = TestProbe() - val initiator = TestFSMRef(new TransportHandler(Initiator.s, Some(Responder.s.pub), pipe, LightningMessageCodecs.varsizebinarydata)) - val responder = TestFSMRef(new TransportHandler(Responder.s, None, pipe, LightningMessageCodecs.varsizebinarydata)) + val initiator = TestFSMRef(new TransportHandler(Initiator.s, Some(Responder.s.pub), pipe, CommonCodecs.varsizebinarydata)) + val responder = TestFSMRef(new TransportHandler(Responder.s, None, pipe, CommonCodecs.varsizebinarydata)) pipe ! (initiator, responder) awaitCond(initiator.stateName == TransportHandler.WaitingForListener) @@ -141,8 +141,8 @@ class TransportHandlerSpec extends TestKit(ActorSystem("test")) with FunSuiteLik val pipe = system.actorOf(Props[MyPipe]) val probe1 = TestProbe() val supervisor = TestActorRef(Props(new MySupervisor())) - val initiator = TestFSMRef(new TransportHandler(Initiator.s, Some(Initiator.s.pub), pipe, LightningMessageCodecs.varsizebinarydata), supervisor, "ini") - val responder = TestFSMRef(new TransportHandler(Responder.s, None, pipe, LightningMessageCodecs.varsizebinarydata), supervisor, "res") + val initiator = TestFSMRef(new TransportHandler(Initiator.s, Some(Initiator.s.pub), pipe, CommonCodecs.varsizebinarydata), supervisor, "ini") + val responder = TestFSMRef(new TransportHandler(Responder.s, None, pipe, CommonCodecs.varsizebinarydata), supervisor, "res") probe1.watch(responder) pipe ! (initiator, responder) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/wire/CommonCodecsSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/wire/CommonCodecsSpec.scala new file mode 100644 index 000000000..f17bc4578 --- /dev/null +++ b/eclair-core/src/test/scala/fr/acinq/eclair/wire/CommonCodecsSpec.scala @@ -0,0 +1,253 @@ +/* + * Copyright 2019 ACINQ SAS + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package fr.acinq.eclair.wire + +import java.net.{Inet4Address, Inet6Address, InetAddress} + +import com.google.common.net.InetAddresses +import fr.acinq.bitcoin.Crypto.PrivateKey +import fr.acinq.eclair.{UInt64, randomBytes32} +import fr.acinq.eclair.wire.CommonCodecs._ +import org.scalatest.FunSuite +import scodec.bits.{BitVector, HexStringSyntax} + +/** + * Created by t-bast on 20/06/2019. + */ + +class CommonCodecsSpec extends FunSuite { + + test("encode/decode with uint64 codec") { + val expected = Map( + UInt64(0) -> hex"00 00 00 00 00 00 00 00", + UInt64(42) -> hex"00 00 00 00 00 00 00 2a", + UInt64(6211610197754262546L) -> hex"56 34 12 90 78 56 34 12", + UInt64(hex"ff ff ff ff ff ff ff ff") -> hex"ff ff ff ff ff ff ff ff" + ).mapValues(_.toBitVector) + + for ((uint, ref) <- expected) { + val encoded = uint64.encode(uint).require + assert(ref === encoded) + val decoded = uint64.decode(encoded).require.value + assert(uint === decoded) + } + } + + test("encode/decode with uint64L codec") { + val expected = Map( + UInt64(0) -> hex"00 00 00 00 00 00 00 00", + UInt64(42) -> hex"2a 00 00 00 00 00 00 00", + UInt64(6211610197754262546L) -> hex"12 34 56 78 90 12 34 56", + UInt64(hex"ff ff ff ff ff ff ff ff") -> hex"ff ff ff ff ff ff ff ff" + ).mapValues(_.toBitVector) + + for ((uint, ref) <- expected) { + val encoded = uint64L.encode(uint).require + assert(ref === encoded) + val decoded = uint64L.decode(encoded).require.value + assert(uint === decoded) + } + } + + test("encode/decode with varint codec") { + val expected = Map( + UInt64(0L) -> hex"00", + UInt64(42L) -> hex"2a", + UInt64(253L) -> hex"fd fd 00", + UInt64(254L) -> hex"fd fe 00", + UInt64(255L) -> hex"fd ff 00", + UInt64(550L) -> hex"fd 26 02", + UInt64(998000L) -> hex"fe 70 3a 0f 00", + UInt64(6211610197754262546L) -> hex"ff 12 34 56 78 90 12 34 56", + UInt64.MaxValue -> hex"ff ff ff ff ff ff ff ff ff" + ).mapValues(_.toBitVector) + + for ((uint, ref) <- expected) { + val encoded = varint.encode(uint).require + assert(ref === encoded, ref) + val decoded = varint.decode(encoded).require.value + assert(uint === decoded, uint) + } + } + + test("decode invalid varint") { + val testCases = Seq( + hex"fd", // truncated + hex"fe 01", // truncated + hex"fe", // truncated + hex"fe 12 34", // truncated + hex"ff", // truncated + hex"ff 12 34 56 78", // truncated + hex"fd 00 00", // not minimally-encoded + hex"fd fc 00", // not minimally-encoded + hex"fe 00 00 00 00", // not minimally-encoded + hex"fe ff ff 00 00", // not minimally-encoded + hex"ff 00 00 00 00 00 00 00 00", // not minimally-encoded + hex"ff ff ff ff 01 00 00 00 00", // not minimally-encoded + hex"ff ff ff ff ff 00 00 00 00" // not minimally-encoded + ).map(_.toBitVector) + + for (testCase <- testCases) { + assert(varint.decode(testCase).isFailure, testCase.toByteVector) + } + } + + test("encode/decode with varlong codec") { + val expected = Map( + 0L -> hex"00", + 42L -> hex"2a", + 253L -> hex"fd fd 00", + 254L -> hex"fd fe 00", + 255L -> hex"fd ff 00", + 550L -> hex"fd 26 02", + 998000L -> hex"fe 70 3a 0f 00", + 6211610197754262546L -> hex"ff 12 34 56 78 90 12 34 56", + Long.MaxValue -> hex"ff ff ff ff ff ff ff ff 7f" + ).mapValues(_.toBitVector) + + for ((long, ref) <- expected) { + val encoded = varintoverflow.encode(long).require + assert(ref === encoded, ref) + val decoded = varintoverflow.decode(encoded).require.value + assert(long === decoded, long) + } + } + + test("decode invalid varlong") { + val testCases = Seq( + hex"ff 00 00 00 00 00 00 00 80", + hex"ff ff ff ff ff ff ff ff ff" + ).map(_.toBitVector) + + for (testCase <- testCases) { + assert(varintoverflow.decode(testCase).isFailure, testCase.toByteVector) + } + } + + test("encode/decode with rgb codec") { + val color = Color(47.toByte, 255.toByte, 142.toByte) + val bin = rgb.encode(color).require + assert(bin === hex"2f ff 8e".toBitVector) + val color2 = rgb.decode(bin).require.value + assert(color === color2) + } + + test("encode/decode all kind of IPv6 addresses with ipv6address codec") { + { + // IPv4 mapped + val bin = hex"00000000000000000000ffffae8a0b08".toBitVector + val ipv6 = Inet6Address.getByAddress(null, bin.toByteArray, null) + val bin2 = ipv6address.encode(ipv6).require + assert(bin === bin2) + } + + { + // regular IPv6 address + val ipv6 = InetAddresses.forString("1080:0:0:0:8:800:200C:417A").asInstanceOf[Inet6Address] + val bin = ipv6address.encode(ipv6).require + val ipv62 = ipv6address.decode(bin).require.value + assert(ipv6 === ipv62) + } + } + + test("encode/decode with nodeaddress codec") { + { + val ipv4addr = InetAddress.getByAddress(Array[Byte](192.toByte, 168.toByte, 1.toByte, 42.toByte)).asInstanceOf[Inet4Address] + val nodeaddr = IPv4(ipv4addr, 4231) + val bin = nodeaddress.encode(nodeaddr).require + assert(bin === hex"01 C0 A8 01 2A 10 87".toBitVector) + val nodeaddr2 = nodeaddress.decode(bin).require.value + assert(nodeaddr === nodeaddr2) + } + { + val ipv6addr = InetAddress.getByAddress(hex"2001 0db8 0000 85a3 0000 0000 ac1f 8001".toArray).asInstanceOf[Inet6Address] + val nodeaddr = IPv6(ipv6addr, 4231) + val bin = nodeaddress.encode(nodeaddr).require + assert(bin === hex"02 2001 0db8 0000 85a3 0000 0000 ac1f 8001 1087".toBitVector) + val nodeaddr2 = nodeaddress.decode(bin).require.value + assert(nodeaddr === nodeaddr2) + } + { + val nodeaddr = Tor2("z4zif3fy7fe7bpg3", 4231) + val bin = nodeaddress.encode(nodeaddr).require + assert(bin === hex"03 cf3282ecb8f949f0bcdb 1087".toBitVector) + val nodeaddr2 = nodeaddress.decode(bin).require.value + assert(nodeaddr === nodeaddr2) + } + { + val nodeaddr = Tor3("mrl2d3ilhctt2vw4qzvmz3etzjvpnc6dczliq5chrxetthgbuczuggyd", 4231) + val bin = nodeaddress.encode(nodeaddr).require + assert(bin === hex"04 6457a1ed0b38a73d56dc866accec93ca6af68bc316568874478dc9399cc1a0b3431b03 1087".toBitVector) + val nodeaddr2 = nodeaddress.decode(bin).require.value + assert(nodeaddr === nodeaddr2) + } + } + + test("encode/decode with private key codec") { + val value = PrivateKey(randomBytes32) + val wire = privateKey.encode(value).require + assert(wire.length == 256) + val value1 = privateKey.decode(wire).require.value + assert(value1 == value) + } + + test("encode/decode with public key codec") { + val value = PrivateKey(randomBytes32).publicKey + val wire = CommonCodecs.publicKey.encode(value).require + assert(wire.length == 33 * 8) + val value1 = CommonCodecs.publicKey.decode(wire).require.value + assert(value1 == value) + } + + test("encode/decode with zeropaddedstring codec") { + val c = zeropaddedstring(32) + + { + val alias = "IRATEMONK" + val bin = c.encode(alias).require + assert(bin === BitVector(alias.getBytes("UTF-8") ++ Array.fill[Byte](32 - alias.length)(0))) + val alias2 = c.decode(bin).require.value + assert(alias === alias2) + } + + { + val alias = "this-alias-is-exactly-32-B-long." + val bin = c.encode(alias).require + assert(bin === BitVector(alias.getBytes("UTF-8") ++ Array.fill[Byte](32 - alias.length)(0))) + val alias2 = c.decode(bin).require.value + assert(alias === alias2) + } + + { + val alias = "this-alias-is-far-too-long-because-we-are-limited-to-32-bytes" + assert(c.encode(alias).isFailure) + } + } + + test("encode/decode UInt64") { + val codec = uint64 + Seq( + UInt64(hex"ffffffffffffffff"), + UInt64(hex"fffffffffffffffe"), + UInt64(hex"efffffffffffffff"), + UInt64(hex"effffffffffffffe") + ).map(value => { + assert(codec.decode(codec.encode(value).require).require.value === value) + }) + } + +} diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/wire/LightningMessageCodecsSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/wire/LightningMessageCodecsSpec.scala index 93917a042..700b55297 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/wire/LightningMessageCodecsSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/wire/LightningMessageCodecsSpec.scala @@ -16,9 +16,8 @@ package fr.acinq.eclair.wire -import java.net.{Inet4Address, Inet6Address, InetAddress} +import java.net.{Inet4Address, InetAddress} -import com.google.common.net.InetAddresses import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey} import fr.acinq.bitcoin.{Block, ByteVector32, ByteVector64} import fr.acinq.eclair._ @@ -44,132 +43,6 @@ class LightningMessageCodecsSpec extends FunSuite { def publicKey(fill: Byte) = PrivateKey(ByteVector.fill(32)(fill)).publicKey - test("encode/decode with uint64 codec") { - val expected = Map( - UInt64(0) -> hex"00 00 00 00 00 00 00 00", - UInt64(42) -> hex"00 00 00 00 00 00 00 2a", - UInt64(hex"ffffffffffffffff") -> hex"ff ff ff ff ff ff ff ff" - ).mapValues(_.toBitVector) - for ((uint, ref) <- expected) { - val encoded = uint64ex.encode(uint).require - assert(ref === encoded) - val decoded = uint64ex.decode(encoded).require.value - assert(uint === decoded) - } - } - - test("encode/decode with rgb codec") { - val color = Color(47.toByte, 255.toByte, 142.toByte) - val bin = rgb.encode(color).require - assert(bin === hex"2f ff 8e".toBitVector) - val color2 = rgb.decode(bin).require.value - assert(color === color2) - } - - test("encode/decode all kind of IPv6 addresses with ipv6address codec") { - { - // IPv4 mapped - val bin = hex"00000000000000000000ffffae8a0b08".toBitVector - val ipv6 = Inet6Address.getByAddress(null, bin.toByteArray, null) - val bin2 = ipv6address.encode(ipv6).require - assert(bin === bin2) - } - - { - // regular IPv6 address - val ipv6 = InetAddresses.forString("1080:0:0:0:8:800:200C:417A").asInstanceOf[Inet6Address] - val bin = ipv6address.encode(ipv6).require - val ipv62 = ipv6address.decode(bin).require.value - assert(ipv6 === ipv62) - } - } - - test("encode/decode with nodeaddress codec") { - { - val ipv4addr = InetAddress.getByAddress(Array[Byte](192.toByte, 168.toByte, 1.toByte, 42.toByte)).asInstanceOf[Inet4Address] - val nodeaddr = IPv4(ipv4addr, 4231) - val bin = nodeaddress.encode(nodeaddr).require - assert(bin === hex"01 C0 A8 01 2A 10 87".toBitVector) - val nodeaddr2 = nodeaddress.decode(bin).require.value - assert(nodeaddr === nodeaddr2) - } - { - val ipv6addr = InetAddress.getByAddress(hex"2001 0db8 0000 85a3 0000 0000 ac1f 8001".toArray).asInstanceOf[Inet6Address] - val nodeaddr = IPv6(ipv6addr, 4231) - val bin = nodeaddress.encode(nodeaddr).require - assert(bin === hex"02 2001 0db8 0000 85a3 0000 0000 ac1f 8001 1087".toBitVector) - val nodeaddr2 = nodeaddress.decode(bin).require.value - assert(nodeaddr === nodeaddr2) - } - { - val nodeaddr = Tor2("z4zif3fy7fe7bpg3", 4231) - val bin = nodeaddress.encode(nodeaddr).require - assert(bin === hex"03 cf3282ecb8f949f0bcdb 1087".toBitVector) - val nodeaddr2 = nodeaddress.decode(bin).require.value - assert(nodeaddr === nodeaddr2) - } - { - val nodeaddr = Tor3("mrl2d3ilhctt2vw4qzvmz3etzjvpnc6dczliq5chrxetthgbuczuggyd", 4231) - val bin = nodeaddress.encode(nodeaddr).require - assert(bin === hex"04 6457a1ed0b38a73d56dc866accec93ca6af68bc316568874478dc9399cc1a0b3431b03 1087".toBitVector) - val nodeaddr2 = nodeaddress.decode(bin).require.value - assert(nodeaddr === nodeaddr2) - } - } - - test("encode/decode with private key codec") { - val value = PrivateKey(randomBytes32) - val wire = LightningMessageCodecs.privateKey.encode(value).require - assert(wire.length == 256) - val value1 = LightningMessageCodecs.privateKey.decode(wire).require.value - assert(value1 == value) - } - - test("encode/decode with public key codec") { - val value = PrivateKey(randomBytes32).publicKey - val wire = LightningMessageCodecs.publicKey.encode(value).require - assert(wire.length == 33 * 8) - val value1 = LightningMessageCodecs.publicKey.decode(wire).require.value - assert(value1 == value) - } - - test("encode/decode with zeropaddedstring codec") { - val c = zeropaddedstring(32) - - { - val alias = "IRATEMONK" - val bin = c.encode(alias).require - assert(bin === BitVector(alias.getBytes("UTF-8") ++ Array.fill[Byte](32 - alias.size)(0))) - val alias2 = c.decode(bin).require.value - assert(alias === alias2) - } - - { - val alias = "this-alias-is-exactly-32-B-long." - val bin = c.encode(alias).require - assert(bin === BitVector(alias.getBytes("UTF-8") ++ Array.fill[Byte](32 - alias.size)(0))) - val alias2 = c.decode(bin).require.value - assert(alias === alias2) - } - - { - val alias = "this-alias-is-far-too-long-because-we-are-limited-to-32-bytes" - assert(c.encode(alias).isFailure) - } - } - - test("encode/decode UInt64") { - val codec = uint64ex - Seq( - UInt64(hex"ffffffffffffffff"), - UInt64(hex"fffffffffffffffe"), - UInt64(hex"efffffffffffffff"), - UInt64(hex"effffffffffffffe") - ).map(value => { - assert(codec.decode(codec.encode(value).require).require.value === value) - }) - } - test("encode/decode live node_announcements") { val anns = List( hex"a58338c9660d135fd7d087eb62afd24a33562c54507a9334e79f0dc4f17d407e6d7c61f0e2f3d0d38599502f61704cf1ae93608df027014ade7ff592f27ce26900005acdf50702d2eabbbacc7c25bbd73b39e65d28237705f7bde76f557e94fb41cb18a9ec00841122116c6e302e646563656e7465722e776f726c64000000000000000000000000000000130200000000000000000000ffffae8a0b082607" @@ -189,7 +62,6 @@ class LightningMessageCodecsSpec extends FunSuite { } test("encode/decode all channel messages") { - val open = OpenChannel(randomBytes32, randomBytes32, 3, 4, 5, UInt64(6), 7, 8, 9, 10, 11, publicKey(1), point(2), point(3), point(4), point(5), point(6), 0.toByte) val accept = AcceptChannel(randomBytes32, 3, UInt64(4), 5, 6, 7, 8, 9, publicKey(1), point(2), point(3), point(4), point(5), point(6)) val funding_created = FundingCreated(randomBytes32, bin32(0), 3, randomBytes64) @@ -222,7 +94,7 @@ class LightningMessageCodecsSpec extends FunSuite { channel_announcement :: node_announcement :: channel_update :: gossip_timestamp_filter :: query_short_channel_id :: query_channel_range :: reply_channel_range :: announcement_signatures :: ping :: pong :: channel_reestablish :: Nil msgs.foreach { - case msg => { + msg => { val encoded = lightningMessageCodec.encode(msg).require val decoded = lightningMessageCodec.decode(encoded).require assert(msg === decoded.value) diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/wire/TlvCodecsSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/wire/TlvCodecsSpec.scala new file mode 100644 index 000000000..a4c4541b8 --- /dev/null +++ b/eclair-core/src/test/scala/fr/acinq/eclair/wire/TlvCodecsSpec.scala @@ -0,0 +1,157 @@ +/* + * Copyright 2019 ACINQ SAS + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package fr.acinq.eclair.wire + +import fr.acinq.bitcoin.Crypto.PublicKey +import fr.acinq.eclair.{ShortChannelId, UInt64} +import fr.acinq.eclair.UInt64.Conversions._ +import fr.acinq.eclair.wire.CommonCodecs.{publicKey, shortchannelid, uint64, varint} +import fr.acinq.eclair.wire.TlvCodecs._ +import org.scalatest.FunSuite +import scodec.bits.HexStringSyntax +import scodec.codecs._ +import scodec.Codec + +/** + * Created by t-bast on 20/06/2019. + */ + +class TlvCodecsSpec extends FunSuite { + + import TlvCodecsSpec._ + + test("encode/decode tlv") { + val testCases = Seq( + (hex"01 08 000000000000002a", TestType1(42)), + (hex"02 08 0000000000000226", TestType2(ShortChannelId(550))), + (hex"03 31 02eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619 0000000000000231 0000000000000451", TestType3(PublicKey(hex"02eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619"), 561, 1105)), + (hex"ff1234567890123456 fd0001 10101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010010101010101", GenericTlv(6211610197754262546L, hex"10101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010010101010101")) + ) + + for ((bin, expected) <- testCases) { + val decoded = testTlvCodec.decode(bin.toBitVector).require.value.asInstanceOf[Tlv] + assert(decoded === expected) + val encoded = testTlvCodec.encode(expected).require.toByteVector + assert(encoded === bin) + } + } + + test("decode invalid tlv") { + val testCases = Seq( + hex"fd02", // type truncated + hex"fd022a", // truncated after type + hex"fd0100", // not minimally encoded type + hex"2a fd02", // length truncated + hex"2a fd0226", // truncated after length + hex"2a fe01010000", // not minimally encoded length + hex"2a fd2602 0231", // value truncated + hex"02 01 2a", // short channel id too short + hex"02 09 010101010101010101", // short channel id length too big + hex"2a ff0000000000000080" // invalid length (too big to fit inside a long) + ) + + for (testCase <- testCases) { + assert(testTlvCodec.decode(testCase.toBitVector).isFailure) + } + } + + test("decode invalid tlv stream") { + val testCases = Seq( + hex"0108000000000000002a 02", // valid tlv record followed by invalid tlv record (only type, length and value are missing) + hex"02080000000000000226 0108000000000000002a", // valid tlv records but invalid ordering + hex"02080000000000000231 02080000000000000451", // duplicate tlv type + hex"0108000000000000002a 2a0101", // unknown even type + hex"0a080000000000000231 0b0400000451" // valid tlv records but from different namespace + ) + + for (testCase <- testCases) { + assert(tlvStream(testTlvCodec).decode(testCase.toBitVector).isFailure, testCase) + } + } + + test("create invalid tlv stream") { + assertThrows[IllegalArgumentException](TlvStream(Seq(GenericTlv(42, hex"2a")))) // unknown even type + assertThrows[IllegalArgumentException](TlvStream(Seq(TestType1(561), TestType2(ShortChannelId(1105)), GenericTlv(42, hex"2a")))) // unknown even type + assertThrows[IllegalArgumentException](TlvStream(Seq(TestType1(561), TestType1(1105)))) // duplicate type + assertThrows[IllegalArgumentException](TlvStream(Seq(TestType2(ShortChannelId(1105)), TestType1(561)))) // invalid ordering + } + + test("encode/decode tlv stream") { + val bin = hex"01080000000000000231 02080000000000000451 033102eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f28368661900000000000002310000000000000451" + val expected = Seq( + TestType1(561), + TestType2(ShortChannelId(1105)), + TestType3(PublicKey(hex"02eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619"), 561, 1105) + ) + + val decoded = tlvStream(testTlvCodec).decode(bin.toBitVector).require.value + assert(decoded === TlvStream(expected)) + + val encoded = tlvStream(testTlvCodec).encode(TlvStream(expected)).require.toByteVector + assert(encoded === bin) + } + + test("encode/decode tlv stream with unknown odd type") { + val bin = hex"01080000000000000231 0b0400000451 0d02002a" + val expected = Seq( + TestType1(561), + GenericTlv(11, hex"00000451"), + TestType13(42) + ) + + val decoded = tlvStream(testTlvCodec).decode(bin.toBitVector).require.value + assert(decoded === TlvStream(expected)) + + val encoded = tlvStream(testTlvCodec).encode(TlvStream(expected)).require.toByteVector + assert(encoded === bin) + } + +} + +object TlvCodecsSpec { + + // @formatter:off + sealed trait TestTlv extends Tlv + case class TestType1(uintValue: UInt64) extends TestTlv { override val `type` = UInt64(1) } + case class TestType2(shortChannelId: ShortChannelId) extends TestTlv { override val `type` = UInt64(2) } + case class TestType3(nodeId: PublicKey, value1: UInt64, value2: UInt64) extends TestTlv { override val `type` = UInt64(3) } + case class TestType13(intValue: Int) extends TestTlv { override val `type` = UInt64(13) } + + val testCodec1: Codec[TestType1] = (("length" | constant(hex"08")) :: ("value" | uint64)).as[TestType1] + val testCodec2: Codec[TestType2] = (("length" | constant(hex"08")) :: ("short_channel_id" | shortchannelid)).as[TestType2] + val testCodec3: Codec[TestType3] = (("length" | constant(hex"31")) :: ("node_id" | publicKey) :: ("value_1" | uint64) :: ("value_2" | uint64)).as[TestType3] + val testCodec13: Codec[TestType13] = (("length" | constant(hex"02")) :: ("value" | uint16)).as[TestType13] + val testTlvCodec = tlvFallback(discriminated[Tlv].by(varint) + .typecase(1, testCodec1) + .typecase(2, testCodec2) + .typecase(3, testCodec3) + .typecase(13, testCodec13) + ) + + sealed trait OtherTlv extends Tlv + case class OtherType1(uintValue: UInt64) extends OtherTlv { override val `type` = UInt64(10) } + case class OtherType2(smallValue: Long) extends OtherTlv { override val `type` = UInt64(11) } + + val otherCodec1: Codec[OtherType1] = (("length" | constant(hex"08")) :: ("value" | uint64)).as[OtherType1] + val otherCodec2: Codec[OtherType2] = (("length" | constant(hex"04")) :: ("value" | uint32)).as[OtherType2] + val otherTlvCodec = tlvFallback(discriminated[Tlv].by(varint) + .typecase(10, otherCodec1) + .typecase(11, otherCodec2) + ) + // @formatter:on + +}