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

Replace BinaryData by scodec.bits.ByteVector (#896)

See https://github.com/ACINQ/bitcoin-lib/pull/31.

We still have to use `Array[Byte]` for low-level cryptographic primitives, and `akka.util.ByteBuffer` for tcp connections. In order to reduce unnecessary copies, we used `ByteVector.view(...)` as much as possible.

Took the opportunity to do a project-wide optimize imports. We might as well do it now since pretty much all files have been touched already.

NB: temporarily use bitcoin-lib 0.10.1-SNAPSHOT because maven central is very slow and we can't access the recently release 0.10 for now.
This commit is contained in:
Pierre-Marie Padiou 2019-03-14 12:43:28 +01:00 committed by GitHub
parent 889e86a908
commit b681cfca47
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
173 changed files with 2524 additions and 2662 deletions

View file

@ -19,7 +19,7 @@ package fr.acinq.eclair
import java.util.BitSet
import fr.acinq.bitcoin.BinaryData
import scodec.bits.ByteVector
/**
@ -39,7 +39,7 @@ object Features {
def hasFeature(features: BitSet, bit: Int): Boolean = features.get(bit)
def hasFeature(features: BinaryData, bit: Int): Boolean = hasFeature(BitSet.valueOf(features.reverse.toArray), bit)
def hasFeature(features: ByteVector, bit: Int): Boolean = hasFeature(BitSet.valueOf(features.reverse.toArray), bit)
/**
@ -58,5 +58,5 @@ object Features {
* A feature set is supported if all even bits are supported.
* We just ignore unknown odd bits.
*/
def areSupported(features: BinaryData): Boolean = areSupported(BitSet.valueOf(features.reverse.toArray))
def areSupported(features: ByteVector): Boolean = areSupported(BitSet.valueOf(features.reverse.toArray))
}

View file

@ -18,12 +18,12 @@ package fr.acinq.eclair
import akka.event.DiagnosticLoggingAdapter
import akka.event.Logging.MDC
import fr.acinq.bitcoin.BinaryData
import fr.acinq.bitcoin.ByteVector32
import fr.acinq.bitcoin.Crypto.PublicKey
object Logs {
def mdc(remoteNodeId_opt: Option[PublicKey] = None, channelId_opt: Option[BinaryData] = None): MDC =
def mdc(remoteNodeId_opt: Option[PublicKey] = None, channelId_opt: Option[ByteVector32] = None): MDC =
Seq(
remoteNodeId_opt.map(n => "nodeId" -> s" n:$n"), // nb: we preformat MDC values so that there is no white spaces in logs
channelId_opt.map(c => "channelId" -> s" c:$c")

View file

@ -22,19 +22,19 @@ import java.nio.file.Files
import java.sql.DriverManager
import java.util.concurrent.TimeUnit
import com.google.common.net.{HostAndPort, InetAddresses}
import com.typesafe.config.{Config, ConfigFactory}
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.bitcoin.{BinaryData, Block}
import fr.acinq.bitcoin.{Block, ByteVector32}
import fr.acinq.eclair.NodeParams.WatcherType
import fr.acinq.eclair.channel.Channel
import fr.acinq.eclair.crypto.KeyManager
import fr.acinq.eclair.db._
import fr.acinq.eclair.db.sqlite._
import fr.acinq.eclair.router.RouterConf
import fr.acinq.eclair.tor.Socks5ProxyParams
import fr.acinq.eclair.wire.{Color, NodeAddress}
import fr.acinq.eclair.router.RouterConf
import fr.acinq.eclair.wire.Color
import scodec.bits.ByteVector
import scala.collection.JavaConversions._
import scala.concurrent.duration.FiniteDuration
@ -45,9 +45,9 @@ case class NodeParams(keyManager: KeyManager,
alias: String,
color: Color,
publicAddresses: List[NodeAddress],
globalFeatures: BinaryData,
localFeatures: BinaryData,
overrideFeatures: Map[PublicKey, (BinaryData, BinaryData)],
globalFeatures: ByteVector,
localFeatures: ByteVector,
overrideFeatures: Map[PublicKey, (ByteVector, ByteVector)],
dustLimitSatoshis: Long,
maxHtlcValueInFlightMsat: UInt64,
maxAcceptedHtlcs: Int,
@ -74,7 +74,7 @@ case class NodeParams(keyManager: KeyManager,
maxFeerateMismatch: Double,
updateFeeMinDiffRatio: Double,
autoReconnect: Boolean,
chainHash: BinaryData,
chainHash: ByteVector32,
channelFlags: Byte,
watcherType: WatcherType,
paymentRequestExpiry: FiniteDuration,
@ -108,19 +108,19 @@ object NodeParams {
.withFallback(overrideDefaults)
.withFallback(ConfigFactory.load()).getConfig("eclair")
def getSeed(datadir: File): BinaryData = {
def getSeed(datadir: File): ByteVector = {
val seedPath = new File(datadir, "seed.dat")
seedPath.exists() match {
case true => Files.readAllBytes(seedPath.toPath)
case true => ByteVector(Files.readAllBytes(seedPath.toPath))
case false =>
datadir.mkdirs()
val seed = randomKey.toBin
Files.write(seedPath.toPath, seed)
Files.write(seedPath.toPath, seed.toArray)
seed
}
}
def makeChainHash(chain: String): BinaryData = {
def makeChainHash(chain: String): ByteVector32 = {
chain match {
case "regtest" => Block.RegtestGenesisBlock.hash
case "testnet" => Block.TestnetGenesisBlock.hash
@ -152,7 +152,7 @@ object NodeParams {
val sqliteAudit = DriverManager.getConnection(s"jdbc:sqlite:${new File(chaindir, "audit.sqlite")}")
val auditDb = new SqliteAuditDb(sqliteAudit)
val color = BinaryData(config.getString("node-color"))
val color = ByteVector.fromValidHex(config.getString("node-color"))
require(color.size == 3, "color should be a 3-bytes hex buffer")
val watcherType = config.getString("watcher-type") match {
@ -175,10 +175,10 @@ object NodeParams {
val nodeAlias = config.getString("node-alias")
require(nodeAlias.getBytes("UTF-8").length <= 32, "invalid alias, too long (max allowed 32 bytes)")
val overrideFeatures: Map[PublicKey, (BinaryData, BinaryData)] = config.getConfigList("override-features").map { e =>
val p = PublicKey(e.getString("nodeid"))
val gf = BinaryData(e.getString("global-features"))
val lf = BinaryData(e.getString("local-features"))
val overrideFeatures: Map[PublicKey, (ByteVector, ByteVector)] = config.getConfigList("override-features").map { e =>
val p = PublicKey(ByteVector.fromValidHex(e.getString("nodeid")))
val gf = ByteVector.fromValidHex(e.getString("global-features"))
val lf = ByteVector.fromValidHex(e.getString("local-features"))
(p -> (gf, lf))
}.toMap
@ -202,10 +202,10 @@ object NodeParams {
NodeParams(
keyManager = keyManager,
alias = nodeAlias,
color = Color(color.data(0), color.data(1), color.data(2)),
color = Color(color(0), color(1), color(2)),
publicAddresses = addresses,
globalFeatures = BinaryData(config.getString("global-features")),
localFeatures = BinaryData(config.getString("local-features")),
globalFeatures = ByteVector.fromValidHex(config.getString("global-features")),
localFeatures = ByteVector.fromValidHex(config.getString("local-features")),
overrideFeatures = overrideFeatures,
dustLimitSatoshis = dustLimitSatoshis,
maxHtlcValueInFlightMsat = UInt64(config.getLong("max-htlc-value-in-flight-msat")),

View file

@ -18,7 +18,6 @@ package fr.acinq.eclair
import java.io.File
import java.net.InetSocketAddress
import java.nio.file.Paths
import java.sql.DriverManager
import java.util.concurrent.TimeUnit
@ -30,7 +29,7 @@ import akka.stream.{ActorMaterializer, BindFailedException}
import akka.util.Timeout
import com.softwaremill.sttp.okhttp.OkHttpFutureBackend
import com.typesafe.config.{Config, ConfigFactory}
import fr.acinq.bitcoin.{BinaryData, Block}
import fr.acinq.bitcoin.{Block, ByteVector32}
import fr.acinq.eclair.NodeParams.{BITCOIND, ELECTRUM}
import fr.acinq.eclair.api.{GetInfoResponse, Service}
import fr.acinq.eclair.blockchain.bitcoind.rpc.{BasicBitcoinJsonRPCClient, BatchingBitcoinJsonRPCClient, ExtendedBitcoinClient}
@ -52,6 +51,7 @@ import fr.acinq.eclair.tor.{Controller, TorProtocolHandler}
import fr.acinq.eclair.wire.NodeAddress
import grizzled.slf4j.Logging
import org.json4s.JsonAST.JArray
import scodec.bits.ByteVector
import scala.concurrent._
import scala.concurrent.duration._
@ -67,7 +67,7 @@ import scala.concurrent.duration._
*/
class Setup(datadir: File,
overrideDefaults: Config = ConfigFactory.empty(),
seed_opt: Option[BinaryData] = None)(implicit system: ActorSystem) extends Logging {
seed_opt: Option[ByteVector] = None)(implicit system: ActorSystem) extends Logging {
logger.info(s"hello!")
logger.info(s"version=${getClass.getPackage.getImplementationVersion} commit=${getClass.getPackage.getSpecificationVersion}")
@ -116,7 +116,7 @@ class Setup(datadir: File,
progress = (json \ "verificationprogress").extract[Double]
blocks = (json \ "blocks").extract[Long]
headers = (json \ "headers").extract[Long]
chainHash <- bitcoinClient.invoke("getblockhash", 0).map(_.extract[String]).map(BinaryData(_)).map(x => BinaryData(x.reverse))
chainHash <- bitcoinClient.invoke("getblockhash", 0).map(_.extract[String]).map(s => ByteVector32.fromValidHex(s)).map(_.reverse)
bitcoinVersion <- bitcoinClient.invoke("getnetworkinfo").map(json => (json \ "version")).map(_.extract[Int])
unspentAddresses <- bitcoinClient.invoke("listunspent").collect { case JArray(values) =>
values

View file

@ -18,7 +18,7 @@ package fr.acinq.eclair
import java.math.BigInteger
import fr.acinq.bitcoin.BinaryData
import scodec.bits.ByteVector
case class UInt64(private val underlying: BigInt) extends Ordered[UInt64] {
@ -27,7 +27,7 @@ case class UInt64(private val underlying: BigInt) extends Ordered[UInt64] {
override def compare(o: UInt64): Int = underlying.compare(o.underlying)
def toByteArray: Array[Byte] = underlying.toByteArray.takeRight(8)
def toByteVector: ByteVector = ByteVector.view(underlying.toByteArray.takeRight(8))
def toBigInt: BigInt = underlying
@ -41,7 +41,7 @@ object UInt64 {
val MaxValue = UInt64(MaxValueBigInt)
def apply(bin: BinaryData) = new UInt64(new BigInteger(1, bin))
def apply(bin: ByteVector) = new UInt64(new BigInteger(1, bin.toArray))
def apply(value: Long) = new UInt64(BigInt(value))

View file

@ -20,7 +20,7 @@ import java.net.InetSocketAddress
import com.google.common.net.HostAndPort
import fr.acinq.bitcoin.Crypto.{Point, PrivateKey, PublicKey, Scalar}
import fr.acinq.bitcoin.{BinaryData, MilliSatoshi, OutPoint, Transaction}
import fr.acinq.bitcoin.{ByteVector32, MilliSatoshi, OutPoint, Transaction}
import fr.acinq.eclair.channel.State
import fr.acinq.eclair.crypto.ShaChain
import fr.acinq.eclair.payment.PaymentRequest
@ -31,13 +31,18 @@ import fr.acinq.eclair.wire._
import fr.acinq.eclair.{ShortChannelId, UInt64}
import org.json4s.JsonAST._
import org.json4s.{CustomKeySerializer, CustomSerializer}
import scodec.bits.ByteVector
/**
* JSON Serializers.
* Note: in general, deserialization does not need to be implemented.
*/
class BinaryDataSerializer extends CustomSerializer[BinaryData](format => ({ null }, {
case x: BinaryData => JString(x.toString())
class ByteVectorSerializer extends CustomSerializer[ByteVector](format => ({ null }, {
case x: ByteVector => JString(x.toHex)
}))
class ByteVector32Serializer extends CustomSerializer[ByteVector32](format => ({ null }, {
case x: ByteVector32 => JString(x.toHex)
}))
class UInt64Serializer extends CustomSerializer[UInt64](format => ({ null }, {

View file

@ -28,24 +28,25 @@ 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.{ActorMaterializer, OverflowStrategy}
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.{BinaryData, MilliSatoshi, Satoshi}
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, Router}
import fr.acinq.eclair.router.{ChannelDesc, RouteRequest, RouteResponse}
import fr.acinq.eclair.wire.{ChannelAnnouncement, ChannelUpdate, NodeAddress, NodeAnnouncement}
import fr.acinq.eclair.{Kit, ShortChannelId, feerateByte2Kw}
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}
@ -56,7 +57,7 @@ case class JsonRPCBody(jsonrpc: String = "1.0", id: String = "eclair-node", meth
case class Error(code: Int, message: String)
case class JsonRPCRes(result: AnyRef, error: Option[Error], id: String)
case class Status(node_id: String)
case class GetInfoResponse(nodeId: PublicKey, alias: String, port: Int, chainHash: BinaryData, blockHeight: Int, publicAddresses: Seq[NodeAddress])
case class GetInfoResponse(nodeId: PublicKey, alias: String, port: Int, chainHash: ByteVector32, blockHeight: Int, publicAddresses: Seq[NodeAddress])
case class AuditResponse(sent: Seq[PaymentSent], received: Seq[PaymentReceived], relayed: Seq[PaymentRelayed])
trait RPCRejection extends Rejection {
def requestId: String
@ -74,7 +75,7 @@ trait Service extends Logging {
def scheduler: Scheduler
implicit val serialization = jackson.Serialization
implicit val formats = org.json4s.DefaultFormats + new BinaryDataSerializer + 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 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 timeout = Timeout(60 seconds)
implicit val shouldWritePretty: ShouldWritePretty = ShouldWritePretty.True
@ -157,18 +158,18 @@ trait Service extends Logging {
}
case "open" => req.params match {
case JString(nodeId) :: JInt(fundingSatoshis) :: Nil =>
completeRpcFuture(req.id, (switchboard ? Peer.OpenChannel(PublicKey(nodeId), Satoshi(fundingSatoshis.toLong), MilliSatoshi(0), fundingTxFeeratePerKw_opt = None, channelFlags = None)).mapTo[String])
completeRpcFuture(req.id, (switchboard ? Peer.OpenChannel(PublicKey(ByteVector.fromValidHex(nodeId)), Satoshi(fundingSatoshis.toLong), MilliSatoshi(0), fundingTxFeeratePerKw_opt = None, channelFlags = None)).mapTo[String])
case JString(nodeId) :: JInt(fundingSatoshis) :: JInt(pushMsat) :: Nil =>
completeRpcFuture(req.id, (switchboard ? Peer.OpenChannel(PublicKey(nodeId), Satoshi(fundingSatoshis.toLong), MilliSatoshi(pushMsat.toLong), channelFlags = None, fundingTxFeeratePerKw_opt = None)).mapTo[String])
completeRpcFuture(req.id, (switchboard ? Peer.OpenChannel(PublicKey(ByteVector.fromValidHex(nodeId)), Satoshi(fundingSatoshis.toLong), MilliSatoshi(pushMsat.toLong), channelFlags = None, fundingTxFeeratePerKw_opt = None)).mapTo[String])
case JString(nodeId) :: JInt(fundingSatoshis) :: JInt(pushMsat) :: JInt(fundingFeerateSatPerByte) :: Nil =>
completeRpcFuture(req.id, (switchboard ? Peer.OpenChannel(PublicKey(nodeId), Satoshi(fundingSatoshis.toLong), MilliSatoshi(pushMsat.toLong), fundingTxFeeratePerKw_opt = Some(feerateByte2Kw(fundingFeerateSatPerByte.toLong)), channelFlags = None)).mapTo[String])
completeRpcFuture(req.id, (switchboard ? Peer.OpenChannel(PublicKey(ByteVector.fromValidHex(nodeId)), Satoshi(fundingSatoshis.toLong), MilliSatoshi(pushMsat.toLong), fundingTxFeeratePerKw_opt = Some(feerateByte2Kw(fundingFeerateSatPerByte.toLong)), channelFlags = None)).mapTo[String])
case JString(nodeId) :: JInt(fundingSatoshis) :: JInt(pushMsat) :: JInt(fundingFeerateSatPerByte) :: JInt(flags) :: Nil =>
completeRpcFuture(req.id, (switchboard ? Peer.OpenChannel(PublicKey(nodeId), Satoshi(fundingSatoshis.toLong), MilliSatoshi(pushMsat.toLong), fundingTxFeeratePerKw_opt = Some(feerateByte2Kw(fundingFeerateSatPerByte.toLong)), channelFlags = Some(flags.toByte))).mapTo[String])
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))).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(scriptPubKey))).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 {
@ -190,14 +191,14 @@ trait Service extends Logging {
case "channels" => req.params match {
case Nil =>
val f = for {
channels_id <- (register ? 'channels).mapTo[Map[BinaryData, ActorRef]].map(_.keys)
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(remoteNodeId)) match {
case JString(remoteNodeId) :: Nil => Try(PublicKey(ByteVector.fromValidHex(remoteNodeId))) match {
case Success(pk) =>
val f = for {
channels_id <- (register ? 'channelsTo).mapTo[Map[BinaryData, PublicKey]].map(_.filter(_._2 == pk).keys)
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)
@ -214,7 +215,7 @@ trait Service extends Logging {
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(nodeId)) 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'"))
}
@ -244,7 +245,7 @@ trait Service extends Logging {
}
case "findroute" => req.params match {
case JString(nodeId) :: JInt(amountMsat) :: Nil if nodeId.length() == 66 => Try(PublicKey(nodeId)) 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'"))
}
@ -264,7 +265,7 @@ trait Service extends Logging {
case "send" => req.params match {
// user manually sets the payment information
case JInt(amountMsat) :: JString(paymentHash) :: JString(nodeId) :: Nil =>
(Try(BinaryData(paymentHash)), Try(PublicKey(nodeId))) match {
(Try(ByteVector32.fromValidHex(paymentHash)), Try(PublicKey(ByteVector.fromValidHex(nodeId)))) match {
case (Success(ph), Success(pk)) => completeRpcFuture(req.id, (paymentInitiator ?
SendPayment(amountMsat.toLong, ph, pk)).mapTo[PaymentResult].map {
case s: PaymentSucceeded => s
@ -303,12 +304,12 @@ trait Service extends Logging {
case JString(identifier) :: Nil => completeRpcFuture(req.id, for {
paymentHash <- Try(PaymentRequest.read(identifier)) match {
case Success(pr) => Future.successful(pr.paymentHash)
case _ => Try(BinaryData(identifier)) match {
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 <- (paymentHandler ? CheckPayment(paymentHash)).map(found => new JBool(found.asInstanceOf[Boolean]))
found <- (paymentHandler ? CheckPayment(ByteVector32.fromValidHex(identifier))).map(found => new JBool(found.asInstanceOf[Boolean]))
} yield found)
case _ => reject(UnknownParamsRejection(req.id, "[paymentHash] or [paymentRequest]"))
}
@ -414,7 +415,7 @@ trait Service extends Logging {
def sendToChannel(channelIdentifier: String, request: Any): Future[Any] =
for {
fwdReq <- Future(Register.ForwardShortId(ShortChannelId(channelIdentifier), request))
.recoverWith { case _ => Future(Register.Forward(BinaryData(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

View file

@ -16,7 +16,8 @@
package fr.acinq.eclair.blockchain
import fr.acinq.bitcoin.{BinaryData, Satoshi, Transaction}
import fr.acinq.bitcoin.{Satoshi, Transaction}
import scodec.bits.ByteVector
import scala.concurrent.Future
@ -29,7 +30,7 @@ trait EclairWallet {
def getFinalAddress: Future[String]
def makeFundingTx(pubkeyScript: BinaryData, amount: Satoshi, feeRatePerKw: Long): Future[MakeFundingTxResponse]
def makeFundingTx(pubkeyScript: ByteVector, amount: Satoshi, feeRatePerKw: Long): Future[MakeFundingTxResponse]
/**
* Committing *must* include publishing the transaction on the network.

View file

@ -18,9 +18,10 @@ package fr.acinq.eclair.blockchain
import akka.actor.ActorRef
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.bitcoin.{BinaryData, Script, ScriptWitness, Transaction}
import fr.acinq.bitcoin.{ByteVector32, Script, ScriptWitness, Transaction}
import fr.acinq.eclair.channel.BitcoinEvent
import fr.acinq.eclair.wire.ChannelAnnouncement
import scodec.bits.ByteVector
import scala.util.{Failure, Success, Try}
@ -35,13 +36,13 @@ sealed trait Watch {
def event: BitcoinEvent
}
// we need a public key script to use electrum apis
final case class WatchConfirmed(channel: ActorRef, txId: BinaryData, publicKeyScript: BinaryData, minDepth: Long, event: BitcoinEvent) extends Watch
final case class WatchConfirmed(channel: ActorRef, txId: ByteVector32, publicKeyScript: ByteVector, minDepth: Long, event: BitcoinEvent) extends Watch
object WatchConfirmed {
// if we have the entire transaction, we can get the redeemScript from the witness, and re-compute the publicKeyScript
// we support both p2pkh and p2wpkh scripts
def apply(channel: ActorRef, tx: Transaction, minDepth: Long, event: BitcoinEvent): WatchConfirmed = WatchConfirmed(channel, tx.txid, tx.txOut.map(_.publicKeyScript).headOption.getOrElse(""), minDepth, event)
def apply(channel: ActorRef, tx: Transaction, minDepth: Long, event: BitcoinEvent): WatchConfirmed = WatchConfirmed(channel, tx.txid, tx.txOut.map(_.publicKeyScript).headOption.getOrElse(ByteVector.empty), minDepth, event)
def extractPublicKeyScript(witness: ScriptWitness): BinaryData = Try(PublicKey(witness.stack.last)) match {
def extractPublicKeyScript(witness: ScriptWitness): ByteVector = Try(PublicKey(witness.stack.last)) match {
case Success(pubKey) =>
// if last element of the witness is a public key, then this is a p2wpkh
Script.write(Script.pay2wpkh(pubKey))
@ -51,18 +52,18 @@ object WatchConfirmed {
}
}
final case class WatchSpent(channel: ActorRef, txId: BinaryData, outputIndex: Int, publicKeyScript: BinaryData, event: BitcoinEvent) extends Watch
final case class WatchSpent(channel: ActorRef, txId: ByteVector32, outputIndex: Int, publicKeyScript: ByteVector, event: BitcoinEvent) extends Watch
object WatchSpent {
// if we have the entire transaction, we can get the publicKeyScript from the relevant output
def apply(channel: ActorRef, tx: Transaction, outputIndex: Int, event: BitcoinEvent): WatchSpent = WatchSpent(channel, tx.txid, outputIndex, tx.txOut(outputIndex).publicKeyScript, event)
}
final case class WatchSpentBasic(channel: ActorRef, txId: BinaryData, outputIndex: Int, publicKeyScript: BinaryData, event: BitcoinEvent) extends Watch // we use this when we don't care about the spending tx, and we also assume txid already exists
final case class WatchSpentBasic(channel: ActorRef, txId: ByteVector32, outputIndex: Int, publicKeyScript: ByteVector, event: BitcoinEvent) extends Watch // we use this when we don't care about the spending tx, and we also assume txid already exists
object WatchSpentBasic {
// if we have the entire transaction, we can get the publicKeyScript from the relevant output
def apply(channel: ActorRef, tx: Transaction, outputIndex: Int, event: BitcoinEvent): WatchSpentBasic = WatchSpentBasic(channel, tx.txid, outputIndex, tx.txOut(outputIndex).publicKeyScript, event)
}
// TODO: notify me if confirmation number gets below minDepth?
final case class WatchLost(channel: ActorRef, txId: BinaryData, minDepth: Long, event: BitcoinEvent) extends Watch
final case class WatchLost(channel: ActorRef, txId: ByteVector32, minDepth: Long, event: BitcoinEvent) extends Watch
trait WatchEvent {
def event: BitcoinEvent

View file

@ -23,6 +23,7 @@ import fr.acinq.eclair.blockchain.bitcoind.rpc.{BitcoinJsonRPCClient, JsonRPCErr
import fr.acinq.eclair.transactions.Transactions
import grizzled.slf4j.Logging
import org.json4s.JsonAST._
import scodec.bits.ByteVector
import scala.concurrent.{ExecutionContext, Future}
@ -43,7 +44,7 @@ class BitcoinCoreWallet(rpcClient: BitcoinJsonRPCClient)(implicit ec: ExecutionC
})
}
def fundTransaction(tx: Transaction, lockUnspents: Boolean, feeRatePerKw: Long): Future[FundTransactionResponse] = fundTransaction(Transaction.write(tx).toString(), lockUnspents, feeRatePerKw)
def fundTransaction(tx: Transaction, lockUnspents: Boolean, feeRatePerKw: Long): Future[FundTransactionResponse] = fundTransaction(Transaction.write(tx).toHex, lockUnspents, feeRatePerKw)
def signTransaction(hex: String): Future[SignTransactionResponse] =
rpcClient.invoke("signrawtransaction", hex).map(json => {
@ -52,11 +53,11 @@ class BitcoinCoreWallet(rpcClient: BitcoinJsonRPCClient)(implicit ec: ExecutionC
SignTransactionResponse(Transaction.read(hex), complete)
})
def signTransaction(tx: Transaction): Future[SignTransactionResponse] = signTransaction(Transaction.write(tx).toString())
def signTransaction(tx: Transaction): Future[SignTransactionResponse] = signTransaction(Transaction.write(tx).toHex)
def getTransaction(txid: BinaryData): Future[Transaction] = rpcClient.invoke("getrawtransaction", txid.toString()) collect { case JString(hex) => Transaction.read(hex) }
def getTransaction(txid: ByteVector32): Future[Transaction] = rpcClient.invoke("getrawtransaction", txid.toString()) collect { case JString(hex) => Transaction.read(hex) }
def publishTransaction(tx: Transaction)(implicit ec: ExecutionContext): Future[String] = publishTransaction(Transaction.write(tx).toString())
def publishTransaction(tx: Transaction)(implicit ec: ExecutionContext): Future[String] = publishTransaction(Transaction.write(tx).toHex)
def publishTransaction(hex: String)(implicit ec: ExecutionContext): Future[String] = rpcClient.invoke("sendrawtransaction", hex) collect { case JString(txid) => txid }
@ -82,7 +83,7 @@ class BitcoinCoreWallet(rpcClient: BitcoinJsonRPCClient)(implicit ec: ExecutionC
}
}
override def makeFundingTx(pubkeyScript: BinaryData, amount: Satoshi, feeRatePerKw: Long): Future[MakeFundingTxResponse] = {
override def makeFundingTx(pubkeyScript: ByteVector, amount: Satoshi, feeRatePerKw: Long): Future[MakeFundingTxResponse] = {
// partial funding tx
val partialFundingTx = Transaction(
version = 2,
@ -118,7 +119,7 @@ class BitcoinCoreWallet(rpcClient: BitcoinJsonRPCClient)(implicit ec: ExecutionC
Future.successful(false)
} else {
// if the tx wasn't in the blockchain and one of it's input has been spent, it is doublespent
Future.sequence(tx.txIn.map(txIn => isTransactionOutputSpendable(txIn.outPoint.txid.toString, txIn.outPoint.index.toInt))).map(_.exists(_ == false))
Future.sequence(tx.txIn.map(txIn => isTransactionOutputSpendable(txIn.outPoint.txid.toHex, txIn.outPoint.index.toInt))).map(_.exists(_ == false))
}
} yield doublespent // TODO: should we check confirmations of the overriding tx?

View file

@ -26,6 +26,7 @@ import fr.acinq.eclair.blockchain._
import fr.acinq.eclair.blockchain.bitcoind.rpc.ExtendedBitcoinClient
import fr.acinq.eclair.channel.BITCOIN_PARENT_TX_CONFIRMED
import fr.acinq.eclair.transactions.Scripts
import scodec.bits.ByteVector
import scala.collection.SortedMap
import scala.concurrent.duration._
@ -64,7 +65,7 @@ class ZmqWatcher(client: ExtendedBitcoinClient)(implicit ec: ExecutionContext =
case NewBlock(block) =>
// using a Try because in tests we generate fake blocks
log.debug(s"received blockid=${Try(block.blockId).getOrElse(BinaryData.empty)}")
log.debug(s"received blockid=${Try(block.blockId).getOrElse(ByteVector32(ByteVector.empty))}")
nextTick.map(_.cancel()) // this may fail or succeed, worse case scenario we will have two ticks in a row (no big deal)
log.debug(s"scheduling a new task to check on tx confirmations")
// we do this to avoid herd effects in testing when generating a lots of blocks in a row

View file

@ -55,7 +55,7 @@ class ExtendedBitcoinClient(val rpcClient: BitcoinJsonRPCClient) {
}
// with a verbosity of 0, getblock returns the raw serialized block
block <- rpcClient.invoke("getblock", blockhash, 0).collect { case JString(b) => Block.read(b) }
prevblockhash = BinaryData(block.header.hashPreviousBlock.reverse).toString
prevblockhash = block.header.hashPreviousBlock.reverse.toHex
res <- block.tx.find(tx => tx.txIn.exists(i => i.outPoint.txid.toString() == txid && i.outPoint.index == outputIndex)) match {
case None => lookForSpendingTx(Some(prevblockhash), txid, outputIndex)
case Some(tx) => Future.successful(tx)
@ -144,12 +144,12 @@ class ExtendedBitcoinClient(val rpcClient: BitcoinJsonRPCClient) {
val TxCoordinates(blockHeight, txIndex, outputIndex) = coordinates(c.shortChannelId)
for {
blockHash: String <- rpcClient.invoke("getblockhash", blockHeight).map(_.extractOrElse[String]("00" * 32))
blockHash: String <- rpcClient.invoke("getblockhash", blockHeight).map(_.extractOrElse[String](ByteVector32.Zeroes.toHex))
txid: String <- rpcClient.invoke("getblock", blockHash).map {
case json => Try {
val JArray(txs) = json \ "tx"
txs(txIndex).extract[String]
} getOrElse ("00" * 32)
} getOrElse ByteVector32.Zeroes.toHex
}
tx <- getRawTransaction(txid)
unspent <- isTransactionOutputSpendable(txid, outputIndex, includeMempool = true)

View file

@ -24,8 +24,8 @@ import org.zeromq.ZMQ.Event
import org.zeromq.{SocketType, ZContext, ZMQ, ZMsg}
import scala.annotation.tailrec
import scala.concurrent.{ExecutionContext, Promise}
import scala.concurrent.duration._
import scala.concurrent.{ExecutionContext, Promise}
import scala.util.Try
/**

View file

@ -18,17 +18,17 @@ package fr.acinq.eclair.blockchain.electrum
import java.math.BigInteger
import fr.acinq.bitcoin.{BinaryData, Block, BlockHeader, decodeCompact}
import fr.acinq.bitcoin.{Block, BlockHeader, ByteVector32, decodeCompact}
import fr.acinq.eclair.blockchain.electrum.db.HeaderDb
import grizzled.slf4j.Logging
import scala.annotation.tailrec
case class Blockchain(chainHash: BinaryData,
case class Blockchain(chainHash: ByteVector32,
checkpoints: Vector[CheckPoint],
headersMap: Map[BinaryData, Blockchain.BlockIndex],
headersMap: Map[ByteVector32, Blockchain.BlockIndex],
bestchain: Vector[Blockchain.BlockIndex],
orphans: Map[BinaryData, BlockHeader] = Map()) {
orphans: Map[ByteVector32, BlockHeader] = Map()) {
import Blockchain._
@ -110,14 +110,14 @@ object Blockchain extends Logging {
* @param checkpoints list of checkpoints
* @return a blockchain instance
*/
def fromCheckpoints(chainhash: BinaryData, checkpoints: Vector[CheckPoint]): Blockchain = {
def fromCheckpoints(chainhash: ByteVector32, checkpoints: Vector[CheckPoint]): Blockchain = {
Blockchain(chainhash, checkpoints, Map(), Vector())
}
/**
* Used in tests
*/
def fromGenesisBlock(chainhash: BinaryData, genesis: BlockHeader): Blockchain = {
def fromGenesisBlock(chainhash: ByteVector32, genesis: BlockHeader): Blockchain = {
require(chainhash == Block.RegtestGenesisBlock.hash)
// the height of the genesis block is 0
val blockIndex = BlockIndex(genesis, 0, None, decodeCompact(genesis.bits)._1)
@ -131,7 +131,7 @@ object Blockchain extends Logging {
* @param headerDb
* @return
*/
def load(chainHash: BinaryData, headerDb: HeaderDb): Blockchain = {
def load(chainHash: ByteVector32, headerDb: HeaderDb): Blockchain = {
val checkpoints = CheckPoint.load(chainHash)
val checkpoints1 = headerDb.getTip match {
case Some((height, header)) =>

View file

@ -18,7 +18,7 @@ package fr.acinq.eclair.blockchain.electrum
import java.io.InputStream
import fr.acinq.bitcoin.{BinaryData, Block, encodeCompact}
import fr.acinq.bitcoin.{Block, ByteVector32, encodeCompact}
import fr.acinq.eclair.blockchain.electrum.db.HeaderDb
import org.json4s.JsonAST.{JArray, JInt, JString}
import org.json4s.jackson.JsonMethods
@ -28,9 +28,7 @@ import org.json4s.jackson.JsonMethods
* @param hash block hash
* @param target difficulty target for the next block
*/
case class CheckPoint(hash: BinaryData, nextBits: Long) {
require(hash.length == 32)
}
case class CheckPoint(hash: ByteVector32, nextBits: Long)
object CheckPoint {
@ -42,7 +40,7 @@ object CheckPoint {
* we're on the right chain and to validate proof-of-work by checking the difficulty target
* @return an ordered list of checkpoints, with one checkpoint every 2016 blocks
*/
def load(chainHash: BinaryData): Vector[CheckPoint] = chainHash match {
def load(chainHash: ByteVector32): Vector[CheckPoint] = chainHash match {
case Block.LivenetGenesisBlock.hash => load(classOf[CheckPoint].getResourceAsStream("/electrum/checkpoints_mainnet.json"))
case Block.TestnetGenesisBlock.hash => load(classOf[CheckPoint].getResourceAsStream("/electrum/checkpoints_testnet.json"))
case Block.RegtestGenesisBlock.hash => Vector.empty[CheckPoint] // no checkpoints on regtest
@ -51,7 +49,7 @@ object CheckPoint {
def load(stream: InputStream): Vector[CheckPoint] = {
val JArray(values) = JsonMethods.parse(stream)
val checkpoints = values.collect {
case JArray(JString(a) :: JInt(b) :: Nil) => CheckPoint(BinaryData(a).reverse, encodeCompact(b.bigInteger))
case JArray(JString(a) :: JInt(b) :: Nil) => CheckPoint(ByteVector32.fromValidHex(a).reverse, encodeCompact(b.bigInteger))
}
checkpoints.toVector
}
@ -63,7 +61,7 @@ object CheckPoint {
* @param headerDb header db
* @return a series of checkpoints
*/
def load(chainHash: BinaryData, headerDb: HeaderDb): Vector[CheckPoint] = {
def load(chainHash: ByteVector32, headerDb: HeaderDb): Vector[CheckPoint] = {
val checkpoints = CheckPoint.load(chainHash)
val checkpoints1 = headerDb.getTip match {
case Some((height, header)) =>

View file

@ -37,7 +37,7 @@ import io.netty.util.CharsetUtil
import org.json4s.JsonAST._
import org.json4s.jackson.JsonMethods
import org.json4s.{DefaultFormats, JInt, JLong, JString}
import org.spongycastle.util.encoders.Hex
import scodec.bits.ByteVector
import scala.annotation.tailrec
import scala.concurrent.ExecutionContext
@ -167,7 +167,7 @@ class ElectrumClient(serverAddress: InetSocketAddress, ssl: SSL)(implicit val ec
log.debug("sending {} to {}", request, serverAddress)
val json = ("method" -> request.method) ~ ("params" -> request.params.map {
case s: String => new JString(s)
case b: BinaryData => new JString(b.toString())
case b: ByteVector32 => new JString(b.toHex)
case t: Int => new JInt(t)
case t: Long => new JLong(t)
case t: Double => new JDouble(t)
@ -196,7 +196,7 @@ class ElectrumClient(serverAddress: InetSocketAddress, ssl: SSL)(implicit val ec
}
var addressSubscriptions = Map.empty[String, Set[ActorRef]]
var scriptHashSubscriptions = Map.empty[BinaryData, Set[ActorRef]]
var scriptHashSubscriptions = Map.empty[ByteVector32, Set[ActorRef]]
val headerSubscriptions = collection.mutable.HashSet.empty[ActorRef]
val version = ServerVersion(CLIENT_NAME, PROTOCOL_VERSION)
val statusListeners = collection.mutable.HashSet.empty[ActorRef]
@ -346,7 +346,7 @@ object ElectrumClient {
* @param publicKeyScript public key script
* @return the hash of the public key script, as used by ElectrumX's hash-based methods
*/
def computeScriptHash(publicKeyScript: BinaryData): BinaryData = Crypto.sha256(publicKeyScript).reverse
def computeScriptHash(publicKeyScript: ByteVector): ByteVector32 = Crypto.sha256(publicKeyScript).reverse
// @formatter:off
case class AddStatusListener(actor: ActorRef)
@ -362,25 +362,25 @@ object ElectrumClient {
case object PingResponse extends Response
case class GetAddressHistory(address: String) extends Request
case class TransactionHistoryItem(height: Int, tx_hash: BinaryData)
case class TransactionHistoryItem(height: Int, tx_hash: ByteVector32)
case class GetAddressHistoryResponse(address: String, history: Seq[TransactionHistoryItem]) extends Response
case class GetScriptHashHistory(scriptHash: BinaryData) extends Request
case class GetScriptHashHistoryResponse(scriptHash: BinaryData, history: List[TransactionHistoryItem]) extends Response
case class GetScriptHashHistory(scriptHash: ByteVector32) extends Request
case class GetScriptHashHistoryResponse(scriptHash: ByteVector32, history: List[TransactionHistoryItem]) extends Response
case class AddressListUnspent(address: String) extends Request
case class UnspentItem(tx_hash: BinaryData, tx_pos: Int, value: Long, height: Long) {
case class UnspentItem(tx_hash: ByteVector32, tx_pos: Int, value: Long, height: Long) {
lazy val outPoint = OutPoint(tx_hash.reverse, tx_pos)
}
case class AddressListUnspentResponse(address: String, unspents: Seq[UnspentItem]) extends Response
case class ScriptHashListUnspent(scriptHash: BinaryData) extends Request
case class ScriptHashListUnspentResponse(scriptHash: BinaryData, unspents: Seq[UnspentItem]) extends Response
case class ScriptHashListUnspent(scriptHash: ByteVector32) extends Request
case class ScriptHashListUnspentResponse(scriptHash: ByteVector32, unspents: Seq[UnspentItem]) extends Response
case class BroadcastTransaction(tx: Transaction) extends Request
case class BroadcastTransactionResponse(tx: Transaction, error: Option[Error]) extends Response
case class GetTransaction(txid: BinaryData) extends Request
case class GetTransaction(txid: ByteVector32) extends Request
case class GetTransactionResponse(tx: Transaction) extends Response
case class GetHeader(height: Int) extends Request
@ -392,26 +392,26 @@ object ElectrumClient {
case class GetHeaders(start_height: Int, count: Int, cp_height: Int = 0) extends Request
case class GetHeadersResponse(start_height: Int, headers: Seq[BlockHeader], max: Int) extends Response
case class GetMerkle(txid: BinaryData, height: Int) extends Request
case class GetMerkleResponse(txid: BinaryData, merkle: List[BinaryData], block_height: Int, pos: Int) extends Response {
lazy val root: BinaryData = {
case class GetMerkle(txid: ByteVector32, height: Int) extends Request
case class GetMerkleResponse(txid: ByteVector32, merkle: List[ByteVector32], block_height: Int, pos: Int) extends Response {
lazy val root: ByteVector32 = {
@tailrec
def loop(pos: Int, hashes: Seq[BinaryData]): BinaryData = {
def loop(pos: Int, hashes: Seq[ByteVector32]): ByteVector32 = {
if (hashes.length == 1) hashes(0)
else {
val h = if (pos % 2 == 1) Crypto.hash256(hashes(1) ++ hashes(0)) else Crypto.hash256(hashes(0) ++ hashes(1))
loop(pos / 2, h +: hashes.drop(2))
}
}
loop(pos, BinaryData(txid.reverse) +: merkle.map(b => BinaryData(b.reverse)))
loop(pos, txid.reverse +: merkle.map(b => b.reverse))
}
}
case class AddressSubscription(address: String, actor: ActorRef) extends Request
case class AddressSubscriptionResponse(address: String, status: String) extends Response
case class ScriptHashSubscription(scriptHash: BinaryData, actor: ActorRef) extends Request
case class ScriptHashSubscriptionResponse(scriptHash: BinaryData, status: String) extends Response
case class ScriptHashSubscription(scriptHash: ByteVector32, actor: ActorRef) extends Request
case class ScriptHashSubscriptionResponse(scriptHash: ByteVector32, status: String) extends Response
case class HeaderSubscription(actor: ActorRef) extends Request
case class HeaderSubscriptionResponse(height: Int, header: BlockHeader) extends Response
@ -419,11 +419,11 @@ object ElectrumClient {
def apply(t: (Int, BlockHeader)) = new HeaderSubscriptionResponse(t._1, t._2)
}
case class Header(block_height: Long, version: Long, prev_block_hash: BinaryData, merkle_root: BinaryData, timestamp: Long, bits: Long, nonce: Long) {
case class Header(block_height: Long, version: Long, prev_block_hash: ByteVector32, merkle_root: ByteVector32, timestamp: Long, bits: Long, nonce: Long) {
def blockHeader = BlockHeader(version, prev_block_hash.reverse, merkle_root.reverse, timestamp, bits, nonce)
lazy val block_hash: BinaryData = blockHeader.hash
lazy val block_id: BinaryData = block_hash.reverse
lazy val block_hash: ByteVector32 = blockHeader.hash
lazy val block_id: ByteVector32 = block_hash.reverse
}
object Header {
@ -470,8 +470,8 @@ object ElectrumClient {
case ("blockchain.headers.subscribe", header :: Nil) => HeaderSubscriptionResponse(parseBlockHeader(header))
case ("blockchain.address.subscribe", JString(address) :: JNull :: Nil) => AddressSubscriptionResponse(address, "")
case ("blockchain.address.subscribe", JString(address) :: JString(status) :: Nil) => AddressSubscriptionResponse(address, status)
case ("blockchain.scripthash.subscribe", JString(scriptHashHex) :: JNull :: Nil) => ScriptHashSubscriptionResponse(BinaryData(scriptHashHex), "")
case ("blockchain.scripthash.subscribe", JString(scriptHashHex) :: JString(status) :: Nil) => ScriptHashSubscriptionResponse(BinaryData(scriptHashHex), status)
case ("blockchain.scripthash.subscribe", JString(scriptHashHex) :: JNull :: Nil) => ScriptHashSubscriptionResponse(ByteVector32.fromValidHex(scriptHashHex), "")
case ("blockchain.scripthash.subscribe", JString(scriptHashHex) :: JString(status) :: Nil) => ScriptHashSubscriptionResponse(ByteVector32.fromValidHex(scriptHashHex), status)
})
case _ => Right(parseJsonRpcResponse(json))
}
@ -524,13 +524,13 @@ object ElectrumClient {
case ServerVersion(clientName, protocolVersion) => JsonRPCRequest(id = reqId, method = "server.version", params = clientName :: protocolVersion :: Nil)
case Ping => JsonRPCRequest(id = reqId, method = "server.ping", params = Nil)
case GetAddressHistory(address) => JsonRPCRequest(id = reqId, method = "blockchain.address.get_history", params = address :: Nil)
case GetScriptHashHistory(scripthash) => JsonRPCRequest(id = reqId, method = "blockchain.scripthash.get_history", params = scripthash.toString() :: Nil)
case GetScriptHashHistory(scripthash) => JsonRPCRequest(id = reqId, method = "blockchain.scripthash.get_history", params = scripthash.toHex :: Nil)
case AddressListUnspent(address) => JsonRPCRequest(id = reqId, method = "blockchain.address.listunspent", params = address :: Nil)
case ScriptHashListUnspent(scripthash) => JsonRPCRequest(id = reqId, method = "blockchain.scripthash.listunspent", params = scripthash.toString() :: Nil)
case ScriptHashListUnspent(scripthash) => JsonRPCRequest(id = reqId, method = "blockchain.scripthash.listunspent", params = scripthash.toHex :: Nil)
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 = Hex.toHexString(Transaction.write(tx)) :: Nil)
case GetTransaction(txid: BinaryData) => JsonRPCRequest(id = reqId, method = "blockchain.transaction.get", params = txid :: Nil)
case BroadcastTransaction(tx) => JsonRPCRequest(id = reqId, method = "blockchain.transaction.broadcast", params = Transaction.write(tx).toHex :: 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)
@ -556,7 +556,7 @@ object ElectrumClient {
val items = jitems.map(jvalue => {
val JString(tx_hash) = jvalue \ "tx_hash"
val height = intField(jvalue, "height")
TransactionHistoryItem(height, tx_hash)
TransactionHistoryItem(height, ByteVector32.fromValidHex(tx_hash))
})
GetAddressHistoryResponse(address, items)
case GetScriptHashHistory(scripthash) =>
@ -564,7 +564,7 @@ object ElectrumClient {
val items = jitems.map(jvalue => {
val JString(tx_hash) = jvalue \ "tx_hash"
val height = intField(jvalue, "height")
TransactionHistoryItem(height, tx_hash)
TransactionHistoryItem(height, ByteVector32.fromValidHex(tx_hash))
})
GetScriptHashHistoryResponse(scripthash, items)
case AddressListUnspent(address) =>
@ -574,7 +574,7 @@ object ElectrumClient {
val tx_pos = intField(jvalue, "tx_pos")
val height = intField(jvalue, "height")
val value = longField(jvalue, "value")
UnspentItem(tx_hash, tx_pos, value, height)
UnspentItem(ByteVector32.fromValidHex(tx_hash), tx_pos, value, height)
})
AddressListUnspentResponse(address, items)
case ScriptHashListUnspent(scripthash) =>
@ -584,7 +584,7 @@ object ElectrumClient {
val tx_pos = intField(jvalue, "tx_pos")
val height = longField(jvalue, "height")
val value = longField(jvalue, "value")
UnspentItem(tx_hash, tx_pos, value, height)
UnspentItem(ByteVector32.fromValidHex(tx_hash), tx_pos, value, height)
})
ScriptHashListUnspentResponse(scripthash, items)
case GetTransaction(_) =>
@ -600,7 +600,7 @@ object ElectrumClient {
}
case BroadcastTransaction(tx) =>
val JString(txid) = json.result
require(BinaryData(txid) == tx.txid)
require(ByteVector32.fromValidHex(txid) == tx.txid)
BroadcastTransactionResponse(tx, None)
case GetHeader(height) =>
val JString(hex) = json.result
@ -609,12 +609,12 @@ object ElectrumClient {
val count = intField(json.result, "count")
val max = intField(json.result, "max")
val JString(hex) = json.result \ "hex"
val bin = fromHexString(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) =>
val JArray(hashes) = json.result \ "merkle"
val leaves = hashes collect { case JString(value) => BinaryData(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)

View file

@ -18,16 +18,17 @@ package fr.acinq.eclair.blockchain.electrum
import akka.actor.{ActorRef, ActorSystem}
import akka.pattern.ask
import fr.acinq.bitcoin.{BinaryData, Satoshi, Script, Transaction, TxOut}
import fr.acinq.bitcoin.{ByteVector32, Satoshi, Script, Transaction, TxOut}
import fr.acinq.eclair.addressToPublicKeyScript
import fr.acinq.eclair.blockchain.electrum.ElectrumClient.BroadcastTransaction
import fr.acinq.eclair.blockchain.electrum.ElectrumWallet._
import fr.acinq.eclair.blockchain.{EclairWallet, MakeFundingTxResponse}
import grizzled.slf4j.Logging
import scodec.bits.ByteVector
import scala.concurrent.{ExecutionContext, Future}
class ElectrumEclairWallet(val wallet: ActorRef, chainHash: BinaryData)(implicit system: ActorSystem, ec: ExecutionContext, timeout: akka.util.Timeout) extends EclairWallet with Logging {
class ElectrumEclairWallet(val wallet: ActorRef, chainHash: ByteVector32)(implicit system: ActorSystem, ec: ExecutionContext, timeout: akka.util.Timeout) extends EclairWallet with Logging {
override def getBalance = (wallet ? GetBalance).mapTo[GetBalanceResponse].map(balance => balance.confirmed + balance.unconfirmed)
@ -35,7 +36,7 @@ class ElectrumEclairWallet(val wallet: ActorRef, chainHash: BinaryData)(implicit
def getXpub: Future[GetXpubResponse] = (wallet ? GetXpub).mapTo[GetXpubResponse]
override def makeFundingTx(pubkeyScript: BinaryData, amount: Satoshi, feeRatePerKw: Long): Future[MakeFundingTxResponse] = {
override def makeFundingTx(pubkeyScript: ByteVector, amount: Satoshi, feeRatePerKw: Long): Future[MakeFundingTxResponse] = {
val tx = Transaction(version = 2, txIn = Nil, txOut = TxOut(amount, pubkeyScript) :: Nil, lockTime = 0)
(wallet ? CompleteTransaction(tx, feeRatePerKw)).mapTo[CompleteTransactionResponse].map(response => response match {
case CompleteTransactionResponse(tx1, fee1, None) => MakeFundingTxResponse(tx1, 0, fee1)

View file

@ -19,12 +19,13 @@ package fr.acinq.eclair.blockchain.electrum
import akka.actor.{ActorRef, FSM, PoisonPill, Props}
import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey}
import fr.acinq.bitcoin.DeterministicWallet.{ExtendedPrivateKey, derivePrivateKey, hardened}
import fr.acinq.bitcoin.{Base58, Base58Check, BinaryData, Block, BlockHeader, Crypto, DeterministicWallet, OP_PUSHDATA, OutPoint, SIGHASH_ALL, Satoshi, Script, ScriptElt, ScriptWitness, SigVersion, Transaction, TxIn, TxOut}
import fr.acinq.bitcoin.{Base58, Base58Check, Block, ByteVector32, Crypto, DeterministicWallet, OP_PUSHDATA, OutPoint, SIGHASH_ALL, Satoshi, Script, ScriptElt, ScriptWitness, SigVersion, Transaction, TxIn, TxOut}
import fr.acinq.eclair.blockchain.bitcoind.rpc.Error
import fr.acinq.eclair.blockchain.electrum.ElectrumClient._
import fr.acinq.eclair.blockchain.electrum.db.{HeaderDb, WalletDb}
import fr.acinq.eclair.transactions.Transactions
import grizzled.slf4j.Logging
import scodec.bits.ByteVector
import scala.annotation.tailrec
import scala.util.{Failure, Success, Try}
@ -45,7 +46,7 @@ import scala.util.{Failure, Success, Try}
* @param client
* @param params
*/
class ElectrumWallet(seed: BinaryData, client: ActorRef, params: ElectrumWallet.WalletParameters) extends FSM[ElectrumWallet.State, ElectrumWallet.Data] {
class ElectrumWallet(seed: ByteVector, client: ActorRef, params: ElectrumWallet.WalletParameters) extends FSM[ElectrumWallet.State, ElectrumWallet.Data] {
import Blockchain.RETARGETING_PERIOD
import ElectrumWallet._
@ -255,7 +256,7 @@ class ElectrumWallet(seed: BinaryData, client: ActorRef, params: ElectrumWallet.
// let's retrieve the tx history for this key
client ! ElectrumClient.GetScriptHashHistory(scriptHash)
val (newAccountKeys, newChangeKeys) = data.status.get(status) match {
val (newAccountKeys, newChangeKeys) = data.status.get(Try(ByteVector32.fromValidHex(status)).getOrElse(ByteVector32.Zeroes)) match {
case None =>
// first time this script hash is used, need to generate a new key
val newKey = if (isChange) derivePrivateKey(changeMaster, data.changeKeys.last.path.lastChildNumber + 1) else derivePrivateKey(accountMaster, data.accountKeys.last.path.lastChildNumber + 1)
@ -498,9 +499,9 @@ class ElectrumWallet(seed: BinaryData, client: ActorRef, params: ElectrumWallet.
}
object ElectrumWallet {
def props(seed: BinaryData, client: ActorRef, params: WalletParameters): Props = Props(new ElectrumWallet(seed, client, params))
def props(seed: ByteVector, client: ActorRef, params: WalletParameters): Props = Props(new ElectrumWallet(seed, client, params))
case class WalletParameters(chainHash: BinaryData, walletDb: WalletDb, minimumFee: Satoshi = Satoshi(2000), dustLimit: Satoshi = Satoshi(546), swipeRange: Int = 10, allowSpendUnconfirmed: Boolean = true)
case class WalletParameters(chainHash: ByteVector32, walletDb: WalletDb, minimumFee: Satoshi = Satoshi(2000), dustLimit: Satoshi = Satoshi(546), swipeRange: Int = 10, allowSpendUnconfirmed: Boolean = true)
// @formatter:off
sealed trait State
@ -527,7 +528,7 @@ object ElectrumWallet {
case class CompleteTransaction(tx: Transaction, feeRatePerKw: Long) extends Request
case class CompleteTransactionResponse(tx: Transaction, fee: Satoshi, error: Option[Throwable]) extends Response
case class SendAll(publicKeyScript: BinaryData, feeRatePerKw: Long) extends Request
case class SendAll(publicKeyScript: ByteVector, feeRatePerKw: Long) extends Request
case class SendAllResponse(tx: Transaction, fee: Satoshi) extends Response
case class CommitTransaction(tx: Transaction) extends Request
@ -558,7 +559,7 @@ object ElectrumWallet {
* @param feeOpt is set only when we know it (i.e. for outgoing transactions)
*/
case class TransactionReceived(tx: Transaction, depth: Long, received: Satoshi, sent: Satoshi, feeOpt: Option[Satoshi], timestamp: Option[Long]) extends WalletEvent
case class TransactionConfidenceChanged(txid: BinaryData, depth: Long, timestamp: Option[Long]) extends WalletEvent
case class TransactionConfidenceChanged(txid: ByteVector32, depth: Long, timestamp: Option[Long]) extends WalletEvent
case class NewWalletReceiveAddress(address: String) extends WalletEvent
case class WalletReady(confirmedBalance: Satoshi, unconfirmedBalance: Satoshi, height: Long, timestamp: Long) extends WalletEvent
// @formatter:on
@ -568,7 +569,7 @@ object ElectrumWallet {
* @param key public key
* @return the address of the p2sh-of-p2wpkh script for this key
*/
def segwitAddress(key: PublicKey, chainHash: BinaryData): String = {
def segwitAddress(key: PublicKey, chainHash: ByteVector32): String = {
val script = Script.pay2wpkh(key)
val hash = Crypto.hash160(Script.write(script))
chainHash match {
@ -577,9 +578,9 @@ object ElectrumWallet {
}
}
def segwitAddress(key: ExtendedPrivateKey, chainHash: BinaryData): String = segwitAddress(key.publicKey, chainHash)
def segwitAddress(key: ExtendedPrivateKey, chainHash: ByteVector32): String = segwitAddress(key.publicKey, chainHash)
def segwitAddress(key: PrivateKey, chainHash: BinaryData): String = segwitAddress(key.publicKey, chainHash)
def segwitAddress(key: PrivateKey, chainHash: ByteVector32): String = segwitAddress(key.publicKey, chainHash)
/**
*
@ -593,9 +594,9 @@ object ElectrumWallet {
* @param key public key
* @return the hash of the public key script for this key, as used by Electrum's hash-based methods
*/
def computeScriptHashFromPublicKey(key: PublicKey): BinaryData = Crypto.sha256(Script.write(computePublicKeyScript(key))).reverse
def computeScriptHashFromPublicKey(key: PublicKey): ByteVector32 = Crypto.sha256(Script.write(computePublicKeyScript(key))).reverse
def accountPath(chainHash: BinaryData): List[Long] = chainHash match {
def accountPath(chainHash: ByteVector32): List[Long] = chainHash match {
case Block.RegtestGenesisBlock.hash | Block.TestnetGenesisBlock.hash => hardened(49) :: hardened(1) :: hardened(0) :: Nil
case Block.LivenetGenesisBlock.hash => hardened(49) :: hardened(0) :: hardened(0) :: Nil
}
@ -606,7 +607,7 @@ object ElectrumWallet {
* @param master master key
* @return the BIP49 account key for this master key: m/49'/1'/0'/0 on testnet/regtest, m/49'/0'/0'/0 on mainnet
*/
def accountKey(master: ExtendedPrivateKey, chainHash: BinaryData) = DeterministicWallet.derivePrivateKey(master, accountPath(chainHash) ::: 0L :: Nil)
def accountKey(master: ExtendedPrivateKey, chainHash: ByteVector32) = DeterministicWallet.derivePrivateKey(master, accountPath(chainHash) ::: 0L :: Nil)
/**
@ -616,7 +617,7 @@ object ElectrumWallet {
* @param chainHash chain hash
* @return a (xpub, path) tuple where xpub is the encoded account public key, and path is the derivation path for the account key
*/
def computeXpub(master: ExtendedPrivateKey, chainHash: BinaryData): (String, String) = {
def computeXpub(master: ExtendedPrivateKey, chainHash: ByteVector32): (String, String) = {
val xpub = DeterministicWallet.publicKey(DeterministicWallet.derivePrivateKey(master, accountPath(chainHash)))
val prefix = chainHash match {
case Block.LivenetGenesisBlock.hash => DeterministicWallet.ypub
@ -631,7 +632,7 @@ object ElectrumWallet {
* @param master master key
* @return the BIP49 change key for this master key: m/49'/1'/0'/1 on testnet/regtest, m/49'/0'/0'/1 on mainnet
*/
def changeKey(master: ExtendedPrivateKey, chainHash: BinaryData) = DeterministicWallet.derivePrivateKey(master, accountPath(chainHash) ::: 1L :: Nil)
def changeKey(master: ExtendedPrivateKey, chainHash: ByteVector32) = DeterministicWallet.derivePrivateKey(master, accountPath(chainHash) ::: 1L :: Nil)
def totalAmount(utxos: Seq[Utxo]): Satoshi = Satoshi(utxos.map(_.item.value).sum)
@ -693,14 +694,14 @@ object ElectrumWallet {
case class Data(blockchain: Blockchain,
accountKeys: Vector[ExtendedPrivateKey],
changeKeys: Vector[ExtendedPrivateKey],
status: Map[BinaryData, String],
transactions: Map[BinaryData, Transaction],
heights: Map[BinaryData, Int],
history: Map[BinaryData, List[ElectrumClient.TransactionHistoryItem]],
proofs: Map[BinaryData, GetMerkleResponse],
status: Map[ByteVector32, String],
transactions: Map[ByteVector32, Transaction],
heights: Map[ByteVector32, Int],
history: Map[ByteVector32, List[ElectrumClient.TransactionHistoryItem]],
proofs: Map[ByteVector32, GetMerkleResponse],
locks: Set[Transaction],
pendingHistoryRequests: Set[BinaryData],
pendingTransactionRequests: Set[BinaryData],
pendingHistoryRequests: Set[ByteVector32],
pendingTransactionRequests: Set[ByteVector32],
pendingHeadersRequests: Set[GetHeaders],
pendingTransactions: List[Transaction],
lastReadyMessage: Option[WalletReady]) extends Logging {
@ -770,13 +771,13 @@ object ElectrumWallet {
* @param scriptHash
* @return true if txIn spends from an address that matches scriptHash
*/
def isSpend(txIn: TxIn, scriptHash: BinaryData): Boolean = extractPubKeySpentFrom(txIn).exists(pub => computeScriptHashFromPublicKey(pub) == scriptHash)
def isSpend(txIn: TxIn, scriptHash: ByteVector32): Boolean = extractPubKeySpentFrom(txIn).exists(pub => computeScriptHashFromPublicKey(pub) == scriptHash)
def isReceive(txOut: TxOut, scriptHash: BinaryData): Boolean = publicScriptMap.get(txOut.publicKeyScript).exists(key => computeScriptHashFromPublicKey(key.publicKey) == scriptHash)
def isReceive(txOut: TxOut, scriptHash: ByteVector32): Boolean = publicScriptMap.get(txOut.publicKeyScript).exists(key => computeScriptHashFromPublicKey(key.publicKey) == scriptHash)
def isMine(txOut: TxOut): Boolean = publicScriptMap.contains(txOut.publicKeyScript)
def computeTransactionDepth(txid: BinaryData): Long = heights.get(txid).map(height => if (height > 0) computeDepth(blockchain.tip.height, height) else 0).getOrElse(0)
def computeTransactionDepth(txid: ByteVector32): Long = heights.get(txid).map(height => if (height > 0) computeDepth(blockchain.tip.height, height) else 0).getOrElse(0)
/**
*
@ -784,9 +785,9 @@ object ElectrumWallet {
* @param headerDb header db
* @return the timestamp of the block this tx was included in
*/
def computeTimestamp(txid: BinaryData, headerDb: HeaderDb): Option[Long] = {
def computeTimestamp(txid: ByteVector32, headerDb: HeaderDb): Option[Long] = {
for {
height <- heights.get(txid).map(_.toInt)
height <- heights.get(txid)
header <- blockchain.getHeader(height).orElse(headerDb.getHeader(height))
} yield header.time
}
@ -796,7 +797,7 @@ object ElectrumWallet {
* @param scriptHash script hash
* @return the list of UTXOs for this script hash (including unconfirmed UTXOs)
*/
def getUtxos(scriptHash: BinaryData) = {
def getUtxos(scriptHash: ByteVector32) = {
history.get(scriptHash) match {
case None => Seq()
case Some(items) if items.isEmpty => Seq()
@ -828,7 +829,7 @@ object ElectrumWallet {
* @return the (confirmed, unconfirmed) balance for this script hash. This balance may not
* be up-to-date if we have not received all data we've asked for yet.
*/
def balance(scriptHash: BinaryData): (Satoshi, Satoshi) = {
def balance(scriptHash: ByteVector32): (Satoshi, Satoshi) = {
history.get(scriptHash) match {
case None => (Satoshi(0), Satoshi(0))
@ -910,7 +911,7 @@ object ElectrumWallet {
def addUtxosWithDummySig(tx: Transaction, utxos: Seq[Utxo]): Transaction =
tx.copy(txIn = utxos.map { case utxo =>
// we use dummy signature here, because the result is only used to estimate fees
val sig = BinaryData("01" * 71)
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)
TxIn(utxo.outPoint, signatureScript = sigScript, sequence = TxIn.SEQUENCE_FINAL, witness = witness)
@ -1042,7 +1043,7 @@ object ElectrumWallet {
* @return a (tx, fee) tuple, tx is a signed transaction that spends all our balance and
* fee is the associated bitcoin network fee
*/
def spendAll(publicKeyScript: BinaryData, feeRatePerKw: Long): (Transaction, Satoshi) = {
def spendAll(publicKeyScript: ByteVector, feeRatePerKw: Long): (Transaction, Satoshi) = {
// use confirmed and unconfirmed balance
val amount = balance._1 + balance._2
val tx = Transaction(version = 2, txIn = Nil, txOut = TxOut(amount, publicKeyScript) :: Nil, lockTime = 0)
@ -1066,11 +1067,11 @@ object ElectrumWallet {
case class PersistentData(accountKeysCount: Int,
changeKeysCount: Int,
status: Map[BinaryData, String],
transactions: Map[BinaryData, Transaction],
heights: Map[BinaryData, Int],
history: Map[BinaryData, List[ElectrumClient.TransactionHistoryItem]],
proofs: Map[BinaryData, GetMerkleResponse],
status: Map[ByteVector32, String],
transactions: Map[ByteVector32, Transaction],
heights: Map[ByteVector32, Int],
history: Map[ByteVector32, List[ElectrumClient.TransactionHistoryItem]],
proofs: Map[ByteVector32, GetMerkleResponse],
pendingTransactions: List[Transaction],
locks: Set[Transaction])

View file

@ -20,7 +20,7 @@ import java.net.InetSocketAddress
import akka.actor.{Actor, ActorLogging, ActorRef, ActorSystem, Props, Stash, Terminated}
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.bitcoin.{BinaryData, BlockHeader, Satoshi, Script, Transaction, TxIn, TxOut}
import fr.acinq.bitcoin.{BlockHeader, ByteVector32, Satoshi, Script, Transaction, TxIn, TxOut}
import fr.acinq.eclair.blockchain._
import fr.acinq.eclair.blockchain.electrum.ElectrumClient._
import fr.acinq.eclair.channel.{BITCOIN_FUNDING_DEPTHOK, BITCOIN_FUNDING_SPENT, BITCOIN_PARENT_TX_CONFIRMED}
@ -62,7 +62,7 @@ class ElectrumWatcher(client: ActorRef) extends Actor with Stash with ActorLoggi
case publish: PublishAsap => context become disconnected(watches, publishQueue :+ publish, block2tx)
}
def running(height: Int, tip: BlockHeader, watches: Set[Watch], scriptHashStatus: Map[BinaryData, String], block2tx: SortedMap[Long, Seq[Transaction]], sent: Seq[Transaction]): Receive = {
def running(height: Int, tip: BlockHeader, watches: Set[Watch], scriptHashStatus: Map[ByteVector32, String], block2tx: SortedMap[Long, Seq[Transaction]], sent: Seq[Transaction]): Receive = {
case ElectrumClient.HeaderSubscriptionResponse(newheight, newtip) if tip == newtip => ()
case ElectrumClient.HeaderSubscriptionResponse(newheight, newtip) =>

View file

@ -16,8 +16,7 @@
package fr.acinq.eclair.blockchain.electrum.db
import fr.acinq.bitcoin.{BinaryData, BlockHeader, Transaction}
import fr.acinq.eclair.blockchain.electrum.ElectrumClient.GetMerkleResponse
import fr.acinq.bitcoin.{BlockHeader, ByteVector32}
import fr.acinq.eclair.blockchain.electrum.ElectrumWallet.PersistentData
trait HeaderDb {
@ -28,7 +27,7 @@ trait HeaderDb {
def getHeader(height: Int): Option[BlockHeader]
// used only in unit tests
def getHeader(blockHash: BinaryData): Option[(Int, BlockHeader)]
def getHeader(blockHash: ByteVector32): Option[(Int, BlockHeader)]
def getHeaders(startHeight: Int, maxCount: Option[Int]): Seq[BlockHeader]

View file

@ -18,11 +18,11 @@ package fr.acinq.eclair.blockchain.electrum.db.sqlite
import java.sql.Connection
import fr.acinq.bitcoin.{BinaryData, BlockHeader, Transaction}
import fr.acinq.eclair.blockchain.electrum.{ElectrumClient, ElectrumWallet}
import fr.acinq.bitcoin.{BlockHeader, ByteVector32, Transaction}
import fr.acinq.eclair.blockchain.electrum.ElectrumClient.{GetMerkleResponse, TransactionHistoryItem}
import fr.acinq.eclair.blockchain.electrum.ElectrumWallet.PersistentData
import fr.acinq.eclair.blockchain.electrum.db.WalletDb
import fr.acinq.eclair.blockchain.electrum.{ElectrumClient, ElectrumWallet}
import fr.acinq.eclair.db.sqlite.SqliteUtils
import scala.collection.immutable.Queue
@ -39,8 +39,8 @@ class SqliteWalletDb(sqlite: Connection) extends WalletDb {
override def addHeader(height: Int, header: BlockHeader): Unit = {
using(sqlite.prepareStatement("INSERT OR IGNORE INTO headers VALUES (?, ?, ?)")) { statement =>
statement.setInt(1, height)
statement.setBytes(2, header.hash)
statement.setBytes(3, BlockHeader.write(header))
statement.setBytes(2, header.hash.toArray)
statement.setBytes(3, BlockHeader.write(header).toArray)
statement.executeUpdate()
}
}
@ -50,8 +50,8 @@ class SqliteWalletDb(sqlite: Connection) extends WalletDb {
var height = startHeight
headers.foreach(header => {
statement.setInt(1, height)
statement.setBytes(2, header.hash)
statement.setBytes(3, BlockHeader.write(header))
statement.setBytes(2, header.hash.toArray)
statement.setBytes(3, BlockHeader.write(header).toArray)
statement.addBatch()
height = height + 1
})
@ -71,9 +71,9 @@ class SqliteWalletDb(sqlite: Connection) extends WalletDb {
}
}
override def getHeader(blockHash: BinaryData): Option[(Int, BlockHeader)] = {
override def getHeader(blockHash: ByteVector32): Option[(Int, BlockHeader)] = {
using(sqlite.prepareStatement("SELECT height, header FROM headers WHERE block_hash = ?")) { statement =>
statement.setBytes(1, blockHash)
statement.setBytes(1, blockHash.toArray)
val rs = statement.executeQuery()
if (rs.next()) {
Some((rs.getInt("height"), BlockHeader.read(rs.getBytes("header"))))
@ -125,7 +125,7 @@ class SqliteWalletDb(sqlite: Connection) extends WalletDb {
using(sqlite.prepareStatement("SELECT data FROM wallet")) { statement =>
val rs = statement.executeQuery()
if (rs.next()) {
Option(rs.getBytes(1)).map(bin => SqliteWalletDb.deserializePersistentData(BinaryData(bin)))
Option(rs.getBytes(1)).map(bin => SqliteWalletDb.deserializePersistentData(bin))
} else {
None
}
@ -135,62 +135,62 @@ class SqliteWalletDb(sqlite: Connection) extends WalletDb {
object SqliteWalletDb {
import fr.acinq.eclair.wire.LightningMessageCodecs._
import fr.acinq.eclair.wire.ChannelCodecs._
import fr.acinq.eclair.wire.LightningMessageCodecs._
import scodec.Codec
import scodec.bits.BitVector
import scodec.codecs._
val proofCodec: Codec[GetMerkleResponse] = (
("txid" | binarydata(32)) ::
("merkle" | listOfN(uint16, binarydata(32))) ::
("txid" | bytes32) ::
("merkle" | listOfN(uint16, bytes32)) ::
("block_height" | uint24) ::
("pos" | uint24)).as[GetMerkleResponse]
def serializeMerkleProof(proof: GetMerkleResponse): BinaryData = proofCodec.encode(proof).require.toByteArray
def serializeMerkleProof(proof: GetMerkleResponse): Array[Byte] = proofCodec.encode(proof).require.toByteArray
def deserializeMerkleProof(bin: BinaryData): GetMerkleResponse = proofCodec.decode(BitVector(bin.toArray)).require.value
def deserializeMerkleProof(bin: Array[Byte]): GetMerkleResponse = proofCodec.decode(BitVector(bin)).require.value
import fr.acinq.eclair.wire.LightningMessageCodecs._
val statusListCodec: Codec[List[(BinaryData, String)]] = listOfN(uint16, binarydata(32) ~ cstring)
val statusListCodec: Codec[List[(ByteVector32, String)]] = listOfN(uint16, bytes32 ~ cstring)
val statusCodec: Codec[Map[BinaryData, String]] = Codec[Map[BinaryData, String]](
(map: Map[BinaryData, String]) => statusListCodec.encode(map.toList),
val statusCodec: Codec[Map[ByteVector32, String]] = Codec[Map[ByteVector32, String]](
(map: Map[ByteVector32, String]) => statusListCodec.encode(map.toList),
(wire: BitVector) => statusListCodec.decode(wire).map(_.map(_.toMap))
)
val heightsListCodec: Codec[List[(BinaryData, Int)]] = listOfN(uint16, binarydata(32) ~ int32)
val heightsListCodec: Codec[List[(ByteVector32, Int)]] = listOfN(uint16, bytes32 ~ int32)
val heightsCodec: Codec[Map[BinaryData, Int]] = Codec[Map[BinaryData, Int]](
(map: Map[BinaryData, Int]) => heightsListCodec.encode(map.toList),
val heightsCodec: Codec[Map[ByteVector32, Int]] = Codec[Map[ByteVector32, Int]](
(map: Map[ByteVector32, Int]) => heightsListCodec.encode(map.toList),
(wire: BitVector) => heightsListCodec.decode(wire).map(_.map(_.toMap))
)
val transactionListCodec: Codec[List[(BinaryData, Transaction)]] = listOfN(uint16, binarydata(32) ~ txCodec)
val transactionListCodec: Codec[List[(ByteVector32, Transaction)]] = listOfN(uint16, bytes32 ~ txCodec)
val transactionsCodec: Codec[Map[BinaryData, Transaction]] = Codec[Map[BinaryData, Transaction]](
(map: Map[BinaryData, Transaction]) => transactionListCodec.encode(map.toList),
val transactionsCodec: Codec[Map[ByteVector32, Transaction]] = Codec[Map[ByteVector32, Transaction]](
(map: Map[ByteVector32, Transaction]) => transactionListCodec.encode(map.toList),
(wire: BitVector) => transactionListCodec.decode(wire).map(_.map(_.toMap))
)
val transactionHistoryItemCodec: Codec[ElectrumClient.TransactionHistoryItem] = (
("height" | int32) :: ("tx_hash" | binarydata(size = 32))).as[ElectrumClient.TransactionHistoryItem]
("height" | int32) :: ("tx_hash" | bytes32)).as[ElectrumClient.TransactionHistoryItem]
val seqOfTransactionHistoryItemCodec: Codec[List[TransactionHistoryItem]] = listOfN[TransactionHistoryItem](uint16, transactionHistoryItemCodec)
val historyListCodec: Codec[List[(BinaryData, List[ElectrumClient.TransactionHistoryItem])]] =
listOfN[(BinaryData, List[ElectrumClient.TransactionHistoryItem])](uint16, binarydata(32) ~ seqOfTransactionHistoryItemCodec)
val historyListCodec: Codec[List[(ByteVector32, List[ElectrumClient.TransactionHistoryItem])]] =
listOfN[(ByteVector32, List[ElectrumClient.TransactionHistoryItem])](uint16, bytes32 ~ seqOfTransactionHistoryItemCodec)
val historyCodec: Codec[Map[BinaryData, List[ElectrumClient.TransactionHistoryItem]]] = Codec[Map[BinaryData, List[ElectrumClient.TransactionHistoryItem]]](
(map: Map[BinaryData, List[ElectrumClient.TransactionHistoryItem]]) => historyListCodec.encode(map.toList),
val historyCodec: Codec[Map[ByteVector32, List[ElectrumClient.TransactionHistoryItem]]] = Codec[Map[ByteVector32, List[ElectrumClient.TransactionHistoryItem]]](
(map: Map[ByteVector32, List[ElectrumClient.TransactionHistoryItem]]) => historyListCodec.encode(map.toList),
(wire: BitVector) => historyListCodec.decode(wire).map(_.map(_.toMap))
)
val proofsListCodec: Codec[List[(BinaryData, GetMerkleResponse)]] = listOfN(uint16, binarydata(32) ~ proofCodec)
val proofsListCodec: Codec[List[(ByteVector32, GetMerkleResponse)]] = listOfN(uint16, bytes32 ~ proofCodec)
val proofsCodec: Codec[Map[BinaryData, GetMerkleResponse]] = Codec[Map[BinaryData, GetMerkleResponse]](
(map: Map[BinaryData, GetMerkleResponse]) => proofsListCodec.encode(map.toList),
val proofsCodec: Codec[Map[ByteVector32, GetMerkleResponse]] = Codec[Map[ByteVector32, GetMerkleResponse]](
(map: Map[ByteVector32, GetMerkleResponse]) => proofsListCodec.encode(map.toList),
(wire: BitVector) => proofsListCodec.decode(wire).map(_.map(_.toMap))
)
@ -213,7 +213,7 @@ object SqliteWalletDb {
("pendingTransactions" | listOfN(uint16, txCodec)) ::
("locks" | setCodec(txCodec))).as[PersistentData]
def serialize(data: PersistentData): BinaryData = persistentDataCodec.encode(data).require.toByteArray
def serialize(data: PersistentData): Array[Byte] = persistentDataCodec.encode(data).require.toByteArray
def deserializePersistentData(bin: BinaryData): PersistentData = persistentDataCodec.decode(BitVector(bin.toArray)).require.value
def deserializePersistentData(bin: Array[Byte]): PersistentData = persistentDataCodec.decode(BitVector(bin)).require.value
}

View file

@ -18,14 +18,14 @@ package fr.acinq.eclair.blockchain.fee
import com.softwaremill.sttp._
import com.softwaremill.sttp.json4s._
import fr.acinq.bitcoin.{BinaryData, Block}
import fr.acinq.bitcoin.{Block, ByteVector32}
import org.json4s.DefaultFormats
import org.json4s.JsonAST.{JInt, JValue}
import org.json4s.jackson.Serialization
import scala.concurrent.{ExecutionContext, Future}
class BitgoFeeProvider(chainHash: BinaryData)(implicit http: SttpBackend[Future, Nothing], ec: ExecutionContext) extends FeeProvider {
class BitgoFeeProvider(chainHash: ByteVector32)(implicit http: SttpBackend[Future, Nothing], ec: ExecutionContext) extends FeeProvider {
import BitgoFeeProvider._

View file

@ -1287,7 +1287,7 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
// we give them one of the published txes as a hint
// note spendingTx != Nil (that's a requirement of DATA_CLOSING)
val exc = FundingTxSpent(d.channelId, d.spendingTxes.head)
val error = Error(d.channelId, exc.getMessage.getBytes)
val error = Error(d.channelId, exc.getMessage)
stay sending error
case Event(c: CMD_CLOSE, d: DATA_CLOSING) => handleCommandError(ClosingAlreadyInProgress(d.channelId), c)
@ -1323,14 +1323,14 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
// they already proved that we have an outdated commitment
// there isn't much to do except asking them again to publish their current commitment by sending an error
val exc = PleasePublishYourCommitment(d.channelId)
val error = Error(d.channelId, exc.getMessage.getBytes)
val error = Error(d.channelId, exc.getMessage)
val d1 = Helpers.updateFeatures(d, localInit, remoteInit)
goto(WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT) using d1 sending error
case Event(INPUT_RECONNECTED(r, localInit, remoteInit), d: HasCommitments) =>
forwarder ! r
val yourLastPerCommitmentSecret = d.commitments.remotePerCommitmentSecrets.lastIndex.flatMap(d.commitments.remotePerCommitmentSecrets.getHash).getOrElse(Sphinx zeroes 32)
val yourLastPerCommitmentSecret = d.commitments.remotePerCommitmentSecrets.lastIndex.flatMap(d.commitments.remotePerCommitmentSecrets.getHash).getOrElse(ByteVector32.Zeroes)
val myCurrentPerCommitmentPoint = keyManager.commitmentPoint(d.commitments.localParams.channelKeyPath, d.commitments.localCommit.index)
val channelReestablish = ChannelReestablish(
@ -1399,7 +1399,7 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
// their data checks out, we indeed seem to be using an old revoked commitment, and must absolutely *NOT* publish it, because that would be a cheating attempt and they
// would punish us by taking all the funds in the channel
val exc = PleasePublishYourCommitment(d.channelId)
val error = Error(d.channelId, exc.getMessage.getBytes)
val error = Error(d.channelId, exc.getMessage)
goto(WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT) using store(DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT(d.commitments, channelReestablish)) sending error
} else {
// they lied! the last per_commitment_secret they claimed to have received from us is invalid
@ -1412,7 +1412,7 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
// maybe they will publish their commitment, in that case we need to remember their commitment point in order to be able to claim our outputs
// not that if they don't comply, we could publish our own commitment (it is not stale, otherwise we would be in the case above)
val exc = PleasePublishYourCommitment(d.channelId)
val error = Error(d.channelId, exc.getMessage.getBytes)
val error = Error(d.channelId, exc.getMessage)
goto(WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT) using store(DATA_WAIT_FOR_REMOTE_PUBLISH_FUTURE_COMMITMENT(d.commitments, channelReestablish)) sending error
case _ =>
// normal case, our data is up-to-date
@ -1636,7 +1636,7 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
def replyToUser(message: Either[Channel.ChannelError, String]) = {
val m = message match {
case Left(LocalError(t)) => Status.Failure(t)
case Left(RemoteError(e)) => Status.Failure(new RuntimeException(s"peer sent error: '${if (isAsciiPrintable(e.data)) new String(e.data, StandardCharsets.US_ASCII) else e.data.toString()}'"))
case Left(RemoteError(e)) => Status.Failure(new RuntimeException(s"peer sent error: '${if (isAsciiPrintable(e.data)) new String(e.data.toArray, StandardCharsets.US_ASCII) else e.data.toString()}'"))
case Right(s) => s
}
origin_opt.map(_ ! m)
@ -1658,7 +1658,7 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
def handleFundingPublishFailed(d: DATA_WAIT_FOR_FUNDING_CONFIRMED) = {
log.error(s"failed to publish funding tx")
val exc = ChannelFundingError(d.channelId)
val error = Error(d.channelId, exc.getMessage.getBytes)
val error = Error(d.channelId, exc.getMessage)
// NB: we don't use the handleLocalError handler because it would result in the commit tx being published, which we don't want:
// implementation *guarantees* that in case of BITCOIN_FUNDING_PUBLISH_FAILED, the funding tx hasn't and will never be published, so we can close the channel right away
context.system.eventStream.publish(ChannelFailed(self, Helpers.getChannelId(stateData), remoteNodeId, stateData, LocalError(exc)))
@ -1668,7 +1668,7 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
def handleFundingTimeout(d: DATA_WAIT_FOR_FUNDING_CONFIRMED) = {
log.warning(s"funding tx hasn't been confirmed in time, cancelling channel delay=$FUNDING_TIMEOUT_FUNDEE")
val exc = FundingTxTimedout(d.channelId)
val error = Error(d.channelId, exc.getMessage.getBytes)
val error = Error(d.channelId, exc.getMessage)
context.system.eventStream.publish(ChannelFailed(self, Helpers.getChannelId(stateData), remoteNodeId, stateData, LocalError(exc)))
goto(ERR_FUNDING_TIMEOUT) sending error
}
@ -1692,7 +1692,7 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
case _: ChannelException => ()
case _ => log.error(cause, s"msg=${msg.getOrElse("n/a")} stateData=$stateData ")
}
val error = Error(Helpers.getChannelId(d), cause.getMessage.getBytes)
val error = Error(Helpers.getChannelId(d), cause.getMessage)
context.system.eventStream.publish(ChannelFailed(self, Helpers.getChannelId(stateData), remoteNodeId, stateData, LocalError(cause)))
d match {
@ -1708,7 +1708,7 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
def handleRemoteError(e: Error, d: Data) = {
// see BOLT 1: only print out data verbatim if is composed of printable ASCII characters
log.error(s"peer sent error: ascii='${if (isAsciiPrintable(e.data)) new String(e.data, StandardCharsets.US_ASCII) else "n/a"}' bin=${e.data}")
log.error(s"peer sent error: ascii='${if (isAsciiPrintable(e.data)) new String(e.data.toArray, StandardCharsets.US_ASCII) else "n/a"}' bin=${e.data}")
context.system.eventStream.publish(ChannelFailed(self, Helpers.getChannelId(stateData), remoteNodeId, stateData, RemoteError(e)))
d match {
@ -1771,7 +1771,7 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
* @param txes
* @param irrevocablySpent
*/
def publishIfNeeded(txes: Iterable[Transaction], irrevocablySpent: Map[OutPoint, BinaryData]) = {
def publishIfNeeded(txes: Iterable[Transaction], irrevocablySpent: Map[OutPoint, ByteVector32]) = {
val (skip, process) = txes.partition(Closing.inputsAlreadySpent(_, irrevocablySpent))
process.foreach { tx =>
log.info(s"publishing txid=${tx.txid}")
@ -1786,7 +1786,7 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
* @param txes
* @param irrevocablySpent
*/
def watchConfirmedIfNeeded(txes: Iterable[Transaction], irrevocablySpent: Map[OutPoint, BinaryData]) = {
def watchConfirmedIfNeeded(txes: Iterable[Transaction], irrevocablySpent: Map[OutPoint, ByteVector32]) = {
val (skip, process) = txes.partition(Closing.inputsAlreadySpent(_, irrevocablySpent))
process.foreach(tx => blockchain ! WatchConfirmed(self, tx, nodeParams.minDepthBlocks, BITCOIN_TX_CONFIRMED(tx)))
skip.foreach(tx => log.info(s"no need to watch txid=${tx.txid}, it has already been confirmed"))
@ -1799,7 +1799,7 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
* @param txes
* @param irrevocablySpent
*/
def watchSpentIfNeeded(parentTx: Transaction, txes: Iterable[Transaction], irrevocablySpent: Map[OutPoint, BinaryData]) = {
def watchSpentIfNeeded(parentTx: Transaction, txes: Iterable[Transaction], irrevocablySpent: Map[OutPoint, ByteVector32]) = {
val (skip, process) = txes.partition(Closing.inputsAlreadySpent(_, irrevocablySpent))
process.foreach(tx => blockchain ! WatchSpent(self, parentTx, tx.txIn.head.outPoint.index.toInt, BITCOIN_OUTPUT_SPENT))
skip.foreach(tx => log.info(s"no need to watch txid=${tx.txid}, it has already been confirmed"))
@ -1891,7 +1891,7 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
case Some(revokedCommitPublished) =>
log.warning(s"txid=${tx.txid} was a revoked commitment, publishing the penalty tx")
val exc = FundingTxSpent(d.channelId, tx)
val error = Error(d.channelId, exc.getMessage.getBytes)
val error = Error(d.channelId, exc.getMessage)
doPublish(revokedCommitPublished)
@ -1929,7 +1929,7 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
// this is never supposed to happen !!
log.error(s"our funding tx ${d.commitments.commitInput.outPoint.txid} was spent by txid=${tx.txid} !!")
val exc = FundingTxSpent(d.channelId, tx)
val error = Error(d.channelId, exc.getMessage.getBytes)
val error = Error(d.channelId, exc.getMessage)
// let's try to spend our current local tx
val commitTx = d.commitments.localCommit.publishableTxs.commitTx.tx
@ -2038,7 +2038,7 @@ class Channel(val nodeParams: NodeParams, wallet: EclairWallet, remoteNodeId: Pu
case Some(u) => Relayed(u.channelId, u.id, u.amountMsat, c.amountMsat)
}
def feePaid(fee: Satoshi, tx: Transaction, desc: String, channelId: BinaryData) = {
def feePaid(fee: Satoshi, tx: Transaction, desc: String, channelId: ByteVector32) = {
log.info(s"paid feeSatoshi=${fee.amount} for txid=${tx.txid} desc=$desc")
context.system.eventStream.publish(NetworkFeePaid(self, remoteNodeId, channelId, tx, fee, desc))
}

View file

@ -18,7 +18,7 @@ package fr.acinq.eclair.channel
import akka.actor.ActorRef
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.bitcoin.{BinaryData, Satoshi, Transaction}
import fr.acinq.bitcoin.{ByteVector32, Satoshi, Transaction}
import fr.acinq.eclair.ShortChannelId
import fr.acinq.eclair.channel.Channel.ChannelError
import fr.acinq.eclair.wire.{ChannelAnnouncement, ChannelUpdate}
@ -29,17 +29,17 @@ import fr.acinq.eclair.wire.{ChannelAnnouncement, ChannelUpdate}
trait ChannelEvent
case class ChannelCreated(channel: ActorRef, peer: ActorRef, remoteNodeId: PublicKey, isFunder: Boolean, temporaryChannelId: BinaryData) extends ChannelEvent
case class ChannelCreated(channel: ActorRef, peer: ActorRef, remoteNodeId: PublicKey, isFunder: Boolean, temporaryChannelId: ByteVector32) extends ChannelEvent
case class ChannelRestored(channel: ActorRef, peer: ActorRef, remoteNodeId: PublicKey, isFunder: Boolean, channelId: BinaryData, currentData: HasCommitments) extends ChannelEvent
case class ChannelRestored(channel: ActorRef, peer: ActorRef, remoteNodeId: PublicKey, isFunder: Boolean, channelId: ByteVector32, currentData: HasCommitments) extends ChannelEvent
case class ChannelIdAssigned(channel: ActorRef, remoteNodeId: PublicKey, temporaryChannelId: BinaryData, channelId: BinaryData) extends ChannelEvent
case class ChannelIdAssigned(channel: ActorRef, remoteNodeId: PublicKey, temporaryChannelId: ByteVector32, channelId: ByteVector32) extends ChannelEvent
case class ShortChannelIdAssigned(channel: ActorRef, channelId: BinaryData, shortChannelId: ShortChannelId) extends ChannelEvent
case class ShortChannelIdAssigned(channel: ActorRef, channelId: ByteVector32, shortChannelId: ShortChannelId) extends ChannelEvent
case class LocalChannelUpdate(channel: ActorRef, channelId: BinaryData, shortChannelId: ShortChannelId, remoteNodeId: PublicKey, channelAnnouncement_opt: Option[ChannelAnnouncement], channelUpdate: ChannelUpdate, commitments: Commitments) extends ChannelEvent
case class LocalChannelUpdate(channel: ActorRef, channelId: ByteVector32, shortChannelId: ShortChannelId, remoteNodeId: PublicKey, channelAnnouncement_opt: Option[ChannelAnnouncement], channelUpdate: ChannelUpdate, commitments: Commitments) extends ChannelEvent
case class LocalChannelDown(channel: ActorRef, channelId: BinaryData, shortChannelId: ShortChannelId, remoteNodeId: PublicKey) extends ChannelEvent
case class LocalChannelDown(channel: ActorRef, channelId: ByteVector32, shortChannelId: ShortChannelId, remoteNodeId: PublicKey) extends ChannelEvent
case class ChannelStateChanged(channel: ActorRef, peer: ActorRef, remoteNodeId: PublicKey, previousState: State, currentState: State, currentData: Data) extends ChannelEvent
@ -47,15 +47,15 @@ case class ChannelSignatureSent(channel: ActorRef, commitments: Commitments) ext
case class ChannelSignatureReceived(channel: ActorRef, commitments: Commitments) extends ChannelEvent
case class ChannelFailed(channel: ActorRef, channelId: BinaryData, remoteNodeId: PublicKey, data: Data, error: ChannelError) extends ChannelEvent
case class ChannelFailed(channel: ActorRef, channelId: ByteVector32, remoteNodeId: PublicKey, data: Data, error: ChannelError) extends ChannelEvent
case class NetworkFeePaid(channel: ActorRef, remoteNodeId: PublicKey, channelId: BinaryData, tx: Transaction, fee: Satoshi, txType: String) extends ChannelEvent
case class NetworkFeePaid(channel: ActorRef, remoteNodeId: PublicKey, channelId: ByteVector32, tx: Transaction, fee: Satoshi, txType: String) extends ChannelEvent
// NB: this event is only sent when the channel is available
case class AvailableBalanceChanged(channel: ActorRef, channelId: BinaryData, shortChannelId: ShortChannelId, localBalanceMsat: Long, commitments: Commitments) extends ChannelEvent
case class AvailableBalanceChanged(channel: ActorRef, channelId: ByteVector32, shortChannelId: ShortChannelId, localBalanceMsat: Long, commitments: Commitments) extends ChannelEvent
case class ChannelPersisted(channel: ActorRef, remoteNodeId: PublicKey, channelId: BinaryData, data: Data) extends ChannelEvent
case class ChannelPersisted(channel: ActorRef, remoteNodeId: PublicKey, channelId: ByteVector32, data: Data) extends ChannelEvent
case class LocalCommitConfirmed(channel: ActorRef, remoteNodeId: PublicKey, channelId: BinaryData, refundAtBlock: Long) extends ChannelEvent
case class LocalCommitConfirmed(channel: ActorRef, remoteNodeId: PublicKey, channelId: ByteVector32, refundAtBlock: Long) extends ChannelEvent
case class ChannelClosed(channel: ActorRef, channelId: BinaryData, closeType: String, commitments: Commitments)
case class ChannelClosed(channel: ActorRef, channelId: ByteVector32, closeType: String, commitments: Commitments)

View file

@ -17,69 +17,68 @@
package fr.acinq.eclair.channel
import fr.acinq.bitcoin.Crypto.Scalar
import fr.acinq.bitcoin.{BinaryData, Transaction}
import fr.acinq.bitcoin.{ByteVector32, Transaction}
import fr.acinq.eclair.UInt64
import fr.acinq.eclair.payment.{Origin, Relayed}
import fr.acinq.eclair.payment.Origin
import fr.acinq.eclair.wire.{ChannelUpdate, UpdateAddHtlc}
/**
* Created by PM on 11/04/2017.
*/
class ChannelException(val channelId: BinaryData, message: String) extends RuntimeException(message)
class ChannelException(val channelId: ByteVector32, message: String) extends RuntimeException(message)
// @formatter:off
case class DebugTriggeredException (override val channelId: BinaryData) extends ChannelException(channelId, "debug-mode triggered failure")
case class InvalidChainHash (override val channelId: BinaryData, local: BinaryData, remote: BinaryData) extends ChannelException(channelId, s"invalid chainHash (local=$local remote=$remote)")
case class InvalidFundingAmount (override val channelId: BinaryData, fundingSatoshis: Long, min: Long, max: Long) extends ChannelException(channelId, s"invalid funding_satoshis=$fundingSatoshis (min=$min max=$max)")
case class InvalidPushAmount (override val channelId: BinaryData, pushMsat: Long, max: Long) extends ChannelException(channelId, s"invalid pushMsat=$pushMsat (max=$max)")
case class InvalidMaxAcceptedHtlcs (override val channelId: BinaryData, maxAcceptedHtlcs: Int, max: Int) extends ChannelException(channelId, s"invalid max_accepted_htlcs=$maxAcceptedHtlcs (max=$max)")
case class DustLimitTooSmall (override val channelId: BinaryData, dustLimitSatoshis: Long, min: Long) extends ChannelException(channelId, s"dustLimitSatoshis=$dustLimitSatoshis is too small (min=$min)")
case class DustLimitTooLarge (override val channelId: BinaryData, dustLimitSatoshis: Long, max: Long) extends ChannelException(channelId, s"dustLimitSatoshis=$dustLimitSatoshis is too large (max=$max)")
case class DustLimitAboveOurChannelReserve (override val channelId: BinaryData, dustLimitSatoshis: Long, channelReserveSatoshis: Long) extends ChannelException(channelId, s"dustLimitSatoshis dustLimitSatoshis=$dustLimitSatoshis is above our channelReserveSatoshis=$channelReserveSatoshis")
case class ToSelfDelayTooHigh (override val channelId: BinaryData, toSelfDelay: Int, max: Int) extends ChannelException(channelId, s"unreasonable to_self_delay=$toSelfDelay (max=$max)")
case class ChannelReserveTooHigh (override val channelId: BinaryData, channelReserveSatoshis: Long, reserveToFundingRatio: Double, maxReserveToFundingRatio: Double) extends ChannelException(channelId, s"channelReserveSatoshis too high: reserve=$channelReserveSatoshis fundingRatio=$reserveToFundingRatio maxFundingRatio=$maxReserveToFundingRatio")
case class ChannelReserveBelowOurDustLimit (override val channelId: BinaryData, channelReserveSatoshis: Long, dustLimitSatoshis: Long) extends ChannelException(channelId, s"their channelReserveSatoshis=$channelReserveSatoshis is below our dustLimitSatoshis=$dustLimitSatoshis")
case class ChannelReserveNotMet (override val channelId: BinaryData, toLocalMsat: Long, toRemoteMsat: Long, reserveSatoshis: Long) extends ChannelException(channelId, s"channel reserve is not met toLocalMsat=$toLocalMsat toRemoteMsat=$toRemoteMsat reserveSat=$reserveSatoshis")
case class ChannelFundingError (override val channelId: BinaryData) extends ChannelException(channelId, "channel funding error")
case class NoMoreHtlcsClosingInProgress (override val channelId: BinaryData) extends ChannelException(channelId, "cannot send new htlcs, closing in progress")
case class ClosingAlreadyInProgress (override val channelId: BinaryData) extends ChannelException(channelId, "closing already in progress")
case class CannotCloseWithUnsignedOutgoingHtlcs(override val channelId: BinaryData) extends ChannelException(channelId, "cannot close when there are unsigned outgoing htlcs")
case class ChannelUnavailable (override val channelId: BinaryData) extends ChannelException(channelId, "channel is unavailable (offline or closing)")
case class InvalidFinalScript (override val channelId: BinaryData) extends ChannelException(channelId, "invalid final script")
case class FundingTxTimedout (override val channelId: BinaryData) extends ChannelException(channelId, "funding tx timed out")
case class FundingTxSpent (override val channelId: BinaryData, spendingTx: Transaction) extends ChannelException(channelId, s"funding tx has been spent by txid=${spendingTx.txid}")
case class HtlcTimedout (override val channelId: BinaryData, htlcs: Set[UpdateAddHtlc]) extends ChannelException(channelId, s"one or more htlcs timed out: ids=${htlcs.take(10).map(_.id).mkString}") // we only display the first 10 ids
case class HtlcOverridenByLocalCommit (override val channelId: BinaryData) extends ChannelException(channelId, "htlc was overriden by local commit")
case class FeerateTooSmall (override val channelId: BinaryData, remoteFeeratePerKw: Long) extends ChannelException(channelId, s"remote fee rate is too small: remoteFeeratePerKw=$remoteFeeratePerKw")
case class FeerateTooDifferent (override val channelId: BinaryData, localFeeratePerKw: Long, remoteFeeratePerKw: Long) extends ChannelException(channelId, s"local/remote feerates are too different: remoteFeeratePerKw=$remoteFeeratePerKw localFeeratePerKw=$localFeeratePerKw")
case class InvalidCommitmentSignature (override val channelId: BinaryData, tx: Transaction) extends ChannelException(channelId, s"invalid commitment signature: tx=$tx")
case class InvalidHtlcSignature (override val channelId: BinaryData, tx: Transaction) extends ChannelException(channelId, s"invalid htlc signature: tx=$tx")
case class InvalidCloseSignature (override val channelId: BinaryData, tx: Transaction) extends ChannelException(channelId, s"invalid close signature: tx=$tx")
case class InvalidCloseFee (override val channelId: BinaryData, feeSatoshi: Long) extends ChannelException(channelId, s"invalid close fee: fee_satoshis=$feeSatoshi")
case class HtlcSigCountMismatch (override val channelId: BinaryData, expected: Int, actual: Int) extends ChannelException(channelId, s"htlc sig count mismatch: expected=$expected actual: $actual")
case class ForcedLocalCommit (override val channelId: BinaryData) extends ChannelException(channelId, s"forced local commit")
case class UnexpectedHtlcId (override val channelId: BinaryData, expected: Long, actual: Long) extends ChannelException(channelId, s"unexpected htlc id: expected=$expected actual=$actual")
case class InvalidPaymentHash (override val channelId: BinaryData) extends ChannelException(channelId, "invalid payment hash")
case class ExpiryTooSmall (override val channelId: BinaryData, minimum: Long, actual: Long, blockCount: Long) extends ChannelException(channelId, s"expiry too small: minimum=$minimum actual=$actual blockCount=$blockCount")
case class ExpiryTooBig (override val channelId: BinaryData, maximum: Long, actual: Long, blockCount: Long) extends ChannelException(channelId, s"expiry too big: maximum=$maximum actual=$actual blockCount=$blockCount")
case class HtlcValueTooSmall (override val channelId: BinaryData, minimum: Long, actual: Long) extends ChannelException(channelId, s"htlc value too small: minimum=$minimum actual=$actual")
case class HtlcValueTooHighInFlight (override val channelId: BinaryData, maximum: UInt64, actual: UInt64) extends ChannelException(channelId, s"in-flight htlcs hold too much value: maximum=$maximum actual=$actual")
case class TooManyAcceptedHtlcs (override val channelId: BinaryData, maximum: Long) extends ChannelException(channelId, s"too many accepted htlcs: maximum=$maximum")
case class InsufficientFunds (override val channelId: BinaryData, amountMsat: Long, missingSatoshis: Long, reserveSatoshis: Long, feesSatoshis: Long) extends ChannelException(channelId, s"insufficient funds: missingSatoshis=$missingSatoshis reserveSatoshis=$reserveSatoshis fees=$feesSatoshis")
case class InvalidHtlcPreimage (override val channelId: BinaryData, id: Long) extends ChannelException(channelId, s"invalid htlc preimage for htlc id=$id")
case class UnknownHtlcId (override val channelId: BinaryData, id: Long) extends ChannelException(channelId, s"unknown htlc id=$id")
case class CannotExtractSharedSecret (override val channelId: BinaryData, htlc: UpdateAddHtlc) extends ChannelException(channelId, s"can't extract shared secret: paymentHash=${htlc.paymentHash} onion=${htlc.onionRoutingPacket}")
case class FundeeCannotSendUpdateFee (override val channelId: BinaryData) extends ChannelException(channelId, s"only the funder should send update_fee messages")
case class CannotAffordFees (override val channelId: BinaryData, missingSatoshis: Long, reserveSatoshis: Long, feesSatoshis: Long) extends ChannelException(channelId, s"can't pay the fee: missingSatoshis=$missingSatoshis reserveSatoshis=$reserveSatoshis feesSatoshis=$feesSatoshis")
case class CannotSignWithoutChanges (override val channelId: BinaryData) extends ChannelException(channelId, "cannot sign when there are no changes")
case class CannotSignBeforeRevocation (override val channelId: BinaryData) extends ChannelException(channelId, "cannot sign until next revocation hash is received")
case class UnexpectedRevocation (override val channelId: BinaryData) extends ChannelException(channelId, "received unexpected RevokeAndAck message")
case class InvalidRevocation (override val channelId: BinaryData) extends ChannelException(channelId, "invalid revocation")
case class InvalidRevokedCommitProof (override val channelId: BinaryData, 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 CommitmentSyncError (override val channelId: BinaryData) extends ChannelException(channelId, "commitment sync error")
case class RevocationSyncError (override val channelId: BinaryData) extends ChannelException(channelId, "revocation sync error")
case class InvalidFailureCode (override val channelId: BinaryData) extends ChannelException(channelId, "UpdateFailMalformedHtlc message doesn't have BADONION bit set")
case class PleasePublishYourCommitment (override val channelId: BinaryData) extends ChannelException(channelId, "please publish your local commitment")
case class AddHtlcFailed (override val channelId: BinaryData, paymentHash: BinaryData, t: Throwable, origin: Origin, channelUpdate: Option[ChannelUpdate], originalCommand: Option[CMD_ADD_HTLC]) extends ChannelException(channelId, s"cannot add htlc with origin=$origin reason=${t.getMessage}")
case class CommandUnavailableInThisState (override val channelId: BinaryData, command: String, state: State) extends ChannelException(channelId, s"cannot execute command=$command in state=$state")
case class DebugTriggeredException (override val channelId: ByteVector32) extends ChannelException(channelId, "debug-mode triggered failure")
case class InvalidChainHash (override val channelId: ByteVector32, local: ByteVector32, remote: ByteVector32) extends ChannelException(channelId, s"invalid chainHash (local=$local remote=$remote)")
case class InvalidFundingAmount (override val channelId: ByteVector32, fundingSatoshis: Long, min: Long, max: Long) extends ChannelException(channelId, s"invalid funding_satoshis=$fundingSatoshis (min=$min max=$max)")
case class InvalidPushAmount (override val channelId: ByteVector32, pushMsat: Long, max: Long) extends ChannelException(channelId, s"invalid pushMsat=$pushMsat (max=$max)")
case class InvalidMaxAcceptedHtlcs (override val channelId: ByteVector32, maxAcceptedHtlcs: Int, max: Int) extends ChannelException(channelId, s"invalid max_accepted_htlcs=$maxAcceptedHtlcs (max=$max)")
case class DustLimitTooSmall (override val channelId: ByteVector32, dustLimitSatoshis: Long, min: Long) extends ChannelException(channelId, s"dustLimitSatoshis=$dustLimitSatoshis is too small (min=$min)")
case class DustLimitTooLarge (override val channelId: ByteVector32, dustLimitSatoshis: Long, max: Long) extends ChannelException(channelId, s"dustLimitSatoshis=$dustLimitSatoshis is too large (max=$max)")
case class DustLimitAboveOurChannelReserve (override val channelId: ByteVector32, dustLimitSatoshis: Long, channelReserveSatoshis: Long) extends ChannelException(channelId, s"dustLimitSatoshis dustLimitSatoshis=$dustLimitSatoshis is above our channelReserveSatoshis=$channelReserveSatoshis")
case class ToSelfDelayTooHigh (override val channelId: ByteVector32, toSelfDelay: Int, max: Int) extends ChannelException(channelId, s"unreasonable to_self_delay=$toSelfDelay (max=$max)")
case class ChannelReserveTooHigh (override val channelId: ByteVector32, channelReserveSatoshis: Long, reserveToFundingRatio: Double, maxReserveToFundingRatio: Double) extends ChannelException(channelId, s"channelReserveSatoshis too high: reserve=$channelReserveSatoshis fundingRatio=$reserveToFundingRatio maxFundingRatio=$maxReserveToFundingRatio")
case class ChannelReserveBelowOurDustLimit (override val channelId: ByteVector32, channelReserveSatoshis: Long, dustLimitSatoshis: Long) extends ChannelException(channelId, s"their channelReserveSatoshis=$channelReserveSatoshis is below our dustLimitSatoshis=$dustLimitSatoshis")
case class ChannelReserveNotMet (override val channelId: ByteVector32, toLocalMsat: Long, toRemoteMsat: Long, reserveSatoshis: Long) extends ChannelException(channelId, s"channel reserve is not met toLocalMsat=$toLocalMsat toRemoteMsat=$toRemoteMsat reserveSat=$reserveSatoshis")
case class ChannelFundingError (override val channelId: ByteVector32) extends ChannelException(channelId, "channel funding error")
case class NoMoreHtlcsClosingInProgress (override val channelId: ByteVector32) extends ChannelException(channelId, "cannot send new htlcs, closing in progress")
case class ClosingAlreadyInProgress (override val channelId: ByteVector32) extends ChannelException(channelId, "closing already in progress")
case class CannotCloseWithUnsignedOutgoingHtlcs(override val channelId: ByteVector32) extends ChannelException(channelId, "cannot close when there are unsigned outgoing htlcs")
case class ChannelUnavailable (override val channelId: ByteVector32) extends ChannelException(channelId, "channel is unavailable (offline or closing)")
case class InvalidFinalScript (override val channelId: ByteVector32) extends ChannelException(channelId, "invalid final script")
case class FundingTxTimedout (override val channelId: ByteVector32) extends ChannelException(channelId, "funding tx timed out")
case class FundingTxSpent (override val channelId: ByteVector32, spendingTx: Transaction) extends ChannelException(channelId, s"funding tx has been spent by txid=${spendingTx.txid}")
case class HtlcTimedout (override val channelId: ByteVector32, htlcs: Set[UpdateAddHtlc]) extends ChannelException(channelId, s"one or more htlcs timed out: ids=${htlcs.take(10).map(_.id).mkString}") // we only display the first 10 ids
case class HtlcOverridenByLocalCommit (override val channelId: ByteVector32) extends ChannelException(channelId, "htlc was overriden by local commit")
case class FeerateTooSmall (override val channelId: ByteVector32, remoteFeeratePerKw: Long) extends ChannelException(channelId, s"remote fee rate is too small: remoteFeeratePerKw=$remoteFeeratePerKw")
case class FeerateTooDifferent (override val channelId: ByteVector32, localFeeratePerKw: Long, remoteFeeratePerKw: Long) extends ChannelException(channelId, s"local/remote feerates are too different: remoteFeeratePerKw=$remoteFeeratePerKw localFeeratePerKw=$localFeeratePerKw")
case class InvalidCommitmentSignature (override val channelId: ByteVector32, tx: Transaction) extends ChannelException(channelId, s"invalid commitment signature: tx=$tx")
case class InvalidHtlcSignature (override val channelId: ByteVector32, tx: Transaction) extends ChannelException(channelId, s"invalid htlc signature: tx=$tx")
case class InvalidCloseSignature (override val channelId: ByteVector32, tx: Transaction) extends ChannelException(channelId, s"invalid close signature: tx=$tx")
case class InvalidCloseFee (override val channelId: ByteVector32, feeSatoshi: Long) extends ChannelException(channelId, s"invalid close fee: fee_satoshis=$feeSatoshi")
case class HtlcSigCountMismatch (override val channelId: ByteVector32, expected: Int, actual: Int) extends ChannelException(channelId, s"htlc sig count mismatch: expected=$expected actual: $actual")
case class ForcedLocalCommit (override val channelId: ByteVector32) extends ChannelException(channelId, s"forced local commit")
case class UnexpectedHtlcId (override val channelId: ByteVector32, expected: Long, actual: Long) extends ChannelException(channelId, s"unexpected htlc id: expected=$expected actual=$actual")
case class ExpiryTooSmall (override val channelId: ByteVector32, minimum: Long, actual: Long, blockCount: Long) extends ChannelException(channelId, s"expiry too small: minimum=$minimum actual=$actual blockCount=$blockCount")
case class ExpiryTooBig (override val channelId: ByteVector32, maximum: Long, actual: Long, blockCount: Long) extends ChannelException(channelId, s"expiry too big: maximum=$maximum actual=$actual blockCount=$blockCount")
case class HtlcValueTooSmall (override val channelId: ByteVector32, minimum: Long, actual: Long) extends ChannelException(channelId, s"htlc value too small: minimum=$minimum actual=$actual")
case class HtlcValueTooHighInFlight (override val channelId: ByteVector32, maximum: UInt64, actual: UInt64) extends ChannelException(channelId, s"in-flight htlcs hold too much value: maximum=$maximum actual=$actual")
case class TooManyAcceptedHtlcs (override val channelId: ByteVector32, maximum: Long) extends ChannelException(channelId, s"too many accepted htlcs: maximum=$maximum")
case class InsufficientFunds (override val channelId: ByteVector32, amountMsat: Long, missingSatoshis: Long, reserveSatoshis: Long, feesSatoshis: Long) extends ChannelException(channelId, s"insufficient funds: missingSatoshis=$missingSatoshis reserveSatoshis=$reserveSatoshis fees=$feesSatoshis")
case class InvalidHtlcPreimage (override val channelId: ByteVector32, id: Long) extends ChannelException(channelId, s"invalid htlc preimage for htlc id=$id")
case class UnknownHtlcId (override val channelId: ByteVector32, id: Long) extends ChannelException(channelId, s"unknown htlc id=$id")
case class CannotExtractSharedSecret (override val channelId: ByteVector32, htlc: UpdateAddHtlc) extends ChannelException(channelId, s"can't extract shared secret: paymentHash=${htlc.paymentHash} onion=${htlc.onionRoutingPacket}")
case class FundeeCannotSendUpdateFee (override val channelId: ByteVector32) extends ChannelException(channelId, s"only the funder should send update_fee messages")
case class CannotAffordFees (override val channelId: ByteVector32, missingSatoshis: Long, reserveSatoshis: Long, feesSatoshis: Long) extends ChannelException(channelId, s"can't pay the fee: missingSatoshis=$missingSatoshis reserveSatoshis=$reserveSatoshis feesSatoshis=$feesSatoshis")
case class CannotSignWithoutChanges (override val channelId: ByteVector32) extends ChannelException(channelId, "cannot sign when there are no changes")
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 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")
case class PleasePublishYourCommitment (override val channelId: ByteVector32) extends ChannelException(channelId, "please publish your local commitment")
case class AddHtlcFailed (override val channelId: ByteVector32, paymentHash: ByteVector32, t: Throwable, origin: Origin, channelUpdate: Option[ChannelUpdate], originalCommand: Option[CMD_ADD_HTLC]) extends ChannelException(channelId, s"cannot add htlc with origin=$origin reason=${t.getMessage}")
case class CommandUnavailableInThisState (override val channelId: ByteVector32, command: String, state: State) extends ChannelException(channelId, s"cannot execute command=$command in state=$state")
// @formatter:on

View file

@ -18,12 +18,13 @@ package fr.acinq.eclair.channel
import akka.actor.ActorRef
import fr.acinq.bitcoin.Crypto.{Point, PublicKey}
import fr.acinq.bitcoin.{BinaryData, DeterministicWallet, OutPoint, Satoshi, Transaction}
import fr.acinq.eclair.{ShortChannelId, UInt64}
import fr.acinq.bitcoin.{ByteVector32, DeterministicWallet, OutPoint, Satoshi, Transaction}
import fr.acinq.eclair.crypto.Sphinx
import fr.acinq.eclair.transactions.CommitmentSpec
import fr.acinq.eclair.transactions.Transactions.CommitTx
import fr.acinq.eclair.wire.{AcceptChannel, ChannelAnnouncement, ChannelReestablish, ChannelUpdate, ClosingSigned, FailureMessage, FundingCreated, FundingLocked, FundingSigned, Init, OpenChannel, Shutdown, UpdateAddHtlc}
import fr.acinq.eclair.{ShortChannelId, UInt64}
import scodec.bits.ByteVector
/**
@ -74,8 +75,8 @@ case object ERR_INFORMATION_LEAK extends State
8888888888 Y8P 8888888888 888 Y888 888 "Y8888P"
*/
case class INPUT_INIT_FUNDER(temporaryChannelId: BinaryData, fundingSatoshis: Long, pushMsat: Long, initialFeeratePerKw: Long, fundingTxFeeratePerKw: Long, localParams: LocalParams, remote: ActorRef, remoteInit: Init, channelFlags: Byte)
case class INPUT_INIT_FUNDEE(temporaryChannelId: BinaryData, localParams: LocalParams, remote: ActorRef, remoteInit: Init)
case class INPUT_INIT_FUNDER(temporaryChannelId: ByteVector32, fundingSatoshis: Long, pushMsat: Long, initialFeeratePerKw: Long, fundingTxFeeratePerKw: Long, localParams: LocalParams, remote: ActorRef, remoteInit: Init, channelFlags: Byte)
case class INPUT_INIT_FUNDEE(temporaryChannelId: ByteVector32, localParams: LocalParams, remote: ActorRef, remoteInit: Init)
case object INPUT_CLOSE_COMPLETE_TIMEOUT // when requesting a mutual close, we wait for as much as this timeout, then unilateral close
case object INPUT_DISCONNECTED
case class INPUT_RECONNECTED(remote: ActorRef, localInit: Init, remoteInit: Init)
@ -105,19 +106,19 @@ case class BITCOIN_PARENT_TX_CONFIRMED(childTx: Transaction) extends BitcoinEven
*/
sealed trait Command
final case class CMD_ADD_HTLC(amountMsat: Long, paymentHash: BinaryData, cltvExpiry: Long, onion: BinaryData = Sphinx.LAST_PACKET.serialize, upstream_opt: Option[UpdateAddHtlc] = None, commit: Boolean = false, redirected: Boolean = false) extends Command
final case class CMD_FULFILL_HTLC(id: Long, r: BinaryData, commit: Boolean = false) extends Command
final case class CMD_FAIL_HTLC(id: Long, reason: Either[BinaryData, FailureMessage], commit: Boolean = false) extends Command
final case class CMD_FAIL_MALFORMED_HTLC(id: Long, onionHash: BinaryData, failureCode: Int, commit: Boolean = false) extends Command
final case class CMD_ADD_HTLC(amountMsat: Long, paymentHash: ByteVector32, cltvExpiry: Long, onion: ByteVector = Sphinx.LAST_PACKET.serialize, upstream_opt: Option[UpdateAddHtlc] = None, commit: Boolean = false, redirected: Boolean = false) 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
final case class CMD_UPDATE_FEE(feeratePerKw: Long, commit: Boolean = false) extends Command
final case object CMD_SIGN extends Command
final case class CMD_CLOSE(scriptPubKey: Option[BinaryData]) extends Command
final case class CMD_CLOSE(scriptPubKey: Option[ByteVector]) extends Command
final case class CMD_UPDATE_RELAY_FEE(feeBaseMsat: Long, feeProportionalMillionths: Long) extends Command
final case object CMD_FORCECLOSE extends Command
final case object CMD_GETSTATE extends Command
final case object CMD_GETSTATEDATA extends Command
final case object CMD_GETINFO extends Command
final case class RES_GETINFO(nodeId: BinaryData, channelId: BinaryData, state: State, data: Data)
final case class RES_GETINFO(nodeId: PublicKey, channelId: ByteVector32, state: State, data: Data)
/*
8888888b. d8888 88888888888 d8888
@ -141,15 +142,15 @@ sealed trait HasCommitments extends Data {
case class ClosingTxProposed(unsignedTx: Transaction, localClosingSigned: ClosingSigned)
case class LocalCommitPublished(commitTx: Transaction, claimMainDelayedOutputTx: Option[Transaction], htlcSuccessTxs: List[Transaction], htlcTimeoutTxs: List[Transaction], claimHtlcDelayedTxs: List[Transaction], irrevocablySpent: Map[OutPoint, BinaryData])
case class RemoteCommitPublished(commitTx: Transaction, claimMainOutputTx: Option[Transaction], claimHtlcSuccessTxs: List[Transaction], claimHtlcTimeoutTxs: List[Transaction], irrevocablySpent: Map[OutPoint, BinaryData])
case class RevokedCommitPublished(commitTx: Transaction, claimMainOutputTx: Option[Transaction], mainPenaltyTx: Option[Transaction], htlcPenaltyTxs: List[Transaction], claimHtlcDelayedPenaltyTxs: List[Transaction], irrevocablySpent: Map[OutPoint, BinaryData])
case class LocalCommitPublished(commitTx: Transaction, claimMainDelayedOutputTx: Option[Transaction], htlcSuccessTxs: List[Transaction], htlcTimeoutTxs: List[Transaction], claimHtlcDelayedTxs: List[Transaction], irrevocablySpent: Map[OutPoint, ByteVector32])
case class RemoteCommitPublished(commitTx: Transaction, claimMainOutputTx: Option[Transaction], claimHtlcSuccessTxs: List[Transaction], claimHtlcTimeoutTxs: List[Transaction], irrevocablySpent: Map[OutPoint, ByteVector32])
case class RevokedCommitPublished(commitTx: Transaction, claimMainOutputTx: Option[Transaction], mainPenaltyTx: Option[Transaction], htlcPenaltyTxs: List[Transaction], claimHtlcDelayedPenaltyTxs: List[Transaction], irrevocablySpent: Map[OutPoint, ByteVector32])
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: BinaryData, 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: BinaryData, 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_SIGNED(channelId: BinaryData, 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_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_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],
waitingSince: Long,
@ -195,9 +196,9 @@ final case class LocalParams(nodeId: PublicKey,
toSelfDelay: Int,
maxAcceptedHtlcs: Int,
isFunder: Boolean,
defaultFinalScriptPubKey: BinaryData,
globalFeatures: BinaryData,
localFeatures: BinaryData)
defaultFinalScriptPubKey: ByteVector,
globalFeatures: ByteVector,
localFeatures: ByteVector)
final case class RemoteParams(nodeId: PublicKey,
dustLimitSatoshis: Long,
@ -211,8 +212,8 @@ final case class RemoteParams(nodeId: PublicKey,
paymentBasepoint: Point,
delayedPaymentBasepoint: Point,
htlcBasepoint: Point,
globalFeatures: BinaryData,
localFeatures: BinaryData)
globalFeatures: ByteVector,
localFeatures: ByteVector)
object ChannelFlags {
val AnnounceChannel = 0x01.toByte

View file

@ -18,13 +18,14 @@ package fr.acinq.eclair.channel
import akka.event.LoggingAdapter
import fr.acinq.bitcoin.Crypto.{Point, PrivateKey, sha256}
import fr.acinq.bitcoin.{BinaryData, Crypto, MilliSatoshi, Satoshi, Transaction}
import fr.acinq.bitcoin.{ByteVector32, Crypto, Satoshi}
import fr.acinq.eclair.crypto.{Generators, KeyManager, ShaChain, Sphinx}
import fr.acinq.eclair.payment._
import fr.acinq.eclair.transactions.Transactions._
import fr.acinq.eclair.transactions._
import fr.acinq.eclair.wire._
import fr.acinq.eclair.{Globals, UInt64}
import scodec.bits.ByteVector
import scala.util.{Failure, Success}
@ -34,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: BinaryData, remoteSig: BinaryData)
case class HtlcTxAndSigs(txinfo: TransactionWithInputInfo, localSig: ByteVector, remoteSig: ByteVector)
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: BinaryData, remotePerCommitmentPoint: Point)
case class RemoteCommit(index: Long, spec: CommitmentSpec, txid: ByteVector32, remotePerCommitmentPoint: Point)
case class WaitingForRevocation(nextRemoteCommit: RemoteCommit, sent: CommitSig, sentAfterLocalCommitIndex: Long, reSignAsap: Boolean = false)
// @formatter:on
@ -57,7 +58,7 @@ case class Commitments(localParams: LocalParams, remoteParams: RemoteParams,
originChannels: Map[Long, Origin], // for outgoing htlcs relayed through us, the id of the previous channel
remoteNextCommitInfo: Either[WaitingForRevocation, Point],
commitInput: InputInfo,
remotePerCommitmentSecrets: ShaChain, channelId: BinaryData) {
remotePerCommitmentSecrets: ShaChain, channelId: ByteVector32) {
def hasNoPendingHtlcs: Boolean = localCommit.spec.htlcs.isEmpty && remoteCommit.spec.htlcs.isEmpty && remoteNextCommitInfo.isRight
@ -101,10 +102,6 @@ object Commitments {
*/
def sendAdd(commitments: Commitments, cmd: CMD_ADD_HTLC, origin: Origin): Either[ChannelException, (Commitments, UpdateAddHtlc)] = {
if (cmd.paymentHash.size != 32) {
return Left(InvalidPaymentHash(commitments.channelId))
}
val blockCount = Globals.blockCount.get()
// our counterparty needs a reasonable amount of time to pull the funds from downstream before we can get refunded (see BOLT 2 and BOLT 11 for a calculation and rationale)
val minExpiry = blockCount + Channel.MIN_CLTV_EXPIRY
@ -157,10 +154,6 @@ object Commitments {
throw UnexpectedHtlcId(commitments.channelId, expected = commitments.remoteNextHtlcId, actual = add.id)
}
if (add.paymentHash.size != 32) {
throw InvalidPaymentHash(commitments.channelId)
}
if (add.amountMsat < commitments.localParams.htlcMinimumMsat) {
throw HtlcValueTooSmall(commitments.channelId, minimum = commitments.localParams.htlcMinimumMsat, actual = add.amountMsat)
}
@ -353,9 +346,9 @@ object Commitments {
def remoteHasChanges(commitments: Commitments): Boolean = commitments.localChanges.acked.size > 0 || commitments.remoteChanges.proposed.size > 0
def revocationPreimage(seed: BinaryData, index: Long): BinaryData = ShaChain.shaChainFromSeed(seed, 0xFFFFFFFFFFFFFFFFL - index)
def revocationPreimage(seed: ByteVector32, index: Long): ByteVector32 = ShaChain.shaChainFromSeed(seed, 0xFFFFFFFFFFFFFFFFL - index)
def revocationHash(seed: BinaryData, index: Long): BinaryData = Crypto.sha256(revocationPreimage(seed, index))
def revocationHash(seed: ByteVector32, index: Long): ByteVector32 = Crypto.sha256(revocationPreimage(seed, index))
def sendCommit(commitments: Commitments, keyManager: KeyManager)(implicit log: LoggingAdapter): (Commitments, CommitSig) = {
import commitments._
@ -499,7 +492,7 @@ object Commitments {
remoteChanges = remoteChanges.copy(signed = Nil),
remoteCommit = theirNextCommit,
remoteNextCommitInfo = Right(revocation.nextPerCommitmentPoint),
remotePerCommitmentSecrets = commitments.remotePerCommitmentSecrets.addHash(revocation.perCommitmentSecret, 0xFFFFFFFFFFFFL - commitments.remoteCommit.index),
remotePerCommitmentSecrets = commitments.remotePerCommitmentSecrets.addHash(revocation.perCommitmentSecret.toBin, 0xFFFFFFFFFFFFL - commitments.remoteCommit.index),
originChannels = originChannels1)
(commitments1, forwards)
case Right(_) =>

View file

@ -28,8 +28,8 @@ import fr.acinq.eclair.transactions.Transactions._
import fr.acinq.eclair.transactions._
import fr.acinq.eclair.wire._
import fr.acinq.eclair.{Globals, NodeParams, ShortChannelId, addressToPublicKeyScript}
import scodec.bits.ByteVector
import scala.compat.Platform
import scala.concurrent.Await
import scala.concurrent.duration.FiniteDuration
import scala.util.{Failure, Success, Try}
@ -46,8 +46,8 @@ object Helpers {
* @param stateData
* @return the long identifier of the channel
*/
def getChannelId(stateData: Data): BinaryData = stateData match {
case Nothing => BinaryData("00" * 32)
def getChannelId(stateData: Data): ByteVector32 = stateData match {
case Nothing => ByteVector32.Zeroes
case d: DATA_WAIT_FOR_OPEN_CHANNEL => d.initFundee.temporaryChannelId
case d: DATA_WAIT_FOR_ACCEPT_CHANNEL => d.initFunder.temporaryChannelId
case d: DATA_WAIT_FOR_FUNDING_INTERNAL => d.temporaryChannelId
@ -184,7 +184,7 @@ object Helpers {
}
def makeAnnouncementSignatures(nodeParams: NodeParams, commitments: Commitments, shortChannelId: ShortChannelId) = {
val features = BinaryData.empty // empty features for now
val features = ByteVector.empty // empty features for now
val (localNodeSig, localBitcoinSig) = nodeParams.keyManager.signChannelAnnouncement(commitments.localParams.channelKeyPath, nodeParams.chainHash, shortChannelId, commitments.remoteParams.nodeId, commitments.remoteParams.fundingPubKey, features)
AnnouncementSignatures(commitments.channelId, shortChannelId, localNodeSig, localBitcoinSig)
}
@ -206,7 +206,7 @@ object Helpers {
result
}
def getFinalScriptPubKey(wallet: EclairWallet, chainHash: BinaryData): BinaryData = {
def getFinalScriptPubKey(wallet: EclairWallet, chainHash: ByteVector32): ByteVector = {
import scala.concurrent.duration._
val finalAddress = Await.result(wallet.getFinalAddress, 40 seconds)
@ -215,7 +215,7 @@ object Helpers {
object Funding {
def makeFundingInputInfo(fundingTxId: BinaryData, fundingTxOutputIndex: Int, fundingSatoshis: Satoshi, fundingPubkey1: PublicKey, fundingPubkey2: PublicKey): InputInfo = {
def makeFundingInputInfo(fundingTxId: ByteVector32, fundingTxOutputIndex: Int, fundingSatoshis: Satoshi, fundingPubkey1: PublicKey, fundingPubkey2: PublicKey): InputInfo = {
val fundingScript = multiSig2of2(fundingPubkey1, fundingPubkey2)
val fundingTxOut = TxOut(fundingSatoshis, pay2wsh(fundingScript))
InputInfo(OutPoint(fundingTxId, fundingTxOutputIndex), fundingTxOut, write(fundingScript))
@ -232,7 +232,7 @@ object Helpers {
* @param remoteFirstPerCommitmentPoint
* @return (localSpec, localTx, remoteSpec, remoteTx, fundingTxOutput)
*/
def makeFirstCommitTxs(keyManager: KeyManager, temporaryChannelId: BinaryData, localParams: LocalParams, remoteParams: RemoteParams, fundingSatoshis: Long, pushMsat: Long, initialFeeratePerKw: Long, fundingTxHash: BinaryData, 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: Point, 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
@ -350,9 +350,9 @@ object Helpers {
data.commitments.remoteNextCommitInfo.isRight
// used only to compute tx weights and estimate fees
lazy val dummyPublicKey = PrivateKey(BinaryData("01" * 32), true).publicKey
lazy val dummyPublicKey = PrivateKey(ByteVector32(ByteVector.fill(32)(1)), true).publicKey
def isValidFinalScriptPubkey(scriptPubKey: BinaryData): Boolean = {
def isValidFinalScriptPubkey(scriptPubKey: ByteVector): Boolean = {
Try(Script.parse(scriptPubKey)) match {
case Success(OP_DUP :: OP_HASH160 :: OP_PUSHDATA(pubkeyHash, _) :: OP_EQUALVERIFY :: OP_CHECKSIG :: Nil) if pubkeyHash.size == 20 => true
case Success(OP_HASH160 :: OP_PUSHDATA(scriptHash, _) :: OP_EQUAL :: Nil) if scriptHash.size == 20 => true
@ -362,11 +362,11 @@ object Helpers {
}
}
def firstClosingFee(commitments: Commitments, localScriptPubkey: BinaryData, remoteScriptPubkey: BinaryData)(implicit log: LoggingAdapter): Satoshi = {
def firstClosingFee(commitments: Commitments, localScriptPubkey: ByteVector, remoteScriptPubkey: ByteVector)(implicit log: LoggingAdapter): Satoshi = {
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, "aa" * 71, "bb" * 71).tx)
val closingWeight = Transaction.weight(Transactions.addSigs(dummyClosingTx, dummyPublicKey, remoteParams.fundingPubKey, ByteVector.fill(71)(0xaa), ByteVector.fill(71)(0xbb)).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")
@ -375,12 +375,12 @@ object Helpers {
def nextClosingFee(localClosingFee: Satoshi, remoteClosingFee: Satoshi): Satoshi = ((localClosingFee + remoteClosingFee) / 4) * 2
def makeFirstClosingTx(keyManager: KeyManager, commitments: Commitments, localScriptPubkey: BinaryData, remoteScriptPubkey: BinaryData)(implicit log: LoggingAdapter): (ClosingTx, ClosingSigned) = {
def makeFirstClosingTx(keyManager: KeyManager, commitments: Commitments, localScriptPubkey: ByteVector, remoteScriptPubkey: ByteVector)(implicit log: LoggingAdapter): (ClosingTx, ClosingSigned) = {
val closingFee = firstClosingFee(commitments, localScriptPubkey, remoteScriptPubkey)
makeClosingTx(keyManager, commitments, localScriptPubkey, remoteScriptPubkey, closingFee)
}
def makeClosingTx(keyManager: KeyManager, commitments: Commitments, localScriptPubkey: BinaryData, remoteScriptPubkey: BinaryData, closingFee: Satoshi)(implicit log: LoggingAdapter): (ClosingTx, ClosingSigned) = {
def makeClosingTx(keyManager: KeyManager, commitments: Commitments, localScriptPubkey: ByteVector, remoteScriptPubkey: ByteVector, closingFee: Satoshi)(implicit log: LoggingAdapter): (ClosingTx, ClosingSigned) = {
import commitments._
require(isValidFinalScriptPubkey(localScriptPubkey), "invalid localScriptPubkey")
require(isValidFinalScriptPubkey(remoteScriptPubkey), "invalid remoteScriptPubkey")
@ -395,7 +395,7 @@ object Helpers {
(closingTx, closingSigned)
}
def checkClosingSignature(keyManager: KeyManager, commitments: Commitments, localScriptPubkey: BinaryData, remoteScriptPubkey: BinaryData, remoteClosingFee: Satoshi, remoteClosingSig: BinaryData)(implicit log: LoggingAdapter): Try[Transaction] = {
def checkClosingSignature(keyManager: KeyManager, commitments: Commitments, localScriptPubkey: ByteVector, remoteScriptPubkey: ByteVector, remoteClosingFee: Satoshi, remoteClosingSig: ByteVector)(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) {
@ -734,10 +734,10 @@ object Helpers {
val paymentPreimages = tx.txIn.map(_.witness match {
case ScriptWitness(Seq(localSig, paymentPreimage, htlcOfferedScript)) if paymentPreimage.size == 32 =>
log.info(s"extracted paymentPreimage=$paymentPreimage from tx=$tx (claim-htlc-success)")
Some(paymentPreimage)
case ScriptWitness(Seq(BinaryData.empty, remoteSig, localSig, paymentPreimage, htlcReceivedScript)) if paymentPreimage.size == 32 =>
Some(ByteVector32(paymentPreimage))
case ScriptWitness(Seq(ByteVector.empty, remoteSig, localSig, paymentPreimage, htlcReceivedScript)) if paymentPreimage.size == 32 =>
log.info(s"extracted paymentPreimage=$paymentPreimage from tx=$tx (htlc-success)")
Some(paymentPreimage)
Some(ByteVector32(paymentPreimage))
case _ => None
}).toSet.flatten
paymentPreimages flatMap { paymentPreimage =>
@ -770,8 +770,8 @@ object Helpers {
} else {
// maybe this is a timeout tx, in that case we can resolve and fail the corresponding htlc
tx.txIn.map(_.witness match {
case ScriptWitness(Seq(BinaryData.empty, remoteSig, localSig, BinaryData.empty, htlcOfferedScript)) =>
val paymentHash160 = BinaryData(htlcOfferedScript.slice(109, 109 + 20))
case ScriptWitness(Seq(ByteVector.empty, remoteSig, localSig, ByteVector.empty, htlcOfferedScript)) =>
val paymentHash160 = htlcOfferedScript.slice(109, 109 + 20)
log.info(s"extracted paymentHash160=$paymentHash160 from tx=$tx (htlc-timeout)")
localCommit.spec.htlcs.filter(_.direction == OUT).map(_.add).filter(add => ripemd160(add.paymentHash) == paymentHash160)
case _ => Set.empty
@ -794,8 +794,8 @@ object Helpers {
} else {
// maybe this is a timeout tx, in that case we can resolve and fail the corresponding htlc
tx.txIn.map(_.witness match {
case ScriptWitness(Seq(remoteSig, BinaryData.empty, htlcReceivedScript)) =>
val paymentHash160 = BinaryData(htlcReceivedScript.slice(69, 69 + 20))
case ScriptWitness(Seq(remoteSig, ByteVector.empty, htlcReceivedScript)) =>
val paymentHash160 = htlcReceivedScript.slice(69, 69 + 20)
log.info(s"extracted paymentHash160=$paymentHash160 from tx=$tx (claim-htlc-timeout)")
remoteCommit.spec.htlcs.filter(_.direction == IN).map(_.add).filter(add => ripemd160(add.paymentHash) == paymentHash160)
case _ => Set.empty
@ -989,7 +989,7 @@ object Helpers {
* @param irrevocablySpent a map of known spent outpoints
* @return true if we know for sure that the utxos consumed by the tx have already irrevocably been spent, false otherwise
*/
def inputsAlreadySpent(tx: Transaction, irrevocablySpent: Map[OutPoint, BinaryData]): Boolean = {
def inputsAlreadySpent(tx: Transaction, irrevocablySpent: Map[OutPoint, ByteVector32]): Boolean = {
require(tx.txIn.size == 1, "only tx with one input is supported")
val outPoint = tx.txIn.head.outPoint
irrevocablySpent.contains(outPoint)
@ -1008,7 +1008,7 @@ object Helpers {
// only funder pays the fee
if (d.commitments.localParams.isFunder) {
// we build a map with all known txes (that's not particularly efficient, but it doesn't really matter)
val txes: Map[BinaryData, (Transaction, String)] = (
val txes: Map[ByteVector32, (Transaction, String)] = (
d.mutualClosePublished.map(_ -> "mutual") ++
d.localCommitPublished.map(_.commitTx).map(_ -> "local-commit").toSeq ++
d.localCommitPublished.flatMap(_.claimMainDelayedOutputTx).map(_ -> "local-main-delayed") ++

View file

@ -18,7 +18,7 @@ package fr.acinq.eclair.channel
import akka.actor.Status.Failure
import akka.actor.{Actor, ActorLogging, ActorRef, Terminated}
import fr.acinq.bitcoin.BinaryData
import fr.acinq.bitcoin.ByteVector32
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.eclair.ShortChannelId
import fr.acinq.eclair.channel.Register._
@ -36,7 +36,7 @@ class Register extends Actor with ActorLogging {
override def receive: Receive = main(Map.empty, Map.empty, Map.empty)
def main(channels: Map[BinaryData, ActorRef], shortIds: Map[ShortChannelId, BinaryData], channelsTo: Map[BinaryData, PublicKey]): Receive = {
def main(channels: Map[ByteVector32, ActorRef], shortIds: Map[ShortChannelId, ByteVector32], channelsTo: Map[ByteVector32, PublicKey]): Receive = {
case ChannelCreated(channel, _, remoteNodeId, _, temporaryChannelId) =>
context.watch(channel)
context become main(channels + (temporaryChannelId -> channel), shortIds, channelsTo + (temporaryChannelId -> remoteNodeId))
@ -79,7 +79,7 @@ class Register extends Actor with ActorLogging {
object Register {
// @formatter:off
case class Forward[T](channelId: BinaryData, message: T)
case class Forward[T](channelId: ByteVector32, message: T)
case class ForwardShortId[T](shortChannelId: ShortChannelId, message: T)
case class ForwardFailure[T](fwd: Forward[T]) extends RuntimeException(s"channel ${fwd.channelId} not found")

View file

@ -18,10 +18,11 @@ package fr.acinq.eclair.crypto
import java.nio.ByteOrder
import fr.acinq.bitcoin.{BinaryData, Protocol}
import fr.acinq.bitcoin.{ByteVector32, Protocol}
import grizzled.slf4j.Logging
import org.spongycastle.crypto.engines.{ChaCha7539Engine, ChaChaEngine}
import org.spongycastle.crypto.engines.ChaCha7539Engine
import org.spongycastle.crypto.params.{KeyParameter, ParametersWithIV}
import scodec.bits.ByteVector
/**
* Poly1305 authenticator
@ -34,13 +35,13 @@ object Poly1305 {
* @param data input data
* @return a 16 byte authentication tag
*/
def mac(key: BinaryData, data: BinaryData): BinaryData = {
def mac(key: ByteVector, datas: ByteVector*): ByteVector = {
val out = new Array[Byte](16)
val poly = new org.spongycastle.crypto.macs.Poly1305()
poly.init(new KeyParameter(key))
poly.update(data, 0, data.length)
poly.init(new KeyParameter(key.toArray))
datas.foreach(data => poly.update(data.toArray, 0, data.length.toInt))
poly.doFinal(out, 0)
out
ByteVector.view(out)
}
}
@ -50,10 +51,10 @@ object Poly1305 {
*/
object ChaCha20 {
def encrypt(plaintext: BinaryData, key: BinaryData, nonce: BinaryData, counter: Int = 0): BinaryData = {
def encrypt(plaintext: ByteVector, key: ByteVector, nonce: ByteVector, counter: Int = 0): ByteVector = {
val engine = new ChaCha7539Engine()
engine.init(true, new ParametersWithIV(new KeyParameter(key), nonce))
val ciphertext: BinaryData = new Array[Byte](plaintext.length)
engine.init(true, new ParametersWithIV(new KeyParameter(key.toArray), nonce.toArray))
val ciphertext: Array[Byte] = new Array[Byte](plaintext.length.toInt)
counter match {
case 0 => ()
case 1 =>
@ -62,15 +63,15 @@ object ChaCha20 {
engine.processBytes(new Array[Byte](64), 0, 64, dummy, 0)
case _ => throw new RuntimeException(s"chacha20 counter must be 0 or 1")
}
val len = engine.processBytes(plaintext.toArray, 0, plaintext.length, ciphertext, 0)
val len = engine.processBytes(plaintext.toArray, 0, plaintext.length.toInt, ciphertext, 0)
require(len == plaintext.length, "ChaCha20 encryption failed")
ciphertext
ByteVector.view(ciphertext)
}
def decrypt(ciphertext: BinaryData, key: BinaryData, nonce: BinaryData, counter: Int = 0): BinaryData = {
def decrypt(ciphertext: ByteVector, key: ByteVector, nonce: ByteVector, counter: Int = 0): ByteVector = {
val engine = new ChaCha7539Engine
engine.init(false, new ParametersWithIV(new KeyParameter(key), nonce))
val plaintext: BinaryData = new Array[Byte](ciphertext.length)
engine.init(false, new ParametersWithIV(new KeyParameter(key.toArray), nonce.toArray))
val plaintext: Array[Byte] = new Array[Byte](ciphertext.length.toInt)
counter match {
case 0 => ()
case 1 =>
@ -79,9 +80,9 @@ object ChaCha20 {
engine.processBytes(new Array[Byte](64), 0, 64, dummy, 0)
case _ => throw new RuntimeException(s"chacha20 counter must be 0 or 1")
}
val len = engine.processBytes(ciphertext.toArray, 0, ciphertext.length, plaintext, 0)
val len = engine.processBytes(ciphertext.toArray, 0, ciphertext.length.toInt, plaintext, 0)
require(len == ciphertext.length, "ChaCha20 decryption failed")
plaintext
ByteVector.view(plaintext)
}
}
@ -92,6 +93,7 @@ object ChaCha20 {
* This what we should be using (see BOLT #8)
*/
object ChaCha20Poly1305 extends Logging {
/**
*
* @param key 32 bytes encryption key
@ -100,11 +102,10 @@ object ChaCha20Poly1305 extends Logging {
* @param aad additional authentication data. can be empty
* @return a (ciphertext, mac) tuple
*/
def encrypt(key: BinaryData, nonce: BinaryData, plaintext: BinaryData, aad: BinaryData): (BinaryData, BinaryData) = {
val polykey: BinaryData = ChaCha20.encrypt(new Array[Byte](32), key, nonce)
def encrypt(key: ByteVector, nonce: ByteVector, plaintext: ByteVector, aad: ByteVector): (ByteVector, ByteVector) = {
val polykey = ChaCha20.encrypt(ByteVector32.Zeroes, key, nonce)
val ciphertext = ChaCha20.encrypt(plaintext, key, nonce, 1)
val data = aad ++ pad16(aad) ++ ciphertext ++ pad16(ciphertext) ++ Protocol.writeUInt64(aad.length, ByteOrder.LITTLE_ENDIAN) ++ Protocol.writeUInt64(ciphertext.length, ByteOrder.LITTLE_ENDIAN)
val tag = Poly1305.mac(polykey, data)
val tag = Poly1305.mac(polykey, aad, pad16(aad), ciphertext, pad16(ciphertext), Protocol.writeUInt64(aad.length, ByteOrder.LITTLE_ENDIAN), Protocol.writeUInt64(ciphertext.length, ByteOrder.LITTLE_ENDIAN))
logger.debug(s"encrypt($key, $nonce, $aad, $plaintext) = ($ciphertext, $tag)")
(ciphertext, tag)
}
@ -118,80 +119,19 @@ object ChaCha20Poly1305 extends Logging {
* @param mac authentication mac
* @return the decrypted plaintext if the mac is valid.
*/
def decrypt(key: BinaryData, nonce: BinaryData, ciphertext: BinaryData, aad: BinaryData, mac: BinaryData): BinaryData = {
val polykey: BinaryData = ChaCha20.encrypt(new Array[Byte](32), key, nonce)
val data = aad ++ pad16(aad) ++ ciphertext ++ pad16(ciphertext) ++ Protocol.writeUInt64(aad.length, ByteOrder.LITTLE_ENDIAN) ++ Protocol.writeUInt64(ciphertext.length, ByteOrder.LITTLE_ENDIAN)
val tag = Poly1305.mac(polykey, data)
def decrypt(key: ByteVector, nonce: ByteVector, ciphertext: ByteVector, aad: ByteVector, mac: ByteVector): ByteVector = {
val polykey = ChaCha20.encrypt(ByteVector32.Zeroes, key, nonce)
val tag = Poly1305.mac(polykey, aad, pad16(aad), ciphertext, pad16(ciphertext), Protocol.writeUInt64(aad.length, ByteOrder.LITTLE_ENDIAN), Protocol.writeUInt64(ciphertext.length, ByteOrder.LITTLE_ENDIAN))
require(tag == mac, "invalid mac")
val plaintext = ChaCha20.decrypt(ciphertext, key, nonce, 1)
logger.debug(s"decrypt($key, $nonce, $aad, $ciphertext, $mac) = $plaintext")
plaintext
}
def pad16(data: Seq[Byte]): Seq[Byte] =
def pad16(data: ByteVector): ByteVector =
if (data.size % 16 == 0)
Seq.empty[Byte]
ByteVector.empty
else
Seq.fill[Byte](16 - (data.size % 16))(0)
ByteVector.fill(16 - (data.size % 16))(0)
}
object ChaCha20Legacy {
def encrypt(plaintext: BinaryData, key: BinaryData, nonce: BinaryData, counter: Int = 0): BinaryData = {
val engine = new ChaChaEngine(20)
engine.init(true, new ParametersWithIV(new KeyParameter(key), nonce))
val ciphertext: BinaryData = new Array[Byte](plaintext.length)
counter match {
case 0 => ()
case 1 =>
// skip 1 block == set counter to 1 instead of 0
val dummy = new Array[Byte](64)
engine.processBytes(new Array[Byte](64), 0, 64, dummy, 0)
case _ => throw new RuntimeException(s"chacha20 counter must be 0 or 1")
}
val len = engine.processBytes(plaintext.toArray, 0, plaintext.length, ciphertext, 0)
require(len == plaintext.length, "ChaCha20Legacy encryption failed")
ciphertext
}
def decrypt(ciphertext: BinaryData, key: BinaryData, nonce: BinaryData, counter: Int = 0): BinaryData = {
val engine = new ChaChaEngine(20)
engine.init(false, new ParametersWithIV(new KeyParameter(key), nonce))
val plaintext: BinaryData = new Array[Byte](ciphertext.length)
counter match {
case 0 => ()
case 1 =>
// skip 1 block == set counter to 1 instead of 0
val dummy = new Array[Byte](64)
engine.processBytes(new Array[Byte](64), 0, 64, dummy, 0)
case _ => throw new RuntimeException(s"chacha20 counter must be 0 or 1")
}
val len = engine.processBytes(ciphertext.toArray, 0, ciphertext.length, plaintext, 0)
require(len == ciphertext.length, "ChaCha20Legacy decryption failed")
plaintext
}
}
/**
* Legacy implementation of ChaCha20Poly1305
* Nonce is 8 bytes instead of 12 and the output tag computation is different
*
* Used in our first interop tests with lightning-c, should not be needed anymore
*/
object Chacha20Poly1305Legacy {
def encrypt(key: BinaryData, nonce: BinaryData, plaintext: BinaryData, aad: BinaryData): (BinaryData, BinaryData) = {
val polykey: BinaryData = ChaCha20Legacy.encrypt(new Array[Byte](32), key, nonce)
val ciphertext = ChaCha20Legacy.encrypt(plaintext, key, nonce, 1)
val data = aad ++ Protocol.writeUInt64(aad.length, ByteOrder.LITTLE_ENDIAN) ++ ciphertext ++ Protocol.writeUInt64(ciphertext.length, ByteOrder.LITTLE_ENDIAN)
val tag = Poly1305.mac(polykey, data)
(ciphertext, tag)
}
def decrypt(key: BinaryData, nonce: BinaryData, ciphertext: BinaryData, aad: BinaryData, mac: BinaryData): BinaryData = {
val polykey: BinaryData = ChaCha20Legacy.encrypt(new Array[Byte](32), key, nonce)
val data = aad ++ Protocol.writeUInt64(aad.length, ByteOrder.LITTLE_ENDIAN) ++ ciphertext ++ Protocol.writeUInt64(ciphertext.length, ByteOrder.LITTLE_ENDIAN)
val tag = Poly1305.mac(polykey, data)
require(tag == mac, "invalid mac")
val plaintext = ChaCha20Legacy.decrypt(ciphertext, key, nonce, 1)
plaintext
}
}

View file

@ -17,21 +17,22 @@
package fr.acinq.eclair.crypto
import fr.acinq.bitcoin.Crypto.{Point, PrivateKey, PublicKey, Scalar}
import fr.acinq.bitcoin.{BinaryData, Crypto}
import fr.acinq.bitcoin.{ByteVector32, Crypto}
import scodec.bits.ByteVector
/**
* Created by PM on 07/12/2016.
*/
object Generators {
def fixSize(data: BinaryData): BinaryData = data.length match {
case 32 => data
case length if length < 32 => Array.fill(32 - length)(0.toByte) ++ data
def fixSize(data: ByteVector): ByteVector32 = data.length match {
case 32 => ByteVector32(data)
case length if length < 32 => ByteVector32(data.padLeft(32))
}
def perCommitSecret(seed: BinaryData, index: Long): Scalar = Scalar(ShaChain.shaChainFromSeed(seed, 0xFFFFFFFFFFFFL - index))
def perCommitSecret(seed: ByteVector32, index: Long): Scalar = Scalar(ShaChain.shaChainFromSeed(seed, 0xFFFFFFFFFFFFL - index))
def perCommitPoint(seed: BinaryData, index: Long): Point = perCommitSecret(seed, index).toPoint
def perCommitPoint(seed: ByteVector32, index: Long): Point = perCommitSecret(seed, index).toPoint
def derivePrivKey(secret: Scalar, perCommitPoint: Point): PrivateKey = {
// secretkey = basepoint-secret + SHA256(per-commitment-point || basepoint)

View file

@ -16,11 +16,12 @@
package fr.acinq.eclair.crypto
import fr.acinq.bitcoin.{BinaryData, Crypto, DeterministicWallet}
import fr.acinq.bitcoin.Crypto.{Point, PublicKey, Scalar}
import fr.acinq.bitcoin.DeterministicWallet.ExtendedPublicKey
import fr.acinq.bitcoin.{ByteVector32, Crypto, DeterministicWallet}
import fr.acinq.eclair.ShortChannelId
import fr.acinq.eclair.transactions.Transactions.TransactionWithInputInfo
import scodec.bits.ByteVector
trait KeyManager {
def nodeKey: DeterministicWallet.ExtendedPrivateKey
@ -48,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): BinaryData
def sign(tx: TransactionWithInputInfo, publicKey: ExtendedPublicKey): ByteVector
/**
* This method is used to spend funds send to htlc keys/delayed keys
@ -59,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): BinaryData
def sign(tx: TransactionWithInputInfo, publicKey: ExtendedPublicKey, remotePoint: Point): ByteVector
/**
* Ths method is used to spend revoked transactions, with the corresponding revocation key
@ -70,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): BinaryData
def sign(tx: TransactionWithInputInfo, publicKey: ExtendedPublicKey, remoteSecret: Scalar): ByteVector
def signChannelAnnouncement(channelKeyPath: DeterministicWallet.KeyPath, chainHash: BinaryData, shortChannelId: ShortChannelId, remoteNodeId: PublicKey, remoteFundingKey: PublicKey, features: BinaryData): (BinaryData, BinaryData)
def signChannelAnnouncement(channelKeyPath: DeterministicWallet.KeyPath, chainHash: ByteVector32, shortChannelId: ShortChannelId, remoteNodeId: PublicKey, remoteFundingKey: PublicKey, features: ByteVector): (ByteVector, ByteVector)
}

View file

@ -19,14 +19,15 @@ 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.DeterministicWallet.{derivePrivateKey, _}
import fr.acinq.bitcoin.{BinaryData, Block, Crypto, DeterministicWallet}
import fr.acinq.bitcoin.{Block, ByteVector32, Crypto, DeterministicWallet}
import fr.acinq.eclair.ShortChannelId
import fr.acinq.eclair.router.Announcements
import fr.acinq.eclair.transactions.Transactions
import fr.acinq.eclair.transactions.Transactions.TransactionWithInputInfo
import scodec.bits.ByteVector
object LocalKeyManager {
def channelKeyBasePath(chainHash: BinaryData) = chainHash match {
def channelKeyBasePath(chainHash: ByteVector32) = chainHash match {
case Block.RegtestGenesisBlock.hash | Block.TestnetGenesisBlock.hash => DeterministicWallet.hardened(46) :: DeterministicWallet.hardened(1) :: Nil
case Block.LivenetGenesisBlock.hash => DeterministicWallet.hardened(47) :: DeterministicWallet.hardened(1) :: Nil
}
@ -35,7 +36,7 @@ object LocalKeyManager {
// WARNING: if you change this path, you will change your node id even if the seed remains the same!!!
// Note that the node path and the above channel path are on different branches so even if the
// node key is compromised there is no way to retrieve the wallet keys
def nodeKeyBasePath(chainHash: BinaryData) = chainHash match {
def nodeKeyBasePath(chainHash: ByteVector32) = chainHash match {
case Block.RegtestGenesisBlock.hash | Block.TestnetGenesisBlock.hash => DeterministicWallet.hardened(46) :: DeterministicWallet.hardened(0) :: Nil
case Block.LivenetGenesisBlock.hash => DeterministicWallet.hardened(47) :: DeterministicWallet.hardened(0) :: Nil
}
@ -47,7 +48,7 @@ object LocalKeyManager {
*
* @param seed seed from which keys will be derived
*/
class LocalKeyManager(seed: BinaryData, chainHash: BinaryData) extends KeyManager {
class LocalKeyManager(seed: ByteVector, chainHash: ByteVector32) extends KeyManager {
private val master = DeterministicWallet.generate(seed)
override val nodeKey = DeterministicWallet.derivePrivateKey(master, LocalKeyManager.nodeKeyBasePath(chainHash))
@ -100,7 +101,7 @@ class LocalKeyManager(seed: BinaryData, chainHash: BinaryData) extends KeyManage
* @return a signature generated with the private key that matches the input
* extended public key
*/
def sign(tx: TransactionWithInputInfo, publicKey: ExtendedPublicKey): BinaryData = {
def sign(tx: TransactionWithInputInfo, publicKey: ExtendedPublicKey): ByteVector = {
val privateKey = privateKeys.get(publicKey.path)
Transactions.sign(tx, privateKey.privateKey)
}
@ -114,7 +115,7 @@ class LocalKeyManager(seed: BinaryData, chainHash: BinaryData) extends KeyManage
* @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): BinaryData = {
def sign(tx: TransactionWithInputInfo, publicKey: ExtendedPublicKey, remotePoint: Point): ByteVector = {
val privateKey = privateKeys.get(publicKey.path)
val currentKey = Generators.derivePrivKey(privateKey.privateKey, remotePoint)
Transactions.sign(tx, currentKey)
@ -129,14 +130,14 @@ class LocalKeyManager(seed: BinaryData, chainHash: BinaryData) extends KeyManage
* @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): BinaryData = {
def sign(tx: TransactionWithInputInfo, publicKey: ExtendedPublicKey, remoteSecret: Scalar): ByteVector = {
val privateKey = privateKeys.get(publicKey.path)
val currentKey = Generators.revocationPrivKey(privateKey.privateKey, remoteSecret)
Transactions.sign(tx, currentKey)
}
override def signChannelAnnouncement(channelKeyPath: DeterministicWallet.KeyPath, chainHash: BinaryData, shortChannelId: ShortChannelId, remoteNodeId: PublicKey, remoteFundingKey: PublicKey, features: BinaryData): (BinaryData, BinaryData) = {
val witness = if (Announcements.isNode1(nodeId.toBin, remoteNodeId.toBin)) {
override def signChannelAnnouncement(channelKeyPath: DeterministicWallet.KeyPath, chainHash: ByteVector32, shortChannelId: ShortChannelId, remoteNodeId: PublicKey, remoteFundingKey: PublicKey, features: ByteVector): (ByteVector, ByteVector) = {
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)

View file

@ -19,19 +19,20 @@ package fr.acinq.eclair.crypto
import java.math.BigInteger
import java.nio.ByteOrder
import fr.acinq.bitcoin.{BinaryData, Crypto, Protocol}
import fr.acinq.bitcoin.{Crypto, Protocol}
import fr.acinq.eclair.randomBytes
import grizzled.slf4j.Logging
import org.spongycastle.crypto.digests.SHA256Digest
import org.spongycastle.crypto.macs.HMac
import org.spongycastle.crypto.params.KeyParameter
import scodec.bits.ByteVector
/**
* see http://noiseprotocol.org/
*/
object Noise {
case class KeyPair(pub: BinaryData, priv: BinaryData)
case class KeyPair(pub: ByteVector, priv: ByteVector)
/**
* Diffie-Helmann functions
@ -39,9 +40,9 @@ object Noise {
trait DHFunctions {
def name: String
def generateKeyPair(priv: BinaryData): KeyPair
def generateKeyPair(priv: ByteVector): KeyPair
def dh(keyPair: KeyPair, publicKey: BinaryData): BinaryData
def dh(keyPair: KeyPair, publicKey: ByteVector): ByteVector
def dhLen: Int
@ -51,7 +52,7 @@ object Noise {
object Secp256k1DHFunctions extends DHFunctions {
override val name = "secp256k1"
override def generateKeyPair(priv: BinaryData): KeyPair = {
override def generateKeyPair(priv: ByteVector): KeyPair = {
require(priv.length == 32)
KeyPair(Crypto.publicKeyFromPrivateKey(priv :+ 1.toByte), priv)
}
@ -63,11 +64,11 @@ object Noise {
* @param publicKey
* @return sha256(publicKey * keyPair.priv in compressed format)
*/
override def dh(keyPair: KeyPair, publicKey: BinaryData): BinaryData = {
val point = Crypto.curve.getCurve.decodePoint(publicKey)
override def dh(keyPair: KeyPair, publicKey: ByteVector): ByteVector = {
val point = Crypto.curve.getCurve.decodePoint(publicKey.toArray)
val scalar = new BigInteger(1, keyPair.priv.take(32).toArray)
val point1 = point.multiply(scalar).normalize()
Crypto.sha256(point1.getEncoded(true))
Crypto.sha256(ByteVector.view(point1.getEncoded(true)))
}
override def dhLen: Int = 32
@ -85,29 +86,29 @@ object Noise {
// for the key k. Returns the ciphertext. Encryption must be done with an "AEAD" encryption mode with the associated
// data ad (using the terminology from [1]) and returns a ciphertext that is the same size as the plaintext
// plus 16 bytes for authentication data. The entire ciphertext must be indistinguishable from random if the key is secret.
def encrypt(k: BinaryData, n: Long, ad: BinaryData, plaintext: BinaryData): BinaryData
def encrypt(k: ByteVector, n: Long, ad: ByteVector, plaintext: ByteVector): ByteVector
// Decrypts ciphertext using a cipher key k of 32 bytes, an 8-byte unsigned integer nonce n, and associated data ad.
// Returns the plaintext, unless authentication fails, in which case an error is signaled to the caller.
def decrypt(k: BinaryData, n: Long, ad: BinaryData, ciphertext: BinaryData): BinaryData
def decrypt(k: ByteVector, n: Long, ad: ByteVector, ciphertext: ByteVector): ByteVector
}
object Chacha20Poly1305CipherFunctions extends CipherFunctions {
override val name = "ChaChaPoly"
// as specified in BOLT #8
def nonce(n: Long): BinaryData = BinaryData("00000000") ++ Protocol.writeUInt64(n, ByteOrder.LITTLE_ENDIAN)
def nonce(n: Long): ByteVector = ByteVector.fill(4)(0) ++ Protocol.writeUInt64(n, ByteOrder.LITTLE_ENDIAN)
//Encrypts plaintext using the cipher key k of 32 bytes and an 8-byte unsigned integer nonce n which must be unique
override def encrypt(k: BinaryData, n: Long, ad: BinaryData, plaintext: BinaryData): BinaryData = {
override def encrypt(k: ByteVector, n: Long, ad: ByteVector, plaintext: ByteVector): ByteVector = {
val (ciphertext, mac) = ChaCha20Poly1305.encrypt(k, nonce(n), plaintext, ad)
ciphertext ++ mac
}
// Decrypts ciphertext using a cipher key k of 32 bytes, an 8-byte unsigned integer nonce n, and associated data ad.
override def decrypt(k: BinaryData, n: Long, ad: BinaryData, ciphertextAndMac: BinaryData): BinaryData = {
val ciphertext: BinaryData = ciphertextAndMac.dropRight(16)
val mac: BinaryData = ciphertextAndMac.takeRight(16)
override def decrypt(k: ByteVector, n: Long, ad: ByteVector, ciphertextAndMac: ByteVector): ByteVector = {
val ciphertext: ByteVector = ciphertextAndMac.dropRight(16)
val mac: ByteVector = ciphertextAndMac.takeRight(16)
ChaCha20Poly1305.decrypt(k, nonce(n), ciphertext, ad, mac)
}
}
@ -119,7 +120,7 @@ object Noise {
def name: String
// Hashes some arbitrary-length data with a collision-resistant cryptographic hash function and returns an output of HASHLEN bytes.
def hash(data: BinaryData): BinaryData
def hash(data: ByteVector): ByteVector
// A constant specifying the size in bytes of the hash output. Must be 32 or 64.
def hashLen: Int
@ -128,17 +129,17 @@ object Noise {
def blockLen: Int
// Applies HMAC from [2] using the HASH() function. This function is only called as part of HKDF(), below.
def hmacHash(key: BinaryData, data: BinaryData): BinaryData
def hmacHash(key: ByteVector, data: ByteVector): ByteVector
// Takes a chaining_key byte sequence of length HASHLEN, and an input_key_material byte sequence with length either zero bytes, 32 bytes, or DHLEN bytes. Returns two byte sequences of length HASHLEN, as follows:
// Sets temp_key = HMAC-HASH(chaining_key, input_key_material).
// Sets output1 = HMAC-HASH(temp_key, byte(0x01)).
// Sets output2 = HMAC-HASH(temp_key, output1 || byte(0x02)).
// Returns the pair (output1, output2).
def hkdf(chainingKey: BinaryData, inputMaterial: BinaryData): (BinaryData, BinaryData) = {
def hkdf(chainingKey: ByteVector, inputMaterial: ByteVector): (ByteVector, ByteVector) = {
val tempkey = hmacHash(chainingKey, inputMaterial)
val output1 = hmacHash(tempkey, Seq(0x01.toByte))
val output2 = hmacHash(tempkey, output1 ++ Seq(0x02.toByte))
val output1 = hmacHash(tempkey, ByteVector(0x01))
val output2 = hmacHash(tempkey, output1 ++ ByteVector(0x02))
logger.debug(s"HKDF($chainingKey, $inputMaterial) = ($output1, $output2)")
(output1, output2)
@ -152,15 +153,15 @@ object Noise {
override val blockLen = 64
override def hash(data: BinaryData) = Crypto.sha256(data)
override def hash(data: ByteVector) = Crypto.sha256(data)
override def hmacHash(key: BinaryData, data: BinaryData) = {
override def hmacHash(key: ByteVector, data: ByteVector): ByteVector = {
val mac = new HMac(new SHA256Digest())
mac.init(new KeyParameter(key.toArray))
mac.update(data.toArray, 0, data.length)
mac.update(data.toArray, 0, data.length.toInt)
val out = new Array[Byte](32)
mac.doFinal(out, 0)
out
ByteVector.view(out)
}
}
@ -170,13 +171,13 @@ object Noise {
trait CipherState {
def cipher: CipherFunctions
def initializeKey(key: BinaryData): CipherState = CipherState(key, cipher)
def initializeKey(key: ByteVector): CipherState = CipherState(key, cipher)
def hasKey: Boolean
def encryptWithAd(ad: BinaryData, plaintext: BinaryData): (CipherState, BinaryData)
def encryptWithAd(ad: ByteVector, plaintext: ByteVector): (CipherState, ByteVector)
def decryptWithAd(ad: BinaryData, ciphertext: BinaryData): (CipherState, BinaryData)
def decryptWithAd(ad: ByteVector, ciphertext: ByteVector): (CipherState, ByteVector)
}
/**
@ -187,9 +188,9 @@ object Noise {
case class UninitializedCipherState(cipher: CipherFunctions) extends CipherState {
override val hasKey = false
override def encryptWithAd(ad: BinaryData, plaintext: BinaryData): (CipherState, BinaryData) = (this, plaintext)
override def encryptWithAd(ad: ByteVector, plaintext: ByteVector): (CipherState, ByteVector) = (this, plaintext)
override def decryptWithAd(ad: BinaryData, ciphertext: BinaryData): (CipherState, BinaryData) = (this, ciphertext)
override def decryptWithAd(ad: ByteVector, ciphertext: ByteVector): (CipherState, ByteVector) = (this, ciphertext)
}
/**
@ -199,20 +200,20 @@ object Noise {
* @param n nonce
* @param cipher cipher functions
*/
case class InitializedCipherState(k: BinaryData, n: Long, cipher: CipherFunctions) extends CipherState {
case class InitializedCipherState(k: ByteVector, n: Long, cipher: CipherFunctions) extends CipherState {
require(k.length == 32)
def hasKey = true
def encryptWithAd(ad: BinaryData, plaintext: BinaryData): (CipherState, BinaryData) = {
def encryptWithAd(ad: ByteVector, plaintext: ByteVector): (CipherState, ByteVector) = {
(this.copy(n = this.n + 1), cipher.encrypt(k, n, ad, plaintext))
}
def decryptWithAd(ad: BinaryData, ciphertext: BinaryData): (CipherState, BinaryData) = (this.copy(n = this.n + 1), cipher.decrypt(k, n, ad, ciphertext))
def decryptWithAd(ad: ByteVector, ciphertext: ByteVector): (CipherState, ByteVector) = (this.copy(n = this.n + 1), cipher.decrypt(k, n, ad, ciphertext))
}
object CipherState {
def apply(k: BinaryData, cipher: CipherFunctions): CipherState = k.length match {
def apply(k: ByteVector, cipher: CipherFunctions): CipherState = k.length match {
case 0 => UninitializedCipherState(cipher)
case 32 => InitializedCipherState(k, 0, cipher)
}
@ -227,41 +228,41 @@ object Noise {
* @param h hash
* @param hashFunctions hash functions
*/
case class SymmetricState(cipherState: CipherState, ck: BinaryData, h: BinaryData, hashFunctions: HashFunctions) extends Logging {
def mixKey(inputKeyMaterial: BinaryData): SymmetricState = {
case class SymmetricState(cipherState: CipherState, ck: ByteVector, h: ByteVector, hashFunctions: HashFunctions) extends Logging {
def mixKey(inputKeyMaterial: ByteVector): SymmetricState = {
logger.debug(s"ss = 0x$inputKeyMaterial")
val (ck1, tempk) = hashFunctions.hkdf(ck, inputKeyMaterial)
val tempk1: BinaryData = hashFunctions.hashLen match {
val tempk1: ByteVector = hashFunctions.hashLen match {
case 32 => tempk
case 64 => tempk.take(32)
}
this.copy(cipherState = cipherState.initializeKey(tempk1), ck = ck1)
}
def mixHash(data: BinaryData): SymmetricState = {
def mixHash(data: ByteVector): SymmetricState = {
this.copy(h = hashFunctions.hash(h ++ data))
}
def encryptAndHash(plaintext: BinaryData): (SymmetricState, BinaryData) = {
def encryptAndHash(plaintext: ByteVector): (SymmetricState, ByteVector) = {
val (cipherstate1, ciphertext) = cipherState.encryptWithAd(h, plaintext)
(this.copy(cipherState = cipherstate1).mixHash(ciphertext), ciphertext)
}
def decryptAndHash(ciphertext: BinaryData): (SymmetricState, BinaryData) = {
def decryptAndHash(ciphertext: ByteVector): (SymmetricState, ByteVector) = {
val (cipherstate1, plaintext) = cipherState.decryptWithAd(h, ciphertext)
(this.copy(cipherState = cipherstate1).mixHash(ciphertext), plaintext)
}
def split: (CipherState, CipherState, BinaryData) = {
val (tempk1, tempk2) = hashFunctions.hkdf(ck, BinaryData.empty)
def split: (CipherState, CipherState, ByteVector) = {
val (tempk1, tempk2) = hashFunctions.hkdf(ck, ByteVector.empty)
(cipherState.initializeKey(tempk1.take(32)), cipherState.initializeKey(tempk2.take(32)), ck)
}
}
object SymmetricState {
def apply(protocolName: BinaryData, cipherFunctions: CipherFunctions, hashFunctions: HashFunctions): SymmetricState = {
val h: BinaryData = if (protocolName.length <= hashFunctions.hashLen)
protocolName ++ Seq.fill[Byte](hashFunctions.hashLen - protocolName.length)(0)
def apply(protocolName: ByteVector, cipherFunctions: CipherFunctions, hashFunctions: HashFunctions): SymmetricState = {
val h: ByteVector = if (protocolName.length <= hashFunctions.hashLen)
protocolName ++ ByteVector.fill(hashFunctions.hashLen - protocolName.length)(0)
else hashFunctions.hash(protocolName)
new SymmetricState(CipherState(cipherFunctions), ck = h, h = h, hashFunctions)
@ -306,7 +307,7 @@ object Noise {
val handshakePatternXK = HandshakePattern("XK", initiatorPreMessages = Nil, responderPreMessages = S :: Nil, messages = List(E :: ES :: Nil, E :: EE :: Nil, S :: SE :: Nil))
trait ByteStream {
def nextBytes(length: Int): BinaryData
def nextBytes(length: Int): ByteVector
}
object RandomBytes extends ByteStream {
@ -316,7 +317,7 @@ object Noise {
sealed trait HandshakeState
case class HandshakeStateWriter(messages: List[MessagePatterns], state: SymmetricState, s: KeyPair, e: KeyPair, rs: BinaryData, re: BinaryData, dh: DHFunctions, byteStream: ByteStream) extends HandshakeState with Logging {
case class HandshakeStateWriter(messages: List[MessagePatterns], state: SymmetricState, s: KeyPair, e: KeyPair, rs: ByteVector, re: ByteVector, dh: DHFunctions, byteStream: ByteStream) extends HandshakeState with Logging {
def toReader: HandshakeStateReader = HandshakeStateReader(messages, state, s, e, rs, re, dh, byteStream)
/**
@ -327,11 +328,11 @@ object Noise {
* When the handshake is over (i.e. there are no more handshake patterns to process) the last item will
* contain 2 cipherstates than can be used to encrypt/decrypt further communication
*/
def write(payload: BinaryData): (HandshakeStateReader, BinaryData, Option[(CipherState, CipherState, BinaryData)]) = {
def write(payload: ByteVector): (HandshakeStateReader, ByteVector, Option[(CipherState, CipherState, ByteVector)]) = {
require(!messages.isEmpty)
logger.debug(s"write($payload)")
val (writer1, buffer1) = messages.head.foldLeft(this -> BinaryData.empty) {
val (writer1, buffer1) = messages.head.foldLeft(this -> ByteVector.empty) {
case ((writer, buffer), pattern) => pattern match {
case E =>
val e1 = dh.generateKeyPair(byteStream.nextBytes(dh.dhLen))
@ -359,17 +360,17 @@ object Noise {
val buffer2 = buffer1 ++ ciphertext
val writer2 = writer1.copy(messages = messages.tail, state = state1)
logger.debug(s"h = 0x${state1.h}")
logger.debug(s"output: 0x${BinaryData(buffer2)}")
logger.debug(s"output: 0x$buffer2")
(writer2.toReader, buffer2, if (messages.tail.isEmpty) Some(writer2.state.split) else None)
}
}
object HandshakeStateWriter {
def apply(messages: List[MessagePatterns], state: SymmetricState, s: KeyPair, e: KeyPair, rs: BinaryData, re: BinaryData, dh: DHFunctions): HandshakeStateWriter = new HandshakeStateWriter(messages, state, s, e, rs, re, dh, RandomBytes)
def apply(messages: List[MessagePatterns], state: SymmetricState, s: KeyPair, e: KeyPair, rs: ByteVector, re: ByteVector, dh: DHFunctions): HandshakeStateWriter = new HandshakeStateWriter(messages, state, s, e, rs, re, dh, RandomBytes)
}
case class HandshakeStateReader(messages: List[MessagePatterns], state: SymmetricState, s: KeyPair, e: KeyPair, rs: BinaryData, re: BinaryData, dh: DHFunctions, byteStream: ByteStream) extends HandshakeState with Logging {
case class HandshakeStateReader(messages: List[MessagePatterns], state: SymmetricState, s: KeyPair, e: KeyPair, rs: ByteVector, re: ByteVector, dh: DHFunctions, byteStream: ByteStream) extends HandshakeState with Logging {
def toWriter: HandshakeStateWriter = HandshakeStateWriter(messages, state, s, e, rs, re, dh, byteStream)
/** *
@ -380,7 +381,7 @@ object Noise {
* next message. When the handshake is over (i.e. there are no more handshake patterns to process) the last item will
* contain 2 cipherstates than can be used to encrypt/decrypt further communication
*/
def read(message: BinaryData): (HandshakeStateWriter, BinaryData, Option[(CipherState, CipherState, BinaryData)]) = {
def read(message: ByteVector): (HandshakeStateWriter, ByteVector, Option[(CipherState, CipherState, ByteVector)]) = {
logger.debug(s"input: 0x$message")
val (reader1, buffer1) = messages.head.foldLeft(this -> message) {
case ((reader, buffer), pattern) => pattern match {
@ -419,18 +420,18 @@ object Noise {
}
object HandshakeStateReader {
def apply(messages: List[MessagePatterns], state: SymmetricState, s: KeyPair, e: KeyPair, rs: BinaryData, re: BinaryData, dh: DHFunctions): HandshakeStateReader = new HandshakeStateReader(messages, state, s, e, rs, re, dh, RandomBytes)
def apply(messages: List[MessagePatterns], state: SymmetricState, s: KeyPair, e: KeyPair, rs: ByteVector, re: ByteVector, dh: DHFunctions): HandshakeStateReader = new HandshakeStateReader(messages, state, s, e, rs, re, dh, RandomBytes)
}
object HandshakeState {
private def makeSymmetricState(handshakePattern: HandshakePattern, prologue: BinaryData, dh: DHFunctions, cipher: CipherFunctions, hash: HashFunctions): SymmetricState = {
private def makeSymmetricState(handshakePattern: HandshakePattern, prologue: ByteVector, dh: DHFunctions, cipher: CipherFunctions, hash: HashFunctions): SymmetricState = {
val name = "Noise_" + handshakePattern.name + "_" + dh.name + "_" + cipher.name + "_" + hash.name
val symmetricState = SymmetricState(name.getBytes("UTF-8"), cipher, hash)
val symmetricState = SymmetricState(ByteVector.view(name.getBytes("UTF-8")), cipher, hash)
symmetricState.mixHash(prologue)
}
def initializeWriter(handshakePattern: HandshakePattern, prologue: BinaryData, s: KeyPair, e: KeyPair, rs: BinaryData, re: BinaryData, dh: DHFunctions, cipher: CipherFunctions, hash: HashFunctions, byteStream: ByteStream = RandomBytes): HandshakeStateWriter = {
def initializeWriter(handshakePattern: HandshakePattern, prologue: ByteVector, s: KeyPair, e: KeyPair, rs: ByteVector, re: ByteVector, dh: DHFunctions, cipher: CipherFunctions, hash: HashFunctions, byteStream: ByteStream = RandomBytes): HandshakeStateWriter = {
val symmetricState = makeSymmetricState(handshakePattern, prologue, dh, cipher, hash)
val symmetricState1 = (handshakePattern.initiatorPreMessages).foldLeft(symmetricState) {
case (state, E) => state.mixHash(e.pub)
@ -445,7 +446,7 @@ object Noise {
HandshakeStateWriter(handshakePattern.messages, symmetricState2, s, e, rs, re, dh, byteStream)
}
def initializeReader(handshakePattern: HandshakePattern, prologue: BinaryData, s: KeyPair, e: KeyPair, rs: BinaryData, re: BinaryData, dh: DHFunctions, cipher: CipherFunctions, hash: HashFunctions, byteStream: ByteStream = RandomBytes): HandshakeStateReader = {
def initializeReader(handshakePattern: HandshakePattern, prologue: ByteVector, s: KeyPair, e: KeyPair, rs: ByteVector, re: ByteVector, dh: DHFunctions, cipher: CipherFunctions, hash: HashFunctions, byteStream: ByteStream = RandomBytes): HandshakeStateReader = {
val symmetricState = makeSymmetricState(handshakePattern, prologue, dh, cipher, hash)
val symmetricState1 = handshakePattern.initiatorPreMessages.foldLeft(symmetricState) {
case (state, E) => state.mixHash(re)

View file

@ -27,11 +27,9 @@ import scala.annotation.tailrec
*/
object ShaChain {
case class Node(value: BinaryData, height: Int, parent: Option[Node]) {
require(value.length == 32)
}
case class Node(value: ByteVector32, height: Int, parent: Option[Node])
def flip(in: BinaryData, index: Int): BinaryData = in.data.updated(index / 8, (in.data(index / 8) ^ (1 << index % 8)).toByte)
def flip(in: ByteVector32, index: Int): ByteVector32 = ByteVector32(in.update(index / 8, (in(index / 8) ^ (1 << index % 8)).toByte))
/**
*
@ -55,16 +53,16 @@ object ShaChain {
def derive(node: Node, directions: Long): Node = derive(node, moves(directions))
def shaChainFromSeed(hash: BinaryData, index: Long) = derive(Node(hash, 0, None), index).value
def shaChainFromSeed(hash: ByteVector32, index: Long) = derive(Node(hash, 0, None), index).value
type Index = Vector[Boolean]
val empty = ShaChain(Map.empty[Index, BinaryData])
val empty = ShaChain(Map.empty[Index, ByteVector32])
val init = empty
@tailrec
def addHash(receiver: ShaChain, hash: BinaryData, index: Index): ShaChain = {
def addHash(receiver: ShaChain, hash: ByteVector32, index: Index): ShaChain = {
index.last match {
case true => ShaChain(receiver.knownHashes + (index -> hash))
case false =>
@ -77,19 +75,19 @@ object ShaChain {
}
}
def addHash(receiver: ShaChain, hash: BinaryData, index: Long): ShaChain = {
def addHash(receiver: ShaChain, hash: ByteVector32, index: Long): ShaChain = {
receiver.lastIndex.map(value => require(index == value - 1L))
addHash(receiver, hash, moves(index)).copy(lastIndex = Some(index))
}
def getHash(receiver: ShaChain, index: Index): Option[BinaryData] = {
def getHash(receiver: ShaChain, index: Index): Option[ByteVector32] = {
receiver.knownHashes.keys.find(key => index.startsWith(key)).map(key => {
val root = Node(receiver.knownHashes(key), key.length, None)
derive(root, index.drop(key.length)).value
})
}
def getHash(receiver: ShaChain, index: Long): Option[BinaryData] = {
def getHash(receiver: ShaChain, index: Long): Option[ByteVector32] = {
receiver.lastIndex match {
case None => None
case Some(value) if value > index => None
@ -97,14 +95,14 @@ object ShaChain {
}
}
def iterator(chain: ShaChain): Iterator[BinaryData] = chain.lastIndex match {
def iterator(chain: ShaChain): Iterator[ByteVector32] = chain.lastIndex match {
case None => Iterator.empty
case Some(index) => new Iterator[BinaryData] {
case Some(index) => new Iterator[ByteVector32] {
var pos = index
override def hasNext: Boolean = pos >= index && pos <= 0xffffffffffffffffL
override def next(): BinaryData = {
override def next(): ByteVector32 = {
val value = chain.getHash(pos).get
pos = pos + 1
value
@ -118,12 +116,12 @@ object ShaChain {
import scodec.bits.BitVector
import scodec.codecs._
// codec for a single map entry (i.e. Vector[Boolean] -> BinaryData
val entryCodec = vectorOfN(uint16, bool) ~ LightningMessageCodecs.varsizebinarydata
// codec for a single map entry (i.e. Vector[Boolean] -> ByteVector
val entryCodec = vectorOfN(uint16, bool) ~ variableSizeBytes(uint16, LightningMessageCodecs.bytes32)
// codec for a Map[Vector[Boolean], BinaryData]: write all k ->v pairs using the codec defined above
val mapCodec: Codec[Map[Vector[Boolean], BinaryData]] = Codec[Map[Vector[Boolean], BinaryData]](
(m: Map[Vector[Boolean], BinaryData]) => vectorOfN(uint16, entryCodec).encode(m.toVector),
// 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]](
(m: Map[Vector[Boolean], ByteVector32]) => vectorOfN(uint16, entryCodec).encode(m.toVector),
(b: BitVector) => vectorOfN(uint16, entryCodec).decode(b).map(_.map(_.toMap))
)
@ -140,8 +138,8 @@ object ShaChain {
* @param lastIndex index of the last known hash. Hashes are supposed to be added in reverse order i.e.
* from 0xFFFFFFFFFFFFFFFF down to 0
*/
case class ShaChain(knownHashes: Map[Vector[Boolean], BinaryData], lastIndex: Option[Long] = None) {
def addHash(hash: BinaryData, index: Long): ShaChain = ShaChain.addHash(this, hash, index)
case class ShaChain(knownHashes: Map[Vector[Boolean], ByteVector32], lastIndex: Option[Long] = None) {
def addHash(hash: ByteVector32, index: Long): ShaChain = ShaChain.addHash(this, hash, index)
def getHash(index: Long) = ShaChain.getHash(this, index)

View file

@ -19,14 +19,14 @@ package fr.acinq.eclair.crypto
import java.io.{ByteArrayInputStream, ByteArrayOutputStream, InputStream, OutputStream}
import java.nio.ByteOrder
import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey}
import fr.acinq.bitcoin.{BinaryData, Crypto, Protocol}
import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey, Scalar}
import fr.acinq.bitcoin.{ByteVector32, Crypto, Protocol}
import fr.acinq.eclair.wire.{FailureMessage, FailureMessageCodecs}
import grizzled.slf4j.Logging
import org.spongycastle.crypto.digests.SHA256Digest
import org.spongycastle.crypto.macs.HMac
import org.spongycastle.crypto.params.KeyParameter
import scodec.bits.BitVector
import scodec.bits.{BitVector, ByteVector}
import scala.annotation.tailrec
import scala.util.{Failure, Success, Try}
@ -51,39 +51,34 @@ object Sphinx extends Logging {
val PacketLength = 1 + 33 + MacLength + MaxHops * (PayloadLength + MacLength)
// last packet (all zeroes except for the version byte)
val LAST_PACKET = Packet(Version, zeroes(33), zeroes(MacLength), zeroes(MaxHops * (PayloadLength + MacLength)))
val LAST_PACKET = Packet(Version, ByteVector.fill(33)(0), ByteVector32.Zeroes, ByteVector.fill(MaxHops * (PayloadLength + MacLength))(0))
def hmac256(key: Seq[Byte], message: Seq[Byte]): Seq[Byte] = {
def hmac256(key: ByteVector, message: ByteVector): ByteVector32 = {
val mac = new HMac(new SHA256Digest())
mac.init(new KeyParameter(key.toArray))
mac.update(message.toArray, 0, message.length)
mac.update(message.toArray, 0, message.length.toInt)
val output = new Array[Byte](32)
mac.doFinal(output, 0)
output
ByteVector32(ByteVector.view(output))
}
def mac(key: BinaryData, message: BinaryData): BinaryData = hmac256(key, message).take(MacLength)
def mac(key: ByteVector, message: ByteVector): ByteVector32 = hmac256(key, message)
def xor(a: Seq[Byte], b: Seq[Byte]): Seq[Byte] = a.zip(b).map { case (x, y) => ((x ^ y) & 0xff).toByte }
def generateKey(keyType: ByteVector, secret: ByteVector32): ByteVector32 = hmac256(keyType, secret)
def generateKey(keyType: BinaryData, secret: BinaryData): BinaryData = {
require(secret.length == 32, "secret must be 32 bytes")
hmac256(keyType, secret)
}
def generateKey(keyType: String, secret: ByteVector32): ByteVector32 = generateKey(ByteVector.view(keyType.getBytes("UTF-8")), secret)
def generateKey(keyType: String, secret: BinaryData): BinaryData = generateKey(keyType.getBytes("UTF-8"), secret)
def zeroes(length: Int): ByteVector = ByteVector.fill(length)(0)
def zeroes(length: Int): BinaryData = Seq.fill[Byte](length)(0)
def generateStream(key: ByteVector, length: Int): ByteVector = ChaCha20.encrypt(zeroes(length), key, zeroes(12))
def generateStream(key: BinaryData, length: Int): BinaryData = ChaCha20Legacy.encrypt(zeroes(length), key, zeroes(8))
def computeSharedSecret(pub: PublicKey, secret: PrivateKey): ByteVector32 = Crypto.sha256(ByteVector.view(pub.multiply(secret).normalize().getEncoded(true)))
def computeSharedSecret(pub: PublicKey, secret: PrivateKey): BinaryData = Crypto.sha256(pub.multiply(secret).normalize().getEncoded(true))
def computeblindingFactor(pub: PublicKey, secret: ByteVector): ByteVector32 = Crypto.sha256(pub.toBin ++ secret)
def computeblindingFactor(pub: PublicKey, secret: BinaryData): BinaryData = Crypto.sha256(pub.toBin ++ secret)
def blind(pub: PublicKey, blindingFactor: ByteVector32): PublicKey = PublicKey(pub.multiply(Scalar(blindingFactor)).normalize(), compressed = true)
def blind(pub: PublicKey, blindingFactor: BinaryData): PublicKey = PublicKey(pub.multiply(blindingFactor).normalize(), compressed = true)
def blind(pub: PublicKey, blindingFactors: Seq[BinaryData]): PublicKey = blindingFactors.foldLeft(pub)(blind)
def blind(pub: PublicKey, blindingFactors: Seq[ByteVector32]): PublicKey = blindingFactors.foldLeft(pub)(blind)
/**
* computes the ephemeral public keys and shared secrets for all nodes on the route.
@ -92,15 +87,15 @@ object Sphinx extends Logging {
* @param publicKeys public keys of each node on the route
* @return a tuple (ephemeral public keys, shared secrets)
*/
def computeEphemeralPublicKeysAndSharedSecrets(sessionKey: PrivateKey, publicKeys: Seq[PublicKey]): (Seq[PublicKey], Seq[BinaryData]) = {
val ephemeralPublicKey0 = blind(PublicKey(Crypto.curve.getG, compressed = true), sessionKey.value)
val secret0 = computeSharedSecret(publicKeys(0), sessionKey)
def computeEphemeralPublicKeysAndSharedSecrets(sessionKey: PrivateKey, publicKeys: Seq[PublicKey]): (Seq[PublicKey], Seq[ByteVector32]) = {
val ephemeralPublicKey0 = blind(PublicKey(Crypto.curve.getG, compressed = true), sessionKey.value.toBin)
val secret0 = computeSharedSecret(publicKeys.head, sessionKey)
val blindingFactor0 = computeblindingFactor(ephemeralPublicKey0, secret0)
computeEphemeralPublicKeysAndSharedSecrets(sessionKey, publicKeys.tail, Seq(ephemeralPublicKey0), Seq(blindingFactor0), Seq(secret0))
}
@tailrec
def computeEphemeralPublicKeysAndSharedSecrets(sessionKey: PrivateKey, publicKeys: Seq[PublicKey], ephemeralPublicKeys: Seq[PublicKey], blindingFactors: Seq[BinaryData], sharedSecrets: Seq[BinaryData]): (Seq[PublicKey], Seq[BinaryData]) = {
def computeEphemeralPublicKeysAndSharedSecrets(sessionKey: PrivateKey, publicKeys: Seq[PublicKey], ephemeralPublicKeys: Seq[PublicKey], blindingFactors: Seq[ByteVector32], sharedSecrets: Seq[ByteVector32]): (Seq[PublicKey], Seq[ByteVector32]) = {
if (publicKeys.isEmpty)
(ephemeralPublicKeys, sharedSecrets)
else {
@ -111,23 +106,23 @@ object Sphinx extends Logging {
}
}
def generateFiller(keyType: String, sharedSecrets: Seq[BinaryData], hopSize: Int, maxNumberOfHops: Int = MaxHops): BinaryData = {
sharedSecrets.foldLeft(Seq.empty[Byte])((padding, secret) => {
def generateFiller(keyType: String, sharedSecrets: Seq[ByteVector32], hopSize: Int, maxNumberOfHops: Int = MaxHops): ByteVector = {
sharedSecrets.foldLeft(ByteVector.empty)((padding, secret) => {
val key = generateKey(keyType, secret)
val padding1 = padding ++ zeroes(hopSize)
val padding1 = padding ++ ByteVector.fill(hopSize)(0)
val stream = generateStream(key, hopSize * (maxNumberOfHops + 1)).takeRight(padding1.length)
xor(padding1, stream)
padding1.xor(stream)
})
}
case class Packet(version: Int, publicKey: BinaryData, hmac: BinaryData, routingInfo: BinaryData) {
case class Packet(version: Int, publicKey: ByteVector, hmac: ByteVector32, routingInfo: ByteVector) {
require(publicKey.length == 33, "onion packet public key length should be 33")
require(hmac.length == MacLength, s"onion packet hmac length should be $MacLength")
require(routingInfo.length == MaxHops * (PayloadLength + MacLength), s"onion packet routing info length should be ${MaxHops * (PayloadLength + MacLength)}")
def isLastPacket: Boolean = hmac == zeroes(MacLength)
def isLastPacket: Boolean = hmac == ByteVector32.Zeroes
def serialize: BinaryData = Packet.write(this)
def serialize: ByteVector = Packet.write(this)
}
object Packet {
@ -139,26 +134,26 @@ object Sphinx extends Logging {
in.read(routingInfo)
val hmac = new Array[Byte](MacLength)
in.read(hmac)
Packet(version, publicKey, hmac, routingInfo)
Packet(version, ByteVector.view(publicKey), ByteVector32(ByteVector.view(hmac)), ByteVector.view(routingInfo))
}
def read(in: BinaryData): Packet = read(new ByteArrayInputStream(in))
def read(in: ByteVector): Packet = read(new ByteArrayInputStream(in.toArray))
def write(packet: Packet, out: OutputStream): OutputStream = {
out.write(packet.version)
out.write(packet.publicKey)
out.write(packet.routingInfo)
out.write(packet.hmac)
out.write(packet.publicKey.toArray)
out.write(packet.routingInfo.toArray)
out.write(packet.hmac.toArray)
out
}
def write(packet: Packet): BinaryData = {
def write(packet: Packet): ByteVector = {
val out = new ByteArrayOutputStream(PacketLength)
write(packet, out)
out.toByteArray
ByteVector.view(out.toByteArray)
}
def isLastPacket(packet: BinaryData): Boolean = Packet.read(packet).hmac == zeroes(MacLength)
def isLastPacket(packet: ByteVector): Boolean = Packet.read(packet).hmac == ByteVector32.Zeroes
}
/**
@ -167,7 +162,7 @@ object Sphinx extends Logging {
* @param nextPacket packet for the next node
* @param sharedSecret shared secret for the sending node, which we will need to return error messages
*/
case class ParsedPacket(payload: BinaryData, nextPacket: Packet, sharedSecret: BinaryData)
case class ParsedPacket(payload: ByteVector, nextPacket: Packet, sharedSecret: ByteVector32)
/**
*
@ -180,18 +175,18 @@ object Sphinx extends Logging {
* - shared secret is the secret we share with the node that sent the packet. We need it to propagate failure
* messages upstream.
*/
def parsePacket(privateKey: PrivateKey, associatedData: BinaryData, rawPacket: BinaryData): Try[ParsedPacket] = Try {
def parsePacket(privateKey: PrivateKey, associatedData: ByteVector, rawPacket: ByteVector): Try[ParsedPacket] = Try {
require(rawPacket.length == PacketLength, s"onion packet length is ${rawPacket.length}, it should be ${PacketLength}")
val packet = Packet.read(rawPacket)
val sharedSecret = computeSharedSecret(PublicKey(packet.publicKey), privateKey)
val mu = generateKey("mu", sharedSecret)
val check: BinaryData = mac(mu, packet.routingInfo ++ associatedData)
val check = mac(mu, packet.routingInfo ++ associatedData)
require(check == packet.hmac, "invalid header mac")
val rho = generateKey("rho", sharedSecret)
val bin = xor(packet.routingInfo ++ zeroes(PayloadLength + MacLength), generateStream(rho, PayloadLength + MacLength + MaxHops * (PayloadLength + MacLength)))
val bin = (packet.routingInfo ++ ByteVector.fill(PayloadLength + MacLength)(0)) xor generateStream(rho, PayloadLength + MacLength + MaxHops * (PayloadLength + MacLength))
val payload = bin.take(PayloadLength)
val hmac = bin.slice(PayloadLength, PayloadLength + MacLength)
val hmac = ByteVector32(bin.slice(PayloadLength, PayloadLength + MacLength))
val nextRouteInfo = bin.drop(PayloadLength + MacLength)
val nextPubKey = blind(PublicKey(packet.publicKey), computeblindingFactor(PublicKey(packet.publicKey), sharedSecret))
@ -200,7 +195,7 @@ object Sphinx extends Logging {
}
@tailrec
private def extractSharedSecrets(packet: BinaryData, privateKey: PrivateKey, associatedData: BinaryData, acc: Seq[BinaryData] = Nil): Try[Seq[BinaryData]] = {
private def extractSharedSecrets(packet: ByteVector, privateKey: PrivateKey, associatedData: ByteVector32, acc: Seq[ByteVector32] = Nil): Try[Seq[ByteVector32]] = {
parsePacket(privateKey, associatedData, packet) match {
case Success(ParsedPacket(_, nextPacket, sharedSecret)) if nextPacket.isLastPacket => Success(acc :+ sharedSecret)
case Success(ParsedPacket(_, nextPacket, sharedSecret)) => extractSharedSecrets(nextPacket.serialize, privateKey, associatedData, acc :+ sharedSecret)
@ -223,16 +218,16 @@ object Sphinx extends Logging {
* @param routingInfoFiller optional routing info filler, needed only when you're constructing the last packet
* @return the next packet
*/
private def makeNextPacket(payload: BinaryData, associatedData: BinaryData, ephemeralPublicKey: BinaryData, sharedSecret: BinaryData, packet: Packet, routingInfoFiller: BinaryData = BinaryData.empty): Packet = {
private def makeNextPacket(payload: ByteVector, associatedData: ByteVector32, ephemeralPublicKey: ByteVector, sharedSecret: ByteVector32, packet: Packet, routingInfoFiller: ByteVector = ByteVector.empty): Packet = {
require(payload.length == PayloadLength)
val nextRoutingInfo = {
val routingInfo1 = payload ++ packet.hmac ++ packet.routingInfo.dropRight(PayloadLength + MacLength)
val routingInfo2 = xor(routingInfo1, generateStream(generateKey("rho", sharedSecret), MaxHops * (PayloadLength + MacLength)))
val routingInfo2 = routingInfo1 xor generateStream(generateKey("rho", sharedSecret), MaxHops * (PayloadLength + MacLength))
routingInfo2.dropRight(routingInfoFiller.length) ++ routingInfoFiller
}
val nextHmac: BinaryData = mac(generateKey("mu", sharedSecret), nextRoutingInfo ++ associatedData)
val nextHmac = mac(generateKey("mu", sharedSecret), nextRoutingInfo ++ associatedData)
val nextPacket = Packet(Version, ephemeralPublicKey, nextHmac, nextRoutingInfo)
nextPacket
}
@ -244,7 +239,7 @@ object Sphinx extends Logging {
* @param sharedSecrets shared secrets (one per node in the route). Known (and needed) only if you're creating the
* packet. Empty if you're just forwarding the packet to the next node
*/
case class PacketAndSecrets(packet: Packet, sharedSecrets: Seq[(BinaryData, PublicKey)])
case class PacketAndSecrets(packet: Packet, sharedSecrets: Seq[(ByteVector32, PublicKey)])
/**
* A properly decoded error from a node in the route
@ -264,14 +259,14 @@ object Sphinx extends Logging {
* @return an OnionPacket(onion packet, shared secrets). the onion packet can be sent to the first node in the list, and the
* shared secrets (one per node) can be used to parse returned error messages if needed
*/
def makePacket(sessionKey: PrivateKey, publicKeys: Seq[PublicKey], payloads: Seq[BinaryData], associatedData: BinaryData): PacketAndSecrets = {
def makePacket(sessionKey: PrivateKey, publicKeys: Seq[PublicKey], payloads: Seq[ByteVector], associatedData: ByteVector32): PacketAndSecrets = {
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)
@tailrec
def loop(hoppayloads: Seq[BinaryData], ephkeys: Seq[PublicKey], sharedSecrets: Seq[BinaryData], packet: Packet): Packet = {
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)
loop(hoppayloads.dropRight(1), ephkeys.dropRight(1), sharedSecrets.dropRight(1), nextPacket)
@ -299,15 +294,15 @@ object Sphinx extends Logging {
* @param failure failure message
* @return an error packet that can be sent to the destination node
*/
def createErrorPacket(sharedSecret: BinaryData, failure: FailureMessage): BinaryData = {
val message: BinaryData = FailureMessageCodecs.failureMessageCodec.encode(failure).require.toByteArray
def createErrorPacket(sharedSecret: ByteVector32, failure: FailureMessage): ByteVector = {
val message: ByteVector = FailureMessageCodecs.failureMessageCodec.encode(failure).require.toByteVector
require(message.length <= MaxErrorPayloadLength, s"error message length is ${message.length}, it must be less than $MaxErrorPayloadLength")
val um = Sphinx.generateKey("um", sharedSecret)
val padlen = MaxErrorPayloadLength - message.length
val payload = Protocol.writeUInt16(message.length, ByteOrder.BIG_ENDIAN) ++ message ++ Protocol.writeUInt16(padlen, ByteOrder.BIG_ENDIAN) ++ Sphinx.zeroes(padlen)
val payload = Protocol.writeUInt16(message.length.toInt, ByteOrder.BIG_ENDIAN) ++ message ++ Protocol.writeUInt16(padlen.toInt, ByteOrder.BIG_ENDIAN) ++ ByteVector.fill(padlen.toInt)(0)
logger.debug(s"um key: $um")
logger.debug(s"error payload: ${BinaryData(payload)}")
logger.debug(s"raw error packet: ${BinaryData(Sphinx.mac(um, payload) ++ payload)}")
logger.debug(s"error payload: ${payload.toHex}")
logger.debug(s"raw error packet: ${(Sphinx.mac(um, payload) ++ payload).toHex}")
forwardErrorPacket(Sphinx.mac(um, payload) ++ payload, sharedSecret)
}
@ -316,10 +311,10 @@ object Sphinx extends Logging {
* @param packet error packet
* @return the failure message that is embedded in the error packet
*/
private def extractFailureMessage(packet: BinaryData): FailureMessage = {
private def extractFailureMessage(packet: ByteVector): FailureMessage = {
require(packet.length == ErrorPacketLength, s"invalid error packet length ${packet.length}, must be $ErrorPacketLength")
val (mac, payload) = packet.splitAt(Sphinx.MacLength)
val len = Protocol.uint16(payload, ByteOrder.BIG_ENDIAN)
val len = Protocol.uint16(payload.toArray, ByteOrder.BIG_ENDIAN)
require((len >= 0) && (len <= MaxErrorPayloadLength), s"message length must be less than $MaxErrorPayloadLength")
FailureMessageCodecs.failureMessageCodec.decode(BitVector(payload.drop(2).take(len))).require.value
}
@ -330,13 +325,13 @@ object Sphinx extends Logging {
* @param sharedSecret destination node's shared secret
* @return an obfuscated error packet that can be sent to the destination node
*/
def forwardErrorPacket(packet: BinaryData, sharedSecret: BinaryData): BinaryData = {
def forwardErrorPacket(packet: ByteVector, sharedSecret: ByteVector32): ByteVector = {
require(packet.length == ErrorPacketLength, s"invalid error packet length ${packet.length}, must be $ErrorPacketLength")
val key = generateKey("ammag", sharedSecret)
val stream = generateStream(key, ErrorPacketLength)
logger.debug(s"ammag key: $key")
logger.debug(s"error stream: $stream")
Sphinx.xor(packet, stream)
packet xor stream
}
/**
@ -345,10 +340,10 @@ object Sphinx extends Logging {
* @param packet error packet
* @return true if the packet's mac is valid, which means that it has been properly de-obfuscated
*/
private def checkMac(sharedSecret: BinaryData, packet: BinaryData): Boolean = {
private def checkMac(sharedSecret: ByteVector32, packet: ByteVector): Boolean = {
val (mac, payload) = packet.splitAt(Sphinx.MacLength)
val um = Sphinx.generateKey("um", sharedSecret)
BinaryData(mac) == Sphinx.mac(um, payload)
ByteVector32(mac) == Sphinx.mac(um, payload)
}
/**
@ -359,11 +354,11 @@ object Sphinx extends Logging {
* @param sharedSecrets nodes shared secrets
* @return Success(secret, failure message) if the origin of the packet could be identified and the packet de-obfuscated, Failure otherwise
*/
def parseErrorPacket(packet: BinaryData, sharedSecrets: Seq[(BinaryData, PublicKey)]): Try[ErrorPacket] = Try {
def parseErrorPacket(packet: ByteVector, sharedSecrets: Seq[(ByteVector32, PublicKey)]): Try[ErrorPacket] = Try {
require(packet.length == ErrorPacketLength, s"invalid error packet length ${packet.length}, must be $ErrorPacketLength")
@tailrec
def loop(packet: BinaryData, sharedSecrets: Seq[(BinaryData, PublicKey)]): ErrorPacket = sharedSecrets match {
def loop(packet: ByteVector, sharedSecrets: Seq[(ByteVector32, PublicKey)]): ErrorPacket = sharedSecrets match {
case Nil => throw new RuntimeException(s"couldn't parse error packet=$packet with sharedSecrets=$sharedSecrets")
case (secret, pubkey) :: tail =>
val packet1 = forwardErrorPacket(packet, secret)

View file

@ -24,12 +24,11 @@ import akka.event._
import akka.io.Tcp
import akka.util.ByteString
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.bitcoin.{BinaryData, Protocol}
import fr.acinq.eclair.{Diagnostics, FSMDiagnosticActorLogging, Logs}
import fr.acinq.bitcoin.Protocol
import fr.acinq.eclair.crypto.Noise._
import fr.acinq.eclair.wire._
import fr.acinq.eclair.wire.{ChannelAnnouncement, ChannelUpdate, NodeAnnouncement}
import scodec.bits.BitVector
import fr.acinq.eclair.wire.{ChannelAnnouncement, ChannelUpdate, NodeAnnouncement, _}
import fr.acinq.eclair.{Diagnostics, FSMDiagnosticActorLogging, Logs}
import scodec.bits.ByteVector
import scodec.{Attempt, Codec, DecodeResult}
import scala.annotation.tailrec
@ -50,7 +49,7 @@ import scala.util.{Failure, Success, Try}
* @param rs remote node static public key (which must be known before we initiate communication)
* @param connection actor that represents the other node's
*/
class TransportHandler[T: ClassTag](keyPair: KeyPair, rs: Option[BinaryData], connection: ActorRef, codec: Codec[T]) extends Actor with FSMDiagnosticActorLogging[TransportHandler.State, TransportHandler.Data] {
class TransportHandler[T: ClassTag](keyPair: KeyPair, rs: Option[ByteVector], connection: ActorRef, codec: Codec[T]) extends Actor with FSMDiagnosticActorLogging[TransportHandler.State, TransportHandler.Data] {
// will hold the peer's public key once it is available (we don't know it right away in case of an incoming connection)
var remoteNodeId_opt: Option[PublicKey] = rs.map(PublicKey(_))
@ -81,7 +80,7 @@ class TransportHandler[T: ClassTag](keyPair: KeyPair, rs: Option[BinaryData], co
connection ! Tcp.Register(self)
connection ! Tcp.ResumeReading
def buf(message: BinaryData): ByteString = ByteString.fromArray(message)
def buf(message: ByteVector): ByteString = ByteString.fromArray(message.toArray)
// it means we initiate the dialog
val isWriter = rs.isDefined
@ -90,7 +89,7 @@ class TransportHandler[T: ClassTag](keyPair: KeyPair, rs: Option[BinaryData], co
val reader = if (isWriter) {
val state = makeWriter(keyPair, rs.get)
val (state1, message, None) = state.write(BinaryData.empty)
val (state1, message, None) = state.write(ByteVector.empty)
log.debug(s"sending prefix + $message")
connection ! Tcp.Write(buf(TransportHandler.prefix +: message))
state1
@ -98,9 +97,9 @@ class TransportHandler[T: ClassTag](keyPair: KeyPair, rs: Option[BinaryData], co
makeReader(keyPair)
}
def sendToListener(listener: ActorRef, plaintextMessages: Seq[BinaryData]): Map[T, Int] = {
def sendToListener(listener: ActorRef, plaintextMessages: Seq[ByteVector]): Map[T, Int] = {
var m: Map[T, Int] = Map()
plaintextMessages.foreach(plaintext => Try(codec.decode(BitVector(plaintext.data))) match {
plaintextMessages.foreach(plaintext => Try(codec.decode(plaintext.toBitVector)) match {
case Success(Attempt.Successful(DecodeResult(message, _))) =>
diag(message, "IN")
listener ! message
@ -118,15 +117,15 @@ class TransportHandler[T: ClassTag](keyPair: KeyPair, rs: Option[BinaryData], co
when(Handshake) {
case Event(Tcp.Received(data), HandshakeData(reader, buffer)) =>
connection ! Tcp.ResumeReading
log.debug("received {}", BinaryData(data))
log.debug("received {}", ByteVector(data))
val buffer1 = buffer ++ data
if (buffer1.length < expectedLength(reader))
stay using HandshakeData(reader, buffer1)
else {
require(buffer1.head == TransportHandler.prefix, s"invalid transport prefix first64=${BinaryData(buffer1.take(64))}")
require(buffer1.head == TransportHandler.prefix, s"invalid transport prefix first64=${ByteVector(buffer1.take(64))}")
val (payload, remainder) = buffer1.tail.splitAt(expectedLength(reader) - 1)
reader.read(payload) match {
reader.read(ByteVector.view(payload.asByteBuffer)) match {
case (writer, _, Some((dec, enc, ck))) =>
val remoteNodeId = PublicKey(writer.rs)
remoteNodeId_opt = Some(remoteNodeId)
@ -135,7 +134,7 @@ class TransportHandler[T: ClassTag](keyPair: KeyPair, rs: Option[BinaryData], co
goto(WaitingForListener) using nextStateData
case (writer, _, None) => {
writer.write(BinaryData.empty) match {
writer.write(ByteVector.empty) match {
case (reader1, message, None) => {
// we're still in the middle of the handshake process and the other end must first received our next
// message before they can reply
@ -166,11 +165,11 @@ class TransportHandler[T: ClassTag](keyPair: KeyPair, rs: Option[BinaryData], co
val (dec1, plaintextMessages) = dec.decrypt()
if (plaintextMessages.isEmpty) {
connection ! Tcp.ResumeReading
goto(Normal) using NormalData(d.encryptor, dec1, listener, sendBuffer = SendBuffer(Queue.empty, Queue.empty), unackedReceived = Map.empty[T, Int], unackedSent = None)
goto(Normal) using NormalData(d.encryptor, dec1, listener, sendBuffer = SendBuffer(Queue.empty[T], Queue.empty[T]), unackedReceived = Map.empty[T, Int], unackedSent = None)
} else {
log.debug(s"read ${plaintextMessages.size} messages, waiting for readacks")
val unackedReceived = sendToListener(listener, plaintextMessages)
goto(Normal) using NormalData(d.encryptor, dec1, listener, sendBuffer = SendBuffer(Queue.empty, Queue.empty), unackedReceived, unackedSent = None)
goto(Normal) using NormalData(d.encryptor, dec1, listener, sendBuffer = SendBuffer(Queue.empty[T], Queue.empty[T]), unackedReceived, unackedSent = None)
}
}
@ -215,7 +214,7 @@ class TransportHandler[T: ClassTag](keyPair: KeyPair, rs: Option[BinaryData], co
stay using d.copy(sendBuffer = sendBuffer1)
} else {
diag(t, "OUT")
val blob = codec.encode(t).require.toByteArray
val blob = codec.encode(t).require.toByteVector
val (enc1, ciphertext) = d.encryptor.encrypt(blob)
connection ! Tcp.Write(buf(ciphertext), WriteAck)
stay using d.copy(encryptor = enc1, unackedSent = Some(t))
@ -224,7 +223,7 @@ class TransportHandler[T: ClassTag](keyPair: KeyPair, rs: Option[BinaryData], co
case Event(WriteAck, d: NormalData[T]) =>
def send(t: T) = {
diag(t, "OUT")
val blob = codec.encode(t).require.toByteArray
val blob = codec.encode(t).require.toByteVector
val (enc1, ciphertext) = d.encryptor.encrypt(blob)
connection ! Tcp.Write(buf(ciphertext), WriteAck)
enc1
@ -273,7 +272,7 @@ class TransportHandler[T: ClassTag](keyPair: KeyPair, rs: Option[BinaryData], co
object TransportHandler {
def props[T: ClassTag](keyPair: KeyPair, rs: Option[BinaryData], connection: ActorRef, codec: Codec[T]): Props = Props(new TransportHandler(keyPair, rs, connection, codec))
def props[T: ClassTag](keyPair: KeyPair, rs: Option[ByteVector], connection: ActorRef, codec: Codec[T]): Props = Props(new TransportHandler(keyPair, rs, connection, codec))
val MAX_BUFFERED = 100000L
@ -281,7 +280,7 @@ object TransportHandler {
// this prefix is prepended to all Noise messages sent during the handshake phase
val prefix: Byte = 0x00
val prologue = "lightning".getBytes("UTF-8")
val prologue = ByteVector.view("lightning".getBytes("UTF-8"))
/**
* See BOLT #8: during the handshake phase we are expecting 3 messages of 50, 50 and 66 bytes (including the prefix)
@ -294,14 +293,14 @@ object TransportHandler {
case 1 => 66
}
def makeWriter(localStatic: KeyPair, remoteStatic: BinaryData) = Noise.HandshakeState.initializeWriter(
def makeWriter(localStatic: KeyPair, remoteStatic: ByteVector) = Noise.HandshakeState.initializeWriter(
Noise.handshakePatternXK, prologue,
localStatic, KeyPair(BinaryData.empty, BinaryData.empty), remoteStatic, BinaryData.empty,
localStatic, KeyPair(ByteVector.empty, ByteVector.empty), remoteStatic, ByteVector.empty,
Noise.Secp256k1DHFunctions, Noise.Chacha20Poly1305CipherFunctions, Noise.SHA256HashFunctions)
def makeReader(localStatic: KeyPair) = Noise.HandshakeState.initializeReader(
Noise.handshakePatternXK, prologue,
localStatic, KeyPair(BinaryData.empty, BinaryData.empty), BinaryData.empty, BinaryData.empty,
localStatic, KeyPair(ByteVector.empty, ByteVector.empty), ByteVector.empty, ByteVector.empty,
Noise.Secp256k1DHFunctions, Noise.Chacha20Poly1305CipherFunctions, Noise.SHA256HashFunctions)
/**
@ -310,12 +309,12 @@ object TransportHandler {
* @param cs cipher state
* @param ck chaining key
*/
case class ExtendedCipherState(cs: CipherState, ck: BinaryData) extends CipherState {
case class ExtendedCipherState(cs: CipherState, ck: ByteVector) extends CipherState {
override def cipher: CipherFunctions = cs.cipher
override def hasKey: Boolean = cs.hasKey
override def encryptWithAd(ad: BinaryData, plaintext: BinaryData): (CipherState, BinaryData) = {
override def encryptWithAd(ad: ByteVector, plaintext: ByteVector): (CipherState, ByteVector) = {
cs match {
case UninitializedCipherState(_) => (this, plaintext)
case InitializedCipherState(k, n, _) if n == 999 => {
@ -330,7 +329,7 @@ object TransportHandler {
}
}
override def decryptWithAd(ad: BinaryData, ciphertext: BinaryData): (CipherState, BinaryData) = {
override def decryptWithAd(ad: ByteVector, ciphertext: ByteVector): (CipherState, ByteVector) = {
cs match {
case UninitializedCipherState(_) => (this, ciphertext)
case InitializedCipherState(k, n, _) if n == 999 => {
@ -348,18 +347,18 @@ object TransportHandler {
case class Decryptor(state: CipherState, ciphertextLength: Option[Int], buffer: ByteString) {
@tailrec
final def decrypt(acc: Seq[BinaryData] = Vector()): (Decryptor, Seq[BinaryData]) = {
final def decrypt(acc: Seq[ByteVector] = Vector()): (Decryptor, Seq[ByteVector]) = {
(ciphertextLength, buffer.length) match {
case (None, length) if length < 18 => (this, acc)
case (None, _) =>
val (ciphertext, remainder) = buffer.splitAt(18)
val (dec1, plaintext) = state.decryptWithAd(BinaryData.empty, ciphertext)
val length = Protocol.uint16(plaintext, ByteOrder.BIG_ENDIAN)
val (dec1, plaintext) = state.decryptWithAd(ByteVector.empty, ByteVector.view(ciphertext.asByteBuffer))
val length = Protocol.uint16(plaintext.toArray, ByteOrder.BIG_ENDIAN)
Decryptor(dec1, ciphertextLength = Some(length), buffer = remainder).decrypt(acc)
case (Some(expectedLength), length) if length < expectedLength + 16 => (Decryptor(state, ciphertextLength, buffer), acc)
case (Some(expectedLength), _) =>
val (ciphertext, remainder) = buffer.splitAt(expectedLength + 16)
val (dec1, plaintext) = state.decryptWithAd(BinaryData.empty, ciphertext)
val (dec1, plaintext) = state.decryptWithAd(ByteVector.empty, ByteVector.view(ciphertext.asByteBuffer))
Decryptor(dec1, ciphertextLength = None, buffer = remainder).decrypt(acc :+ plaintext)
}
}
@ -387,9 +386,9 @@ object TransportHandler {
* @param plaintext plaintext
* @return a (cipherstate, ciphertext) tuple where ciphertext is encrypted according to BOLT #8
*/
def encrypt(plaintext: BinaryData): (Encryptor, BinaryData) = {
val (state1, ciphertext1) = state.encryptWithAd(BinaryData.empty, Protocol.writeUInt16(plaintext.length, ByteOrder.BIG_ENDIAN))
val (state2, ciphertext2) = state1.encryptWithAd(BinaryData.empty, plaintext)
def encrypt(plaintext: ByteVector): (Encryptor, ByteVector) = {
val (state1, ciphertext1) = state.encryptWithAd(ByteVector.empty, Protocol.writeUInt16(plaintext.length.toInt, ByteOrder.BIG_ENDIAN))
val (state2, ciphertext2) = state1.encryptWithAd(ByteVector.empty, plaintext)
(Encryptor(state2), ciphertext1 ++ ciphertext2)
}
}

View file

@ -16,7 +16,7 @@
package fr.acinq.eclair.db
import fr.acinq.bitcoin.BinaryData
import fr.acinq.bitcoin.ByteVector32
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.eclair.channel._
import fr.acinq.eclair.payment.{PaymentReceived, PaymentRelayed, PaymentSent}
@ -49,8 +49,8 @@ trait AuditDb {
}
case class ChannelLifecycleEvent(channelId: BinaryData, remoteNodeId: PublicKey, capacitySat: Long, isFunder: Boolean, isPrivate: Boolean, event: String)
case class ChannelLifecycleEvent(channelId: ByteVector32, remoteNodeId: PublicKey, capacitySat: Long, isFunder: Boolean, isPrivate: Boolean, event: String)
case class NetworkFee(remoteNodeId: PublicKey, channelId: BinaryData, txId: BinaryData, feeSat: Long, txType: String, timestamp: Long)
case class NetworkFee(remoteNodeId: PublicKey, channelId: ByteVector32, txId: ByteVector32, feeSat: Long, txType: String, timestamp: Long)
case class Stats(channelId: BinaryData, avgPaymentAmountSatoshi: Long, paymentCount: Int, relayFeeSatoshi: Long, networkFeeSatoshi: Long)
case class Stats(channelId: ByteVector32, avgPaymentAmountSatoshi: Long, paymentCount: Int, relayFeeSatoshi: Long, networkFeeSatoshi: Long)

View file

@ -16,20 +16,20 @@
package fr.acinq.eclair.db
import fr.acinq.bitcoin.BinaryData
import fr.acinq.bitcoin.ByteVector32
import fr.acinq.eclair.channel.HasCommitments
trait ChannelsDb {
def addOrUpdateChannel(state: HasCommitments)
def removeChannel(channelId: BinaryData)
def removeChannel(channelId: ByteVector32)
def listChannels(): Seq[HasCommitments]
def addOrUpdateHtlcInfo(channelId: BinaryData, commitmentNumber: Long, paymentHash: BinaryData, cltvExpiry: Long)
def addOrUpdateHtlcInfo(channelId: ByteVector32, commitmentNumber: Long, paymentHash: ByteVector32, cltvExpiry: Long)
def listHtlcInfos(channelId: BinaryData, commitmentNumber: Long): Seq[(BinaryData, Long)]
def listHtlcInfos(channelId: ByteVector32, commitmentNumber: Long): Seq[(ByteVector32, Long)]
def close(): Unit

View file

@ -16,8 +16,8 @@
package fr.acinq.eclair.db
import fr.acinq.bitcoin.{BinaryData, Satoshi}
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.bitcoin.{ByteVector32, Satoshi}
import fr.acinq.eclair.ShortChannelId
import fr.acinq.eclair.wire.{ChannelAnnouncement, ChannelUpdate, NodeAnnouncement}
@ -31,7 +31,7 @@ trait NetworkDb {
def listNodes(): Seq[NodeAnnouncement]
def addChannel(c: ChannelAnnouncement, txid: BinaryData, capacity: Satoshi)
def addChannel(c: ChannelAnnouncement, txid: ByteVector32, capacity: Satoshi)
def removeChannel(shortChannelId: ShortChannelId) = removeChannels(Seq(shortChannelId))
@ -42,7 +42,7 @@ trait NetworkDb {
*/
def removeChannels(shortChannelIds: Iterable[ShortChannelId])
def listChannels(): Map[ChannelAnnouncement, (BinaryData, Satoshi)]
def listChannels(): Map[ChannelAnnouncement, (ByteVector32, Satoshi)]
def addChannelUpdate(u: ChannelUpdate)

View file

@ -16,7 +16,7 @@
package fr.acinq.eclair.db
import fr.acinq.bitcoin.BinaryData
import fr.acinq.bitcoin.ByteVector32
/**
* Store the Lightning payments received by the node. Sent and relayed payments are not persisted.
@ -37,7 +37,7 @@ trait PaymentsDb {
def addPayment(payment: Payment)
def findByPaymentHash(paymentHash: BinaryData): Option[Payment]
def findByPaymentHash(paymentHash: ByteVector32): Option[Payment]
def listPayments(): Seq[Payment]
@ -52,4 +52,4 @@ trait PaymentsDb {
* @param amount_msat amount of the payment, in milli-satoshis
* @param timestamp absolute time in seconds since UNIX epoch when the payment was created.
*/
case class Payment(payment_hash: BinaryData, amount_msat: Long, timestamp: Long)
case class Payment(payment_hash: ByteVector32, amount_msat: Long, timestamp: Long)

View file

@ -16,7 +16,7 @@
package fr.acinq.eclair.db
import fr.acinq.bitcoin.BinaryData
import fr.acinq.bitcoin.ByteVector32
import fr.acinq.eclair.channel.Command
/**
@ -33,11 +33,11 @@ import fr.acinq.eclair.channel.Command
*/
trait PendingRelayDb {
def addPendingRelay(channelId: BinaryData, htlcId: Long, cmd: Command)
def addPendingRelay(channelId: ByteVector32, htlcId: Long, cmd: Command)
def removePendingRelay(channelId: BinaryData, htlcId: Long)
def removePendingRelay(channelId: ByteVector32, htlcId: Long)
def listPendingRelay(channelId: BinaryData): Seq[Command]
def listPendingRelay(channelId: ByteVector32): Seq[Command]
def close(): Unit

View file

@ -19,8 +19,8 @@ package fr.acinq.eclair.db.sqlite
import java.sql.Connection
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.bitcoin.{BinaryData, MilliSatoshi}
import fr.acinq.eclair.channel.{AvailableBalanceChanged, ChannelClosed, ChannelCreated, NetworkFeePaid}
import fr.acinq.bitcoin.MilliSatoshi
import fr.acinq.eclair.channel.{AvailableBalanceChanged, NetworkFeePaid}
import fr.acinq.eclair.db.{AuditDb, ChannelLifecycleEvent, NetworkFee, Stats}
import fr.acinq.eclair.payment.{PaymentReceived, PaymentRelayed, PaymentSent}
@ -30,6 +30,7 @@ import scala.compat.Platform
class SqliteAuditDb(sqlite: Connection) extends AuditDb {
import SqliteUtils._
import ExtendedResultSet._
val DB_NAME = "audit"
val CURRENT_VERSION = 1
@ -53,8 +54,8 @@ class SqliteAuditDb(sqlite: Connection) extends AuditDb {
override def add(e: AvailableBalanceChanged): Unit =
using(sqlite.prepareStatement("INSERT INTO balance_updated VALUES (?, ?, ?, ?, ?, ?)")) { statement =>
statement.setBytes(1, e.channelId)
statement.setBytes(2, e.commitments.remoteParams.nodeId.toBin)
statement.setBytes(1, e.channelId.toArray)
statement.setBytes(2, e.commitments.remoteParams.nodeId.toBin.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
@ -64,8 +65,8 @@ class SqliteAuditDb(sqlite: Connection) extends AuditDb {
override def add(e: ChannelLifecycleEvent): Unit =
using(sqlite.prepareStatement("INSERT INTO channel_events VALUES (?, ?, ?, ?, ?, ?, ?)")) { statement =>
statement.setBytes(1, e.channelId)
statement.setBytes(2, e.remoteNodeId.toBin)
statement.setBytes(1, e.channelId.toArray)
statement.setBytes(2, e.remoteNodeId.toBin.toArray)
statement.setLong(3, e.capacitySat)
statement.setBoolean(4, e.isFunder)
statement.setBoolean(5, e.isPrivate)
@ -78,9 +79,9 @@ class SqliteAuditDb(sqlite: Connection) extends AuditDb {
using(sqlite.prepareStatement("INSERT INTO sent VALUES (?, ?, ?, ?, ?, ?)")) { statement =>
statement.setLong(1, e.amount.toLong)
statement.setLong(2, e.feesPaid.toLong)
statement.setBytes(3, e.paymentHash)
statement.setBytes(4, e.paymentPreimage)
statement.setBytes(5, e.toChannelId)
statement.setBytes(3, e.paymentHash.toArray)
statement.setBytes(4, e.paymentPreimage.toArray)
statement.setBytes(5, e.toChannelId.toArray)
statement.setLong(6, e.timestamp)
statement.executeUpdate()
}
@ -88,8 +89,8 @@ class SqliteAuditDb(sqlite: Connection) extends AuditDb {
override def add(e: PaymentReceived): Unit =
using(sqlite.prepareStatement("INSERT INTO received VALUES (?, ?, ?, ?)")) { statement =>
statement.setLong(1, e.amount.toLong)
statement.setBytes(2, e.paymentHash)
statement.setBytes(3, e.fromChannelId)
statement.setBytes(2, e.paymentHash.toArray)
statement.setBytes(3, e.fromChannelId.toArray)
statement.setLong(4, e.timestamp)
statement.executeUpdate()
}
@ -98,18 +99,18 @@ class SqliteAuditDb(sqlite: Connection) extends AuditDb {
using(sqlite.prepareStatement("INSERT INTO relayed VALUES (?, ?, ?, ?, ?, ?)")) { statement =>
statement.setLong(1, e.amountIn.toLong)
statement.setLong(2, e.amountOut.toLong)
statement.setBytes(3, e.paymentHash)
statement.setBytes(4, e.fromChannelId)
statement.setBytes(5, e.toChannelId)
statement.setBytes(3, e.paymentHash.toArray)
statement.setBytes(4, e.fromChannelId.toArray)
statement.setBytes(5, e.toChannelId.toArray)
statement.setLong(6, e.timestamp)
statement.executeUpdate()
}
override def add(e: NetworkFeePaid): Unit =
using(sqlite.prepareStatement("INSERT INTO network_fees VALUES (?, ?, ?, ?, ?, ?)")) { statement =>
statement.setBytes(1, e.channelId)
statement.setBytes(2, e.remoteNodeId.toBin)
statement.setBytes(3, e.tx.txid)
statement.setBytes(1, e.channelId.toArray)
statement.setBytes(2, e.remoteNodeId.toBin.toArray)
statement.setBytes(3, e.tx.txid.toArray)
statement.setLong(4, e.fee.toLong)
statement.setString(5, e.txType)
statement.setLong(6, Platform.currentTime)
@ -126,9 +127,9 @@ class SqliteAuditDb(sqlite: Connection) extends AuditDb {
q = q :+ PaymentSent(
amount = MilliSatoshi(rs.getLong("amount_msat")),
feesPaid = MilliSatoshi(rs.getLong("fees_msat")),
paymentHash = BinaryData(rs.getBytes("payment_hash")),
paymentPreimage = BinaryData(rs.getBytes("payment_preimage")),
toChannelId = BinaryData(rs.getBytes("to_channel_id")),
paymentHash = rs.getByteVector32("payment_hash"),
paymentPreimage = rs.getByteVector32("payment_preimage"),
toChannelId = rs.getByteVector32("to_channel_id"),
timestamp = rs.getLong("timestamp"))
}
q
@ -143,8 +144,8 @@ class SqliteAuditDb(sqlite: Connection) extends AuditDb {
while (rs.next()) {
q = q :+ PaymentReceived(
amount = MilliSatoshi(rs.getLong("amount_msat")),
paymentHash = BinaryData(rs.getBytes("payment_hash")),
fromChannelId = BinaryData(rs.getBytes("from_channel_id")),
paymentHash = rs.getByteVector32("payment_hash"),
fromChannelId = rs.getByteVector32("from_channel_id"),
timestamp = rs.getLong("timestamp"))
}
q
@ -160,9 +161,9 @@ class SqliteAuditDb(sqlite: Connection) extends AuditDb {
q = q :+ PaymentRelayed(
amountIn = MilliSatoshi(rs.getLong("amount_in_msat")),
amountOut = MilliSatoshi(rs.getLong("amount_out_msat")),
paymentHash = BinaryData(rs.getBytes("payment_hash")),
fromChannelId = BinaryData(rs.getBytes("from_channel_id")),
toChannelId = BinaryData(rs.getBytes("to_channel_id")),
paymentHash = rs.getByteVector32("payment_hash"),
fromChannelId = rs.getByteVector32("from_channel_id"),
toChannelId = rs.getByteVector32("to_channel_id"),
timestamp = rs.getLong("timestamp"))
}
q
@ -176,9 +177,9 @@ class SqliteAuditDb(sqlite: Connection) extends AuditDb {
var q: Queue[NetworkFee] = Queue()
while (rs.next()) {
q = q :+ NetworkFee(
remoteNodeId = PublicKey(rs.getBytes("node_id")),
channelId = BinaryData(rs.getBytes("channel_id")),
txId = BinaryData(rs.getBytes("tx_id")),
remoteNodeId = PublicKey(rs.getByteVector("node_id")),
channelId = rs.getByteVector32("channel_id"),
txId = rs.getByteVector32("tx_id"),
feeSat = rs.getLong("fee_sat"),
txType = rs.getString("tx_type"),
timestamp = rs.getLong("timestamp"))
@ -220,7 +221,7 @@ class SqliteAuditDb(sqlite: Connection) extends AuditDb {
var q: Queue[Stats] = Queue()
while (rs.next()) {
q = q :+ Stats(
channelId = BinaryData(rs.getBytes("channel_id")),
channelId = rs.getByteVector32("channel_id"),
avgPaymentAmountSatoshi = rs.getLong("avg_payment_amount_sat"),
paymentCount = rs.getInt("payment_count"),
relayFeeSatoshi = rs.getLong("relay_fee_sat"),

View file

@ -18,7 +18,7 @@ package fr.acinq.eclair.db.sqlite
import java.sql.Connection
import fr.acinq.bitcoin.BinaryData
import fr.acinq.bitcoin.ByteVector32
import fr.acinq.eclair.channel.HasCommitments
import fr.acinq.eclair.db.ChannelsDb
import fr.acinq.eclair.wire.ChannelCodecs.stateDataCodec
@ -27,6 +27,7 @@ import scala.collection.immutable.Queue
class SqliteChannelsDb(sqlite: Connection) extends ChannelsDb {
import SqliteUtils.ExtendedResultSet._
import SqliteUtils._
val DB_NAME = "channels"
@ -44,10 +45,10 @@ class SqliteChannelsDb(sqlite: Connection) extends ChannelsDb {
val data = stateDataCodec.encode(state).require.toByteArray
using (sqlite.prepareStatement("UPDATE local_channels SET data=? WHERE channel_id=?")) { update =>
update.setBytes(1, data)
update.setBytes(2, state.channelId)
update.setBytes(2, state.channelId.toArray)
if (update.executeUpdate() == 0) {
using(sqlite.prepareStatement("INSERT INTO local_channels VALUES (?, ?)")) { statement =>
statement.setBytes(1, state.channelId)
statement.setBytes(1, state.channelId.toArray)
statement.setBytes(2, data)
statement.executeUpdate()
}
@ -55,19 +56,19 @@ class SqliteChannelsDb(sqlite: Connection) extends ChannelsDb {
}
}
override def removeChannel(channelId: BinaryData): Unit = {
override def removeChannel(channelId: ByteVector32): Unit = {
using(sqlite.prepareStatement("DELETE FROM pending_relay WHERE channel_id=?")) { statement =>
statement.setBytes(1, channelId)
statement.setBytes(1, channelId.toArray)
statement.executeUpdate()
}
using(sqlite.prepareStatement("DELETE FROM htlc_infos WHERE channel_id=?")) { statement =>
statement.setBytes(1, channelId)
statement.setBytes(1, channelId.toArray)
statement.executeUpdate()
}
using(sqlite.prepareStatement("DELETE FROM local_channels WHERE channel_id=?")) { statement =>
statement.setBytes(1, channelId)
statement.setBytes(1, channelId.toArray)
statement.executeUpdate()
}
}
@ -79,24 +80,24 @@ class SqliteChannelsDb(sqlite: Connection) extends ChannelsDb {
}
}
def addOrUpdateHtlcInfo(channelId: BinaryData, commitmentNumber: Long, paymentHash: BinaryData, cltvExpiry: Long): Unit = {
def addOrUpdateHtlcInfo(channelId: ByteVector32, commitmentNumber: Long, paymentHash: ByteVector32, cltvExpiry: Long): Unit = {
using(sqlite.prepareStatement("INSERT OR IGNORE INTO htlc_infos VALUES (?, ?, ?, ?)")) { statement =>
statement.setBytes(1, channelId)
statement.setBytes(1, channelId.toArray)
statement.setLong(2, commitmentNumber)
statement.setBytes(3, paymentHash)
statement.setBytes(3, paymentHash.toArray)
statement.setLong(4, cltvExpiry)
statement.executeUpdate()
}
}
def listHtlcInfos(channelId: BinaryData, commitmentNumber: Long): Seq[(BinaryData, Long)] = {
def listHtlcInfos(channelId: ByteVector32, commitmentNumber: Long): Seq[(ByteVector32, Long)] = {
using(sqlite.prepareStatement("SELECT payment_hash, cltv_expiry FROM htlc_infos WHERE channel_id=? AND commitment_number=?")) { statement =>
statement.setBytes(1, channelId)
statement.setBytes(1, channelId.toArray)
statement.setLong(2, commitmentNumber)
val rs = statement.executeQuery
var q: Queue[(BinaryData, Long)] = Queue()
var q: Queue[(ByteVector32, Long)] = Queue()
while (rs.next()) {
q = q :+ (BinaryData(rs.getBytes("payment_hash")), rs.getLong("cltv_expiry"))
q = q :+ (ByteVector32(rs.getByteVector32("payment_hash")), rs.getLong("cltv_expiry"))
}
q
}

View file

@ -18,9 +18,9 @@ package fr.acinq.eclair.db.sqlite
import java.sql.Connection
import fr.acinq.bitcoin.{BinaryData, Crypto, Satoshi}
import fr.acinq.bitcoin.{ByteVector32, Crypto, Satoshi}
import fr.acinq.eclair.ShortChannelId
import fr.acinq.eclair.db.{NetworkDb, Payment}
import fr.acinq.eclair.db.NetworkDb
import fr.acinq.eclair.router.Announcements
import fr.acinq.eclair.wire.LightningMessageCodecs.{channelAnnouncementCodec, channelUpdateCodec, nodeAnnouncementCodec}
import fr.acinq.eclair.wire.{ChannelAnnouncement, ChannelUpdate, NodeAnnouncement}
@ -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)
statement.setBytes(1, n.nodeId.toBin.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)
statement.setBytes(2, n.nodeId.toBin.toArray)
statement.executeUpdate()
}
}
override def removeNode(nodeId: Crypto.PublicKey): Unit = {
using(sqlite.prepareStatement("DELETE FROM nodes WHERE node_id=?")) { statement =>
statement.setBytes(1, nodeId.toBin)
statement.setBytes(1, nodeId.toBin.toArray)
statement.executeUpdate()
}
}
@ -73,10 +73,10 @@ class SqliteNetworkDb(sqlite: Connection) extends NetworkDb {
}
}
override def addChannel(c: ChannelAnnouncement, txid: BinaryData, capacity: Satoshi): Unit = {
override def addChannel(c: ChannelAnnouncement, txid: ByteVector32, capacity: Satoshi): Unit = {
using(sqlite.prepareStatement("INSERT OR IGNORE INTO channels VALUES (?, ?, ?, ?)")) { statement =>
statement.setLong(1, c.shortChannelId.toLong)
statement.setString(2, txid.toString())
statement.setString(2, txid.toHex)
statement.setBytes(3, channelAnnouncementCodec.encode(c).require.toByteArray)
statement.setLong(4, capacity.amount)
statement.executeUpdate()
@ -99,13 +99,13 @@ class SqliteNetworkDb(sqlite: Connection) extends NetworkDb {
shortChannelIds.grouped(1000).foreach(removeChannelsInternal)
}
override def listChannels(): Map[ChannelAnnouncement, (BinaryData, Satoshi)] = {
override def listChannels(): Map[ChannelAnnouncement, (ByteVector32, Satoshi)] = {
using(sqlite.createStatement()) { statement =>
val rs = statement.executeQuery("SELECT data, txid, capacity_sat FROM channels")
var m: Map[ChannelAnnouncement, (BinaryData, Satoshi)] = Map()
var m: Map[ChannelAnnouncement, (ByteVector32, Satoshi)] = Map()
while (rs.next()) {
m += (channelAnnouncementCodec.decode(BitVector(rs.getBytes("data"))).require.value ->
(BinaryData(rs.getString("txid")), Satoshi(rs.getLong("capacity_sat"))))
(ByteVector32.fromValidHex(rs.getString("txid")), Satoshi(rs.getLong("capacity_sat"))))
}
m
}

View file

@ -18,7 +18,7 @@ package fr.acinq.eclair.db.sqlite
import java.sql.Connection
import fr.acinq.bitcoin.BinaryData
import fr.acinq.bitcoin.ByteVector32
import fr.acinq.eclair.db.sqlite.SqliteUtils.{getVersion, using}
import fr.acinq.eclair.db.{Payment, PaymentsDb}
import grizzled.slf4j.Logging
@ -37,6 +37,8 @@ import scala.collection.immutable.Queue
*/
class SqlitePaymentsDb(sqlite: Connection) extends PaymentsDb with Logging {
import SqliteUtils.ExtendedResultSet._
val DB_NAME = "payments"
val CURRENT_VERSION = 1
@ -47,7 +49,7 @@ class SqlitePaymentsDb(sqlite: Connection) extends PaymentsDb with Logging {
override def addPayment(payment: Payment): Unit = {
using(sqlite.prepareStatement("INSERT INTO payments VALUES (?, ?, ?)")) { statement =>
statement.setBytes(1, payment.payment_hash)
statement.setBytes(1, payment.payment_hash.toArray)
statement.setLong(2, payment.amount_msat)
statement.setLong(3, payment.timestamp)
val res = statement.executeUpdate()
@ -55,12 +57,12 @@ class SqlitePaymentsDb(sqlite: Connection) extends PaymentsDb with Logging {
}
}
override def findByPaymentHash(paymentHash: BinaryData): Option[Payment] = {
override def findByPaymentHash(paymentHash: ByteVector32): Option[Payment] = {
using(sqlite.prepareStatement("SELECT payment_hash, amount_msat, timestamp FROM payments WHERE payment_hash = ?")) { statement =>
statement.setBytes(1, paymentHash)
statement.setBytes(1, paymentHash.toArray)
val rs = statement.executeQuery()
if (rs.next()) {
Some(Payment(BinaryData(rs.getBytes("payment_hash")), rs.getLong("amount_msat"), rs.getLong("timestamp")))
Some(Payment(rs.getByteVector32("payment_hash"), rs.getLong("amount_msat"), rs.getLong("timestamp")))
} else {
None
}
@ -72,7 +74,7 @@ class SqlitePaymentsDb(sqlite: Connection) extends PaymentsDb with Logging {
val rs = statement.executeQuery("SELECT payment_hash, amount_msat, timestamp FROM payments")
var q: Queue[Payment] = Queue()
while (rs.next()) {
q = q :+ Payment(BinaryData(rs.getBytes("payment_hash")), rs.getLong("amount_msat"), rs.getLong("timestamp"))
q = q :+ Payment(rs.getByteVector32("payment_hash"), rs.getLong("amount_msat"), rs.getLong("timestamp"))
}
q
}

View file

@ -27,6 +27,8 @@ import scodec.bits.BitVector
class SqlitePeersDb(sqlite: Connection) extends PeersDb {
import SqliteUtils.ExtendedResultSet._
val DB_NAME = "peers"
val CURRENT_VERSION = 1
@ -39,10 +41,10 @@ class SqlitePeersDb(sqlite: Connection) extends PeersDb {
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)
update.setBytes(2, nodeId.toBin.toArray)
if (update.executeUpdate() == 0) {
using(sqlite.prepareStatement("INSERT INTO peers VALUES (?, ?)")) { statement =>
statement.setBytes(1, nodeId.toBin)
statement.setBytes(1, nodeId.toBin.toArray)
statement.setBytes(2, data)
statement.executeUpdate()
}
@ -52,7 +54,7 @@ class SqlitePeersDb(sqlite: Connection) extends PeersDb {
override def removePeer(nodeId: Crypto.PublicKey): Unit = {
using(sqlite.prepareStatement("DELETE FROM peers WHERE node_id=?")) { statement =>
statement.setBytes(1, nodeId.toBin)
statement.setBytes(1, nodeId.toBin.toArray)
statement.executeUpdate()
}
}
@ -62,7 +64,7 @@ class SqlitePeersDb(sqlite: Connection) extends PeersDb {
val rs = statement.executeQuery("SELECT node_id, data FROM peers")
var m: Map[PublicKey, NodeAddress] = Map()
while (rs.next()) {
val nodeid = PublicKey(rs.getBytes("node_id"))
val nodeid = PublicKey(rs.getByteVector("node_id"))
val nodeaddress = LightningMessageCodecs.nodeaddress.decode(BitVector(rs.getBytes("data"))).require.value
m += (nodeid -> nodeaddress)
}

View file

@ -18,7 +18,7 @@ package fr.acinq.eclair.db.sqlite
import java.sql.Connection
import fr.acinq.bitcoin.BinaryData
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}
@ -35,26 +35,26 @@ class SqlitePendingRelayDb(sqlite: Connection) extends PendingRelayDb {
statement.executeUpdate("CREATE TABLE IF NOT EXISTS pending_relay (channel_id BLOB NOT NULL, htlc_id INTEGER NOT NULL, data BLOB NOT NULL, PRIMARY KEY(channel_id, htlc_id))")
}
override def addPendingRelay(channelId: BinaryData, htlcId: Long, cmd: Command): Unit = {
override def addPendingRelay(channelId: ByteVector32, htlcId: Long, cmd: Command): Unit = {
using(sqlite.prepareStatement("INSERT OR IGNORE INTO pending_relay VALUES (?, ?, ?)")) { statement =>
statement.setBytes(1, channelId)
statement.setBytes(1, channelId.toArray)
statement.setLong(2, htlcId)
statement.setBytes(3, cmdCodec.encode(cmd).require.toByteArray)
statement.executeUpdate()
}
}
override def removePendingRelay(channelId: BinaryData, htlcId: Long): Unit = {
override def removePendingRelay(channelId: ByteVector32, htlcId: Long): Unit = {
using(sqlite.prepareStatement("DELETE FROM pending_relay WHERE channel_id=? AND htlc_id=?")) { statement =>
statement.setBytes(1, channelId)
statement.setBytes(1, channelId.toArray)
statement.setLong(2, htlcId)
statement.executeUpdate()
}
}
override def listPendingRelay(channelId: BinaryData): Seq[Command] = {
override def listPendingRelay(channelId: ByteVector32): Seq[Command] = {
using(sqlite.prepareStatement("SELECT htlc_id, data FROM pending_relay WHERE channel_id=?")) { statement =>
statement.setBytes(1, channelId)
statement.setBytes(1, channelId.toArray)
val rs = statement.executeQuery()
codecSequence(rs, cmdCodec)
}

View file

@ -18,8 +18,9 @@ package fr.acinq.eclair.db.sqlite
import java.sql.{Connection, ResultSet, Statement}
import fr.acinq.bitcoin.ByteVector32
import scodec.Codec
import scodec.bits.BitVector
import scodec.bits.{BitVector, ByteVector}
import scala.collection.immutable.Queue
@ -92,4 +93,15 @@ object SqliteUtils {
statement.executeUpdate("CREATE TABLE IF NOT EXISTS dummy_table_for_locking (a INTEGER NOT NULL)")
statement.executeUpdate("INSERT INTO dummy_table_for_locking VALUES (42)")
}
case class ExtendedResultSet(rs: ResultSet) {
def getByteVector(columnLabel: String): ByteVector = ByteVector(rs.getBytes(columnLabel))
def getByteVector32(columnLabel: String): ByteVector32 = ByteVector32(ByteVector(rs.getBytes(columnLabel)))
}
object ExtendedResultSet {
implicit def conv(rs: ResultSet): ExtendedResultSet = ExtendedResultSet(rs)
}
}

View file

@ -25,8 +25,7 @@ import akka.io.{IO, Tcp}
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.eclair.io.Client.ConnectionFailed
import fr.acinq.eclair.tor.Socks5Connection.{Socks5Connect, Socks5Connected, Socks5Error}
import fr.acinq.eclair.tor.{Socks5Connection, Socks5ProxyParams, TorException}
import fr.acinq.eclair.wire.NodeAddress
import fr.acinq.eclair.tor.{Socks5Connection, Socks5ProxyParams}
import fr.acinq.eclair.{Logs, NodeParams}
import scala.concurrent.duration._

View file

@ -18,6 +18,7 @@ package fr.acinq.eclair.io
import com.google.common.net.HostAndPort
import fr.acinq.bitcoin.Crypto.PublicKey
import scodec.bits.ByteVector
import scala.util.{Failure, Success, Try}
@ -39,7 +40,7 @@ object NodeURI {
@throws[IllegalArgumentException]
def parse(uri: String): NodeURI = {
uri.split("@") match {
case Array(nodeId, address) => (Try(PublicKey(nodeId)), Try(HostAndPort.fromString(address).withDefaultPort(DEFAULT_PORT))) match {
case Array(nodeId, address) => (Try(PublicKey(ByteVector.fromValidHex(nodeId))), Try(HostAndPort.fromString(address).withDefaultPort(DEFAULT_PORT))) match {
case (Success(pk), Success(hostAndPort)) => NodeURI(pk, hostAndPort)
case (Failure(_), _) => throw new IllegalArgumentException("Invalid node id")
case (_, Failure(_)) => throw new IllegalArgumentException("Invalid host:port")

View file

@ -23,7 +23,7 @@ import java.nio.ByteOrder
import akka.actor.{ActorRef, FSM, OneForOneStrategy, PoisonPill, Props, Status, SupervisorStrategy, Terminated}
import akka.event.Logging.MDC
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.bitcoin.{BinaryData, DeterministicWallet, MilliSatoshi, Protocol, Satoshi}
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
@ -31,6 +31,7 @@ import fr.acinq.eclair.router._
import fr.acinq.eclair.wire._
import fr.acinq.eclair.{wire, _}
import scodec.Attempt
import scodec.bits.ByteVector
import scala.compat.Platform
import scala.concurrent.duration._
@ -88,7 +89,7 @@ class Peer(nodeParams: NodeParams, remoteNodeId: PublicKey, authenticator: Actor
case Some((gf, lf)) => wire.Init(globalFeatures = gf, localFeatures = lf)
case None => wire.Init(globalFeatures = nodeParams.globalFeatures, localFeatures = nodeParams.localFeatures)
}
log.info(s"using globalFeatures=${localInit.globalFeatures} and localFeatures=${localInit.localFeatures}")
log.info(s"using globalFeatures=${localInit.globalFeatures.toBin} and localFeatures=${localInit.localFeatures.toBin}")
transport ! localInit
val address_opt = if (outgoing) {
@ -120,6 +121,7 @@ class Peer(nodeParams: NodeParams, remoteNodeId: PublicKey, authenticator: Actor
val remoteHasInitialRoutingSync = Features.hasFeature(remoteInit.localFeatures, Features.INITIAL_ROUTING_SYNC_BIT_OPTIONAL)
val remoteHasChannelRangeQueriesOptional = Features.hasFeature(remoteInit.localFeatures, Features.CHANNEL_RANGE_QUERIES_BIT_OPTIONAL)
val remoteHasChannelRangeQueriesMandatory = Features.hasFeature(remoteInit.localFeatures, Features.CHANNEL_RANGE_QUERIES_BIT_MANDATORY)
log.info(s"peer is using globalFeatures=${remoteInit.globalFeatures.toBin} and localFeatures=${remoteInit.localFeatures.toBin}")
log.info(s"$remoteNodeId has features: initialRoutingSync=$remoteHasInitialRoutingSync channelRangeQueriesOptional=$remoteHasChannelRangeQueriesOptional channelRangeQueriesMandatory=$remoteHasChannelRangeQueriesMandatory")
if (Features.areSupported(remoteInit.localFeatures)) {
d.origin_opt.foreach(origin => origin ! "connected")
@ -193,7 +195,7 @@ class Peer(nodeParams: NodeParams, remoteNodeId: PublicKey, authenticator: Actor
// no need to use secure random here
val pingSize = Random.nextInt(1000)
val pongSize = Random.nextInt(1000)
val ping = wire.Ping(pongSize, BinaryData("00" * pingSize))
val ping = wire.Ping(pongSize, ByteVector.fill(pingSize)(0))
setTimer(PingTimeout.toString, PingTimeout(ping), nodeParams.pingTimeout, repeat = false)
d.transport ! ping
stay using d.copy(expectedPong_opt = Some(ExpectedPong(ping)))
@ -215,7 +217,7 @@ class Peer(nodeParams: NodeParams, remoteNodeId: PublicKey, authenticator: Actor
d.transport ! TransportHandler.ReadAck(ping)
if (pongLength <= 65532) {
// see BOLT 1: we reply only if requested pong length is acceptable
d.transport ! wire.Pong(BinaryData(Seq.fill[Byte](pongLength)(0.toByte)))
d.transport ! wire.Pong(ByteVector.fill(pongLength)(0.toByte))
} else {
log.warning(s"ignoring invalid ping with pongLength=${ping.pongLength}")
}
@ -239,7 +241,7 @@ class Peer(nodeParams: NodeParams, remoteNodeId: PublicKey, authenticator: Actor
case Event(err@wire.Error(channelId, reason), d: ConnectedData) if channelId == CHANNELID_ZERO =>
d.transport ! TransportHandler.ReadAck(err)
log.error(s"connection-level error, failing all channels! reason=${new String(reason)}")
log.error(s"connection-level error, failing all channels! reason=${new String(reason.toArray)}")
d.channels.values.toSet[ActorRef].foreach(_ forward err) // we deduplicate with toSet because there might be two entries per channel (tmp id and final id)
d.transport ! PoisonPill
stay
@ -256,7 +258,7 @@ class Peer(nodeParams: NodeParams, remoteNodeId: PublicKey, authenticator: Actor
case Event(c: Peer.OpenChannel, d: ConnectedData) =>
log.info(s"requesting a new channel to $remoteNodeId with fundingSatoshis=${c.fundingSatoshis}, pushMsat=${c.pushMsat} and fundingFeeratePerByte=${c.fundingTxFeeratePerKw_opt}")
val (channel, localParams) = createNewChannel(nodeParams, funder = true, c.fundingSatoshis.toLong, origin_opt = Some(sender))
val temporaryChannelId = randomBytes(32)
val temporaryChannelId = randomBytes32
val channelFeeratePerKw = Globals.feeratesPerKw.get.blocks_2
val fundingTxFeeratePerKw = c.fundingTxFeeratePerKw_opt.getOrElse(Globals.feeratesPerKw.get.blocks_6)
channel ! INPUT_INIT_FUNDER(temporaryChannelId, c.fundingSatoshis.amount, c.pushMsat.amount, channelFeeratePerKw, fundingTxFeeratePerKw, localParams, d.transport, d.remoteInit, c.channelFlags.getOrElse(nodeParams.channelFlags))
@ -378,14 +380,14 @@ class Peer(nodeParams: NodeParams, remoteNodeId: PublicKey, authenticator: Actor
log.error(s"peer sent us a routing message with invalid sig: r=$r bin=$bin")
// for now we just return an error, maybe ban the peer in the future?
// TODO: this doesn't actually disconnect the peer, once we introduce peer banning we should actively disconnect
d.transport ! Error(CHANNELID_ZERO, s"bad announcement sig! bin=$bin".getBytes())
d.transport ! Error(CHANNELID_ZERO, ByteVector.view(s"bad announcement sig! bin=$bin".getBytes()))
d.behavior
case InvalidAnnouncement(c) =>
// they seem to be sending us fake announcements?
log.error(s"couldn't find funding tx with valid scripts for shortChannelId=${c.shortChannelId}")
// for now we just return an error, maybe ban the peer in the future?
// TODO: this doesn't actually disconnect the peer, once we introduce peer banning we should actively disconnect
d.transport ! Error(CHANNELID_ZERO, s"couldn't verify channel! shortChannelId=${c.shortChannelId}".getBytes())
d.transport ! Error(CHANNELID_ZERO, ByteVector.view(s"couldn't verify channel! shortChannelId=${c.shortChannelId}".getBytes()))
d.behavior
case ChannelClosed(_) =>
if (d.behavior.ignoreNetworkAnnouncement) {
@ -501,9 +503,9 @@ class Peer(nodeParams: NodeParams, remoteNodeId: PublicKey, authenticator: Actor
object Peer {
val CHANNELID_ZERO = BinaryData("00" * 32)
val CHANNELID_ZERO = ByteVector32.Zeroes
val UNKNOWN_CHANNEL_MESSAGE = "unknown channel".getBytes()
val UNKNOWN_CHANNEL_MESSAGE = ByteVector.view("unknown channel".getBytes())
val RECONNECT_TIMER = "reconnect"
@ -517,9 +519,9 @@ object Peer {
// @formatter:off
sealed trait ChannelId { def id: BinaryData }
case class TemporaryChannelId(id: BinaryData) extends ChannelId
case class FinalChannelId(id: BinaryData) extends ChannelId
sealed trait ChannelId { def id: ByteVector32 }
case class TemporaryChannelId(id: ByteVector32) extends ChannelId
case class FinalChannelId(id: ByteVector32) extends ChannelId
sealed trait Data {
def address_opt: Option[InetSocketAddress]
@ -565,7 +567,7 @@ object Peer {
// @formatter:on
def makeChannelParams(nodeParams: NodeParams, defaultFinalScriptPubKey: BinaryData, isFunder: Boolean, fundingSatoshis: Long): LocalParams = {
def makeChannelParams(nodeParams: NodeParams, defaultFinalScriptPubKey: ByteVector, isFunder: Boolean, fundingSatoshis: Long): LocalParams = {
val entropy = new Array[Byte](16)
secureRandom.nextBytes(entropy)
val bis = new ByteArrayInputStream(entropy)
@ -573,7 +575,7 @@ object Peer {
makeChannelParams(nodeParams, defaultFinalScriptPubKey, isFunder, fundingSatoshis, channelKeyPath)
}
def makeChannelParams(nodeParams: NodeParams, defaultFinalScriptPubKey: BinaryData, isFunder: Boolean, fundingSatoshis: Long, channelKeyPath: DeterministicWallet.KeyPath): LocalParams = {
def makeChannelParams(nodeParams: NodeParams, defaultFinalScriptPubKey: ByteVector, isFunder: Boolean, fundingSatoshis: Long, channelKeyPath: DeterministicWallet.KeyPath): LocalParams = {
LocalParams(
nodeParams.nodeId,
channelKeyPath,

View file

@ -19,14 +19,10 @@ package fr.acinq.eclair.io
import java.net.InetSocketAddress
import akka.Done
import akka.actor.{Actor, ActorLogging, ActorRef, OneForOneStrategy, Props, SupervisorStrategy}
import akka.actor.{Actor, ActorLogging, ActorRef, Props}
import akka.io.Tcp.SO.KeepAlive
import akka.io.{IO, Tcp}
import fr.acinq.eclair.NodeParams
import fr.acinq.eclair.crypto.Noise.KeyPair
import fr.acinq.eclair.crypto.TransportHandler
import fr.acinq.eclair.crypto.TransportHandler.HandshakeCompleted
import fr.acinq.eclair.wire.{LightningMessage, LightningMessageCodecs}
import scala.concurrent.Promise

View file

@ -19,9 +19,9 @@ package fr.acinq
import java.security.SecureRandom
import fr.acinq.bitcoin.Crypto.PrivateKey
import fr.acinq.bitcoin.{BinaryData, _}
import fr.acinq.bitcoin._
import scodec.Attempt
import scodec.bits.BitVector
import scodec.bits.{BitVector, ByteVector}
import scala.util.{Failure, Success, Try}
@ -33,23 +33,25 @@ package object eclair {
*/
val secureRandom = new SecureRandom()
def randomBytes(length: Int): BinaryData = {
def randomBytes(length: Int): ByteVector = {
val buffer = new Array[Byte](length)
secureRandom.nextBytes(buffer)
buffer
ByteVector.view(buffer)
}
def randomKey: PrivateKey = PrivateKey(randomBytes(32), compressed = true)
def randomBytes32: ByteVector32 = ByteVector32(randomBytes(32))
def toLongId(fundingTxHash: BinaryData, fundingOutputIndex: Int): BinaryData = {
def randomKey: PrivateKey = PrivateKey(randomBytes32, compressed = true)
def toLongId(fundingTxHash: ByteVector32, fundingOutputIndex: Int): ByteVector32 = {
require(fundingOutputIndex < 65536, "fundingOutputIndex must not be greater than FFFF")
require(fundingTxHash.size == 32, "fundingTxHash must be of length 32B")
val channelId = fundingTxHash.take(30) :+ (fundingTxHash.data(30) ^ (fundingOutputIndex >> 8)).toByte :+ (fundingTxHash.data(31) ^ fundingOutputIndex).toByte
BinaryData(channelId)
val channelId = ByteVector32(fundingTxHash.take(30) :+ (fundingTxHash(30) ^ (fundingOutputIndex >> 8)).toByte :+ (fundingTxHash(31) ^ fundingOutputIndex).toByte)
channelId
}
def serializationResult(attempt: Attempt[BitVector]): BinaryData = attempt match {
case Attempt.Successful(bin) => BinaryData(bin.toByteArray)
def serializationResult(attempt: Attempt[BitVector]): ByteVector = attempt match {
case Attempt.Successful(bin) => bin.toByteVector
case Attempt.Failure(cause) => throw new RuntimeException(s"serialization error: $cause")
}
@ -112,7 +114,7 @@ package object eclair {
*
* @param data to check
*/
def isAsciiPrintable(data: BinaryData): Boolean = data.data.forall(ch => ch >= 32 && ch < 127)
def isAsciiPrintable(data: ByteVector): Boolean = data.toArray.forall(ch => ch >= 32 && ch < 127)
/**
*
@ -130,7 +132,7 @@ package object eclair {
* @return the public key script that matches the input address.
*/
def addressToPublicKeyScript(address: String, chainHash: BinaryData): Seq[ScriptElt] = {
def addressToPublicKeyScript(address: String, chainHash: ByteVector32): Seq[ScriptElt] = {
Try(Base58Check.decode(address)) match {
case Success((Base58.Prefix.PubkeyAddressTestnet, pubKeyHash)) if chainHash == Block.TestnetGenesisBlock.hash || chainHash == Block.RegtestGenesisBlock.hash => Script.pay2pkh(pubKeyHash)
case Success((Base58.Prefix.PubkeyAddress, pubKeyHash)) if chainHash == Block.LivenetGenesisBlock.hash => Script.pay2pkh(pubKeyHash)

View file

@ -17,13 +17,13 @@
package fr.acinq.eclair.payment
import akka.actor.{Actor, ActorLogging, Props}
import fr.acinq.bitcoin.BinaryData
import fr.acinq.bitcoin.ByteVector32
import fr.acinq.eclair.NodeParams
import fr.acinq.eclair.channel._
import fr.acinq.eclair.db.{AuditDb, ChannelLifecycleEvent}
import scala.concurrent.duration._
import scala.concurrent.ExecutionContext
import scala.concurrent.duration._
class Auditor(nodeParams: NodeParams) extends Actor with ActorLogging {
@ -75,11 +75,11 @@ class BalanceEventThrottler(db: AuditDb) extends Actor with ActorLogging {
case class BalanceUpdate(first: AvailableBalanceChanged, last: AvailableBalanceChanged)
case class ProcessEvent(channelId: BinaryData)
case class ProcessEvent(channelId: ByteVector32)
override def receive: Receive = run(Map.empty)
def run(pending: Map[BinaryData, BalanceUpdate]): Receive = {
def run(pending: Map[ByteVector32, BalanceUpdate]): Receive = {
case e: AvailableBalanceChanged =>
pending.get(e.channelId) match {

View file

@ -20,9 +20,9 @@ import akka.actor.{Actor, ActorLogging, ActorRef, Props}
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.eclair.crypto.Sphinx.ErrorPacket
import fr.acinq.eclair.payment.PaymentLifecycle.{PaymentFailed, PaymentResult, RemoteFailure, SendPayment}
import fr.acinq.eclair.{NodeParams, randomBytes, secureRandom}
import fr.acinq.eclair.router.{Announcements, ChannelDesc, Data}
import fr.acinq.eclair.wire.{ChannelUpdate, UnknownPaymentHash}
import fr.acinq.eclair.router.{Announcements, Data}
import fr.acinq.eclair.wire.UnknownPaymentHash
import fr.acinq.eclair.{NodeParams, randomBytes32, secureRandom}
import scala.concurrent.duration._
@ -52,7 +52,7 @@ class Autoprobe(nodeParams: NodeParams, router: ActorRef, paymentInitiator: Acto
case TickProbe =>
pickPaymentDestination(nodeParams.nodeId, routingData) match {
case Some(targetNodeId) =>
val paymentHash = randomBytes(32) // we don't even know the preimage (this needs to be a secure random!)
val paymentHash = randomBytes32 // we don't even know the preimage (this needs to be a secure random!)
log.info(s"sending payment probe to node=$targetNodeId payment_hash=$paymentHash")
paymentInitiator ! SendPayment(PAYMENT_AMOUNT_MSAT, paymentHash, targetNodeId, maxAttempts = 1, randomize = Some(true))
case None =>

View file

@ -17,10 +17,9 @@
package fr.acinq.eclair.payment
import akka.actor.{Actor, ActorLogging, ActorRef}
import fr.acinq.bitcoin.BinaryData
import fr.acinq.bitcoin.ByteVector32
import fr.acinq.eclair.NodeParams
import fr.acinq.eclair.channel._
import fr.acinq.eclair.wire.{UpdateFailHtlc, UpdateFailMalformedHtlc, UpdateFulfillHtlc}
class CommandBuffer(nodeParams: NodeParams, register: ActorRef) extends Actor with ActorLogging {
@ -62,8 +61,8 @@ class CommandBuffer(nodeParams: NodeParams, register: ActorRef) extends Actor wi
object CommandBuffer {
case class CommandSend(channelId: BinaryData, htlcId: Long, cmd: Command)
case class CommandSend(channelId: ByteVector32, htlcId: Long, cmd: Command)
case class CommandAck(channelId: BinaryData, htlcId: Long)
case class CommandAck(channelId: ByteVector32, htlcId: Long)
}

View file

@ -17,12 +17,12 @@
package fr.acinq.eclair.payment
import akka.actor.{Actor, ActorLogging, Props, Status}
import fr.acinq.bitcoin.{BinaryData, Crypto, MilliSatoshi}
import fr.acinq.bitcoin.{ByteVector32, Crypto, MilliSatoshi}
import fr.acinq.eclair.channel.{CMD_FAIL_HTLC, CMD_FULFILL_HTLC, Channel}
import fr.acinq.eclair.db.Payment
import fr.acinq.eclair.payment.PaymentLifecycle.{CheckPayment, ReceivePayment}
import fr.acinq.eclair.wire._
import fr.acinq.eclair.{Globals, NodeParams, randomBytes}
import fr.acinq.eclair.{Globals, NodeParams, randomBytes32}
import scala.compat.Platform
import scala.concurrent.ExecutionContext
@ -45,7 +45,7 @@ class LocalPaymentHandler(nodeParams: NodeParams) extends Actor with ActorLoggin
override def receive: Receive = run(Map.empty)
def run(hash2preimage: Map[BinaryData, PendingPaymentRequest]): Receive = {
def run(hash2preimage: Map[ByteVector32, PendingPaymentRequest]): Receive = {
case PurgeExpiredRequests =>
context.become(run(hash2preimage.filterNot { case (_, pr) => hasExpired(pr) }))
@ -55,7 +55,7 @@ class LocalPaymentHandler(nodeParams: NodeParams) extends Actor with ActorLoggin
if (hash2preimage.size > nodeParams.maxPendingPaymentRequests) {
throw new RuntimeException(s"too many pending payment requests (max=${nodeParams.maxPendingPaymentRequests})")
}
val paymentPreimage = randomBytes(32)
val paymentPreimage = randomBytes32
val paymentHash = Crypto.sha256(paymentPreimage)
val expirySeconds = expirySeconds_opt.getOrElse(nodeParams.paymentRequestExpiry.toSeconds)
val paymentRequest = PaymentRequest(nodeParams.chainHash, amount_opt, paymentHash, nodeParams.privateKey, desc, fallbackAddress = None, expirySeconds = Some(expirySeconds), extraHops = extraHops)
@ -114,7 +114,7 @@ object LocalPaymentHandler {
case object PurgeExpiredRequests
case class PendingPaymentRequest(preimage: BinaryData, paymentRequest: PaymentRequest)
case class PendingPaymentRequest(preimage: ByteVector32, paymentRequest: PaymentRequest)
def hasExpired(pr: PendingPaymentRequest): Boolean = pr.paymentRequest.expiry match {
case Some(expiry) => pr.paymentRequest.timestamp + expiry <= Platform.currentTime / 1000

View file

@ -16,7 +16,7 @@
package fr.acinq.eclair.payment
import fr.acinq.bitcoin.{BinaryData, MilliSatoshi}
import fr.acinq.bitcoin.{ByteVector32, MilliSatoshi}
import scala.compat.Platform
@ -24,11 +24,11 @@ import scala.compat.Platform
* Created by PM on 01/02/2017.
*/
sealed trait PaymentEvent {
val paymentHash: BinaryData
val paymentHash: ByteVector32
}
case class PaymentSent(amount: MilliSatoshi, feesPaid: MilliSatoshi, paymentHash: BinaryData, paymentPreimage: BinaryData, toChannelId: BinaryData, timestamp: Long = Platform.currentTime) extends PaymentEvent
case class PaymentSent(amount: MilliSatoshi, feesPaid: MilliSatoshi, paymentHash: ByteVector32, paymentPreimage: ByteVector32, toChannelId: ByteVector32, timestamp: Long = Platform.currentTime) extends PaymentEvent
case class PaymentRelayed(amountIn: MilliSatoshi, amountOut: MilliSatoshi, paymentHash: BinaryData, fromChannelId: BinaryData, toChannelId: BinaryData, timestamp: Long = Platform.currentTime) extends PaymentEvent
case class PaymentRelayed(amountIn: MilliSatoshi, amountOut: MilliSatoshi, paymentHash: ByteVector32, fromChannelId: ByteVector32, toChannelId: ByteVector32, timestamp: Long = Platform.currentTime) extends PaymentEvent
case class PaymentReceived(amount: MilliSatoshi, paymentHash: BinaryData, fromChannelId: BinaryData, timestamp: Long = Platform.currentTime) extends PaymentEvent
case class PaymentReceived(amount: MilliSatoshi, paymentHash: ByteVector32, fromChannelId: ByteVector32, timestamp: Long = Platform.currentTime) extends PaymentEvent

View file

@ -16,12 +16,9 @@
package fr.acinq.eclair.payment
import java.text.NumberFormat
import java.util.Locale
import akka.actor.{ActorRef, FSM, Props, Status}
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.bitcoin.{BinaryData, MilliSatoshi}
import fr.acinq.bitcoin.{ByteVector32, 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 +28,7 @@ import fr.acinq.eclair.payment.PaymentRequest.ExtraHop
import fr.acinq.eclair.router._
import fr.acinq.eclair.wire._
import scodec.Attempt
import scodec.bits.ByteVector
import scala.util.{Failure, Success}
@ -187,7 +185,7 @@ object PaymentLifecycle {
* @param maxFeePct set by default to 3% as a safety measure (even if a route is found, if fee is higher than that payment won't be attempted)
*/
case class SendPayment(amountMsat: Long,
paymentHash: BinaryData,
paymentHash: ByteVector32,
targetNodeId: PublicKey,
assistedRoutes: Seq[Seq[ExtraHop]] = Nil,
finalCltvExpiry: Long = Channel.MIN_CLTV_EXPIRY,
@ -196,20 +194,20 @@ object PaymentLifecycle {
routeParams: Option[RouteParams] = None) {
require(amountMsat > 0, s"amountMsat must be > 0")
}
case class CheckPayment(paymentHash: BinaryData)
case class CheckPayment(paymentHash: ByteVector32)
sealed trait PaymentResult
case class PaymentSucceeded(amountMsat: Long, paymentHash: BinaryData, paymentPreimage: BinaryData, route: Seq[Hop]) extends PaymentResult // note: the amount includes fees
case class PaymentSucceeded(amountMsat: Long, paymentHash: ByteVector32, paymentPreimage: ByteVector32, route: Seq[Hop]) extends PaymentResult // note: the amount includes fees
sealed trait PaymentFailure
case class LocalFailure(t: Throwable) extends PaymentFailure
case class RemoteFailure(route: Seq[Hop], e: ErrorPacket) extends PaymentFailure
case class UnreadableRemoteFailure(route: Seq[Hop]) extends PaymentFailure
case class PaymentFailed(paymentHash: BinaryData, failures: Seq[PaymentFailure]) extends PaymentResult
case class PaymentFailed(paymentHash: ByteVector32, failures: Seq[PaymentFailure]) extends PaymentResult
sealed trait Data
case object WaitingForRequest extends Data
case class WaitingForRoute(sender: ActorRef, c: SendPayment, failures: Seq[PaymentFailure]) extends Data
case class WaitingForComplete(sender: ActorRef, c: SendPayment, cmd: CMD_ADD_HTLC, failures: Seq[PaymentFailure], sharedSecrets: Seq[(BinaryData, PublicKey)], ignoreNodes: Set[PublicKey], ignoreChannels: Set[ChannelDesc], hops: Seq[Hop]) extends Data
case class WaitingForComplete(sender: ActorRef, c: SendPayment, cmd: CMD_ADD_HTLC, failures: Seq[PaymentFailure], sharedSecrets: Seq[(ByteVector32, PublicKey)], ignoreNodes: Set[PublicKey], ignoreChannels: Set[ChannelDesc], hops: Seq[Hop]) extends Data
sealed trait State
case object WAITING_FOR_REQUEST extends State
@ -219,13 +217,13 @@ object PaymentLifecycle {
// @formatter:on
def buildOnion(nodes: Seq[PublicKey], payloads: Seq[PerHopPayload], associatedData: BinaryData): Sphinx.PacketAndSecrets = {
def buildOnion(nodes: Seq[PublicKey], payloads: Seq[PerHopPayload], associatedData: ByteVector32): Sphinx.PacketAndSecrets = {
require(nodes.size == payloads.size)
val sessionKey = randomKey
val payloadsbin: Seq[BinaryData] = payloads
.map(LightningMessageCodecs.perHopPayloadCodec.encode(_))
val payloadsbin: Seq[ByteVector] = payloads
.map(LightningMessageCodecs.perHopPayloadCodec.encode)
.map {
case Attempt.Successful(bitVector) => BinaryData(bitVector.toByteArray)
case Attempt.Successful(bitVector) => bitVector.toByteVector
case Attempt.Failure(cause) => throw new RuntimeException(s"serialization error: $cause")
}
Sphinx.makePacket(sessionKey, nodes, payloadsbin, associatedData)
@ -248,7 +246,7 @@ object PaymentLifecycle {
(msat + nextFee, expiry + hop.lastUpdate.cltvExpiryDelta, PerHopPayload(hop.lastUpdate.shortChannelId, msat, expiry) +: payloads)
}
def buildCommand(finalAmountMsat: Long, finalExpiry: Long, paymentHash: BinaryData, hops: Seq[Hop]): (CMD_ADD_HTLC, Seq[(BinaryData, PublicKey)]) = {
def buildCommand(finalAmountMsat: Long, finalExpiry: Long, paymentHash: ByteVector32, hops: Seq[Hop]): (CMD_ADD_HTLC, Seq[(ByteVector32, PublicKey)]) = {
val (firstAmountMsat, firstExpiry, payloads) = buildPayloads(finalAmountMsat, finalExpiry, hops.drop(1))
val nodes = hops.map(_.nextNodeId)
// BOLT 2 requires that associatedData == paymentHash
@ -296,7 +294,7 @@ object PaymentLifecycle {
* @return true if channel updates are "equal"
*/
def areSame(u1: ChannelUpdate, u2: ChannelUpdate): Boolean =
u1.copy(signature = BinaryData.empty, timestamp = 0) == u2.copy(signature = BinaryData.empty, timestamp = 0)
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)

View file

@ -19,7 +19,7 @@ package fr.acinq.eclair.payment
import java.math.BigInteger
import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey}
import fr.acinq.bitcoin.{BinaryData, MilliSatoshi, _}
import fr.acinq.bitcoin.{MilliSatoshi, _}
import fr.acinq.eclair.ShortChannelId
import fr.acinq.eclair.payment.PaymentRequest._
import scodec.Codec
@ -39,7 +39,7 @@ import scala.util.Try
* @param tags payment tags; must include a single PaymentHash tag
* @param signature request signature that will be checked against node id
*/
case class PaymentRequest(prefix: String, amount: Option[MilliSatoshi], timestamp: Long, nodeId: PublicKey, tags: List[PaymentRequest.TaggedField], signature: BinaryData) {
case class PaymentRequest(prefix: String, amount: Option[MilliSatoshi], timestamp: Long, nodeId: PublicKey, tags: List[PaymentRequest.TaggedField], signature: ByteVector) {
amount.map(a => require(a.amount > 0 && a.amount <= PaymentRequest.MAX_AMOUNT.amount, s"amount is not valid"))
require(tags.collect { case _: PaymentRequest.PaymentHash => {} }.size == 1, "there must be exactly one payment hash tag")
@ -55,7 +55,7 @@ case class PaymentRequest(prefix: String, amount: Option[MilliSatoshi], timestam
*
* @return the description of the payment, or its hash
*/
lazy val description: Either[String, BinaryData] = tags.collectFirst {
lazy val description: Either[String, ByteVector32] = tags.collectFirst {
case PaymentRequest.Description(d) => Left(d)
case PaymentRequest.DescriptionHash(h) => Right(h)
}.get
@ -82,11 +82,11 @@ case class PaymentRequest(prefix: String, amount: Option[MilliSatoshi], timestam
*
* @return the hash of this payment request
*/
def hash: BinaryData = {
def hash: ByteVector32 = {
val hrp = s"${prefix}${Amount.encode(amount)}".getBytes("UTF-8")
val data = Bolt11Data(timestamp, tags, "00" * 65) // fake sig that we are going to strip next
val data = Bolt11Data(timestamp, tags, ByteVector.fill(65)(0)) // fake sig that we are going to strip next
val bin = Codecs.bolt11DataCodec.encode(data).require
val message: BinaryData = hrp ++ bin.dropRight(520).toByteArray
val message = ByteVector.view(hrp) ++ bin.dropRight(520).toByteVector
Crypto.sha256(message)
}
@ -99,7 +99,7 @@ case class PaymentRequest(prefix: String, amount: Option[MilliSatoshi], timestam
val (r, s) = Crypto.sign(hash, priv)
val (pub1, pub2) = Crypto.recoverPublicKey((r, s), hash)
val recid = if (nodeId == pub1) 0.toByte else 1.toByte
val signature = Crypto.fixSize(r.toByteArray.dropWhile(_ == 0.toByte)) ++ Crypto.fixSize(s.toByteArray.dropWhile(_ == 0.toByte)) :+ recid
val signature = Crypto.fixSize(ByteVector.view(r.toByteArray.dropWhile(_ == 0.toByte))) ++ Crypto.fixSize(ByteVector.view(s.toByteArray.dropWhile(_ == 0.toByte))) :+ recid
this.copy(signature = signature)
}
}
@ -114,7 +114,7 @@ object PaymentRequest {
Block.TestnetGenesisBlock.hash -> "lntb",
Block.LivenetGenesisBlock.hash -> "lnbc")
def apply(chainHash: BinaryData, amount: Option[MilliSatoshi], paymentHash: BinaryData, privateKey: PrivateKey,
def apply(chainHash: ByteVector32, amount: Option[MilliSatoshi], paymentHash: ByteVector32, privateKey: PrivateKey,
description: String, fallbackAddress: Option[String] = None, expirySeconds: Option[Long] = None,
extraHops: List[List[ExtraHop]] = Nil, timestamp: Long = System.currentTimeMillis() / 1000L): PaymentRequest = {
@ -131,11 +131,11 @@ object PaymentRequest {
fallbackAddress.map(FallbackAddress(_)),
expirySeconds.map(Expiry(_))
).flatten ++ extraHops.map(RoutingInfo(_)),
signature = BinaryData.empty)
signature = ByteVector.empty)
.sign(privateKey)
}
case class Bolt11Data(timestamp: Long, taggedFields: List[TaggedField], signature: BinaryData)
case class Bolt11Data(timestamp: Long, taggedFields: List[TaggedField], signature: ByteVector)
sealed trait TaggedField
@ -175,7 +175,7 @@ object PaymentRequest {
*
* @param hash payment hash
*/
case class PaymentHash(hash: BinaryData) extends TaggedField
case class PaymentHash(hash: ByteVector32) extends TaggedField
/**
* Description
@ -190,7 +190,7 @@ object PaymentRequest {
* @param hash hash that will be included in the payment request, and can be checked against the hash of a
* long description, an invoice, ...
*/
case class DescriptionHash(hash: BinaryData) extends TaggedField
case class DescriptionHash(hash: ByteVector32) extends TaggedField
/**
* Fallback Payment that specifies a fallback payment address to be used if LN payment cannot be processed
@ -208,8 +208,6 @@ object PaymentRequest {
Try(fromBase58Address(address)).orElse(Try(fromBech32Address(address))).get
}
def apply(version: Byte, data: BinaryData): FallbackAddress = FallbackAddress(version, ByteVector(data.toArray))
def fromBase58Address(address: String): FallbackAddress = {
val (prefix, hash) = Base58Check.decode(address)
prefix match {
@ -226,7 +224,7 @@ object PaymentRequest {
}
def toAddress(f: FallbackAddress, prefix: String): String = {
val data = BinaryData(f.data.toArray)
import f.data
f.version match {
case 17 if prefix == "lnbc" => Base58Check.encode(Base58.Prefix.PubkeyAddress, data)
case 18 if prefix == "lnbc" => Base58Check.encode(Base58.Prefix.ScriptAddress, data)
@ -331,7 +329,7 @@ object PaymentRequest {
val taggedFieldCodec: Codec[TaggedField] = discriminated[TaggedField].by(ubyte(5))
.typecase(0, dataCodec(bits).as[UnknownTag0])
.typecase(1, dataCodec(binarydata(32)).as[PaymentHash])
.typecase(1, dataCodec(bytes32).as[PaymentHash])
.typecase(2, dataCodec(bits).as[UnknownTag2])
.typecase(3, dataCodec(listOfN(extraHopsLengthCodec, extraHopCodec)).as[RoutingInfo])
.typecase(4, dataCodec(bits).as[UnknownTag4])
@ -353,7 +351,7 @@ object PaymentRequest {
.typecase(20, dataCodec(bits).as[UnknownTag20])
.typecase(21, dataCodec(bits).as[UnknownTag21])
.typecase(22, dataCodec(bits).as[UnknownTag22])
.typecase(23, dataCodec(binarydata(32)).as[DescriptionHash])
.typecase(23, dataCodec(bytes32).as[DescriptionHash])
.typecase(24, dataCodec(bits).as[MinFinalCltvExpiry])
.typecase(25, dataCodec(bits).as[UnknownTag25])
.typecase(26, dataCodec(bits).as[UnknownTag26])
@ -374,7 +372,7 @@ object PaymentRequest {
val bolt11DataCodec: Codec[Bolt11Data] = (
("timestamp" | ulong(35)) ::
("taggedFields" | fixedSizeTrailingCodec(list(taggedFieldCodec), 520)) ::
("signature" | binarydata(65))
("signature" | bytes(65))
).as[Bolt11Data]
}
@ -436,7 +434,7 @@ object PaymentRequest {
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 message: BinaryData = hrp.getBytes ++ data.dropRight(520).toByteArray // we drop the sig bytes
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 pub = if (recid % 2 != 0) pub2 else pub1
@ -464,8 +462,7 @@ object PaymentRequest {
val hrp = s"${pr.prefix}$hramount"
val data = Codecs.bolt11DataCodec.encode(Bolt11Data(pr.timestamp, pr.tags, pr.signature)).require
val int5s = eight2fiveCodec.decode(data).require.value
val checksum = Bech32.checksum(hrp, int5s)
hrp + "1" + (int5s ++ checksum).map(Bech32.pam).mkString
Bech32.encode(hrp, int5s.toArray)
}
}

View file

@ -19,14 +19,13 @@ package fr.acinq.eclair.payment
import akka.actor.{Actor, ActorLogging, ActorRef, Props, Status}
import akka.event.LoggingAdapter
import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey}
import fr.acinq.bitcoin.{BinaryData, Crypto, MilliSatoshi}
import fr.acinq.eclair.nodeFee
import fr.acinq.bitcoin.{ByteVector32, Crypto, MilliSatoshi}
import fr.acinq.eclair.channel._
import fr.acinq.eclair.crypto.Sphinx
import fr.acinq.eclair.payment.PaymentLifecycle.{PaymentFailed, PaymentSucceeded}
import fr.acinq.eclair.router.Announcements
import fr.acinq.eclair.wire._
import fr.acinq.eclair.{NodeParams, ShortChannelId}
import fr.acinq.eclair.{NodeParams, ShortChannelId, nodeFee}
import scodec.bits.BitVector
import scodec.{Attempt, DecodeResult}
@ -37,7 +36,7 @@ import scala.util.{Failure, Success, Try}
sealed trait Origin
case class Local(sender: Option[ActorRef]) extends Origin // we don't persist reference to local actors
case class Relayed(originChannelId: BinaryData, originHtlcId: Long, amountMsatIn: Long, amountMsatOut: Long) extends Origin
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
@ -217,7 +216,7 @@ object Relayer {
.parsePacket(privateKey, add.paymentHash, add.onionRoutingPacket)
.flatMap {
case Sphinx.ParsedPacket(payload, nextPacket, _) =>
LightningMessageCodecs.perHopPayloadCodec.decode(BitVector(payload.data)) match {
LightningMessageCodecs.perHopPayloadCodec.decode(BitVector(payload)) match {
case Attempt.Successful(DecodeResult(perHopPayload, _)) if nextPacket.isLastPacket =>
Success(FinalPayload(add, perHopPayload))
case Attempt.Successful(DecodeResult(perHopPayload, _)) =>

View file

@ -16,13 +16,11 @@
package fr.acinq.eclair.router
import java.net.InetSocketAddress
import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey, sha256, verifySignature}
import fr.acinq.bitcoin.{BinaryData, Crypto, LexicographicalOrdering}
import fr.acinq.bitcoin.{ByteVector32, Crypto, LexicographicalOrdering}
import fr.acinq.eclair.wire._
import fr.acinq.eclair.{ShortChannelId, serializationResult}
import scodec.bits.BitVector
import scodec.bits.{BitVector, ByteVector}
import shapeless.HNil
import scala.compat.Platform
@ -33,17 +31,17 @@ import scala.compat.Platform
*/
object Announcements {
def channelAnnouncementWitnessEncode(chainHash: BinaryData, shortChannelId: ShortChannelId, nodeId1: PublicKey, nodeId2: PublicKey, bitcoinKey1: PublicKey, bitcoinKey2: PublicKey, features: BinaryData): BinaryData =
def channelAnnouncementWitnessEncode(chainHash: ByteVector32, shortChannelId: ShortChannelId, nodeId1: PublicKey, nodeId2: PublicKey, bitcoinKey1: PublicKey, bitcoinKey2: PublicKey, features: ByteVector): ByteVector =
sha256(sha256(serializationResult(LightningMessageCodecs.channelAnnouncementWitnessCodec.encode(features :: chainHash :: shortChannelId :: nodeId1 :: nodeId2 :: bitcoinKey1 :: bitcoinKey2 :: HNil))))
def nodeAnnouncementWitnessEncode(timestamp: Long, nodeId: PublicKey, rgbColor: Color, alias: String, features: BinaryData, addresses: List[NodeAddress]): BinaryData =
def nodeAnnouncementWitnessEncode(timestamp: Long, nodeId: PublicKey, rgbColor: Color, alias: String, features: ByteVector, addresses: List[NodeAddress]): ByteVector =
sha256(sha256(serializationResult(LightningMessageCodecs.nodeAnnouncementWitnessCodec.encode(features :: timestamp :: nodeId :: rgbColor :: alias :: addresses :: HNil))))
def channelUpdateWitnessEncode(chainHash: BinaryData, shortChannelId: ShortChannelId, timestamp: Long, messageFlags: Byte, channelFlags: Byte, cltvExpiryDelta: Int, htlcMinimumMsat: Long, feeBaseMsat: Long, feeProportionalMillionths: Long, htlcMaximumMsat: Option[Long]): BinaryData =
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: BinaryData, shortChannelId: ShortChannelId, localNodeSecret: PrivateKey, remoteNodeId: PublicKey, localFundingPrivKey: PrivateKey, remoteFundingKey: PublicKey, features: BinaryData): (BinaryData, BinaryData) = {
val witness = if (isNode1(localNodeSecret.publicKey.toBin, remoteNodeId.toBin)) {
def signChannelAnnouncement(chainHash: ByteVector32, shortChannelId: ShortChannelId, localNodeSecret: PrivateKey, remoteNodeId: PublicKey, localFundingPrivKey: PrivateKey, remoteFundingKey: PublicKey, features: ByteVector): (ByteVector, ByteVector) = {
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)
@ -53,9 +51,9 @@ object Announcements {
(nodeSig, bitcoinSig)
}
def makeChannelAnnouncement(chainHash: BinaryData, shortChannelId: ShortChannelId, localNodeId: PublicKey, remoteNodeId: PublicKey, localFundingKey: PublicKey, remoteFundingKey: PublicKey, localNodeSignature: BinaryData, remoteNodeSignature: BinaryData, localBitcoinSignature: BinaryData, remoteBitcoinSignature: BinaryData): ChannelAnnouncement = {
def makeChannelAnnouncement(chainHash: ByteVector32, shortChannelId: ShortChannelId, localNodeId: PublicKey, remoteNodeId: PublicKey, localFundingKey: PublicKey, remoteFundingKey: PublicKey, localNodeSignature: ByteVector, remoteNodeSignature: ByteVector, localBitcoinSignature: ByteVector, remoteBitcoinSignature: ByteVector): ChannelAnnouncement = {
val (nodeId1, nodeId2, bitcoinKey1, bitcoinKey2, nodeSignature1, nodeSignature2, bitcoinSignature1, bitcoinSignature2) =
if (isNode1(localNodeId.toBin, remoteNodeId.toBin)) {
if (isNode1(localNodeId, remoteNodeId)) {
(localNodeId, remoteNodeId, localFundingKey, remoteFundingKey, localNodeSignature, remoteNodeSignature, localBitcoinSignature, remoteBitcoinSignature)
} else {
(remoteNodeId, localNodeId, remoteFundingKey, localFundingKey, remoteNodeSignature, localNodeSignature, remoteBitcoinSignature, localBitcoinSignature)
@ -70,14 +68,14 @@ object Announcements {
nodeId2 = nodeId2,
bitcoinKey1 = bitcoinKey1,
bitcoinKey2 = bitcoinKey2,
features = BinaryData.empty,
features = ByteVector.empty,
chainHash = chainHash
)
}
def makeNodeAnnouncement(nodeSecret: PrivateKey, alias: String, color: Color, nodeAddresses: List[NodeAddress], timestamp: Long = Platform.currentTime / 1000): NodeAnnouncement = {
require(alias.size <= 32)
val witness = nodeAnnouncementWitnessEncode(timestamp, nodeSecret.publicKey, color, alias, "", nodeAddresses)
val witness = nodeAnnouncementWitnessEncode(timestamp, nodeSecret.publicKey, color, alias, ByteVector.empty, nodeAddresses)
val sig = Crypto.encodeSignature(Crypto.sign(witness, nodeSecret)) :+ 1.toByte
NodeAnnouncement(
signature = sig,
@ -85,7 +83,7 @@ object Announcements {
nodeId = nodeSecret.publicKey,
rgbColor = color,
alias = alias,
features = "",
features = ByteVector.empty,
addresses = nodeAddresses
)
}
@ -98,7 +96,7 @@ object Announcements {
*
* @return true if localNodeId is node1
*/
def isNode1(localNodeId: BinaryData, remoteNodeId: BinaryData) = LexicographicalOrdering.isLessThan(localNodeId, remoteNodeId)
def isNode1(localNodeId: PublicKey, remoteNodeId: PublicKey) = LexicographicalOrdering.isLessThan(localNodeId, remoteNodeId)
/**
* BOLT 7:
@ -121,9 +119,9 @@ object Announcements {
def makeChannelFlags(isNode1: Boolean, enable: Boolean): Byte = BitVector.bits(!enable :: !isNode1 :: Nil).padLeft(8).toByte()
def makeChannelUpdate(chainHash: BinaryData, nodeSecret: PrivateKey, remoteNodeId: PublicKey, shortChannelId: ShortChannelId, cltvExpiryDelta: Int, htlcMinimumMsat: Long, feeBaseMsat: Long, feeProportionalMillionths: Long, htlcMaximumMsat: Long, enable: Boolean = true, timestamp: Long = Platform.currentTime / 1000): ChannelUpdate = {
def makeChannelUpdate(chainHash: ByteVector32, nodeSecret: PrivateKey, remoteNodeId: PublicKey, shortChannelId: ShortChannelId, cltvExpiryDelta: Int, htlcMinimumMsat: Long, feeBaseMsat: Long, feeProportionalMillionths: Long, htlcMaximumMsat: Long, enable: Boolean = true, timestamp: Long = Platform.currentTime / 1000): ChannelUpdate = {
val messageFlags = makeMessageFlags(hasOptionChannelHtlcMax = true) // NB: we always support option_channel_htlc_max
val channelFlags = makeChannelFlags(isNode1 = isNode1(nodeSecret.publicKey.toBin, remoteNodeId.toBin), enable = enable)
val channelFlags = makeChannelFlags(isNode1 = isNode1(nodeSecret.publicKey, remoteNodeId), enable = enable)
val htlcMaximumMsatOpt = Some(htlcMaximumMsat)
val witness = channelUpdateWitnessEncode(chainHash, shortChannelId, timestamp, messageFlags, channelFlags, cltvExpiryDelta, htlcMinimumMsat, feeBaseMsat, feeProportionalMillionths, htlcMaximumMsatOpt)

View file

@ -20,8 +20,9 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream, InputStream}
import java.nio.ByteOrder
import java.util.zip.{DeflaterOutputStream, GZIPInputStream, GZIPOutputStream, InflaterInputStream}
import fr.acinq.bitcoin.{BinaryData, Protocol}
import fr.acinq.bitcoin.Protocol
import fr.acinq.eclair.ShortChannelId
import scodec.bits.ByteVector
import scala.annotation.tailrec
import scala.collection.SortedSet
@ -31,7 +32,7 @@ object ChannelRangeQueries {
val UNCOMPRESSED_FORMAT = 0.toByte
val ZLIB_FORMAT = 1.toByte
case class ShortChannelIdsBlock(val firstBlock: Long, val numBlocks: Long, shortChannelIds: BinaryData)
case class ShortChannelIdsBlock(val firstBlock: Long, val numBlocks: Long, shortChannelIds: ByteVector)
/**
* Compressed a sequence of *sorted* short channel id.
@ -42,7 +43,7 @@ object ChannelRangeQueries {
def encodeShortChannelIds(firstBlockIn: Long, numBlocksIn: Long, shortChannelIds: SortedSet[ShortChannelId], format: Byte, useGzip: Boolean = false): List[ShortChannelIdsBlock] = {
if (shortChannelIds.isEmpty) {
// special case: reply with an "empty" block
List(ShortChannelIdsBlock(firstBlockIn, numBlocksIn, BinaryData("00")))
List(ShortChannelIdsBlock(firstBlockIn, numBlocksIn, ByteVector(0)))
} else {
// LN messages must fit in 65 Kb so we split ids into groups to make sure that the output message will be valid
val count = format match {
@ -61,7 +62,7 @@ object ChannelRangeQueries {
}
}
def encodeShortChannelIdsSingle(shortChannelIds: Iterable[ShortChannelId], format: Byte, useGzip: Boolean): BinaryData = {
def encodeShortChannelIdsSingle(shortChannelIds: Iterable[ShortChannelId], format: Byte, useGzip: Boolean): ByteVector = {
val bos = new ByteArrayOutputStream()
bos.write(format)
format match {
@ -72,7 +73,7 @@ object ChannelRangeQueries {
shortChannelIds.foreach(id => Protocol.writeUInt64(id.toLong, output, ByteOrder.BIG_ENDIAN))
output.finish()
}
bos.toByteArray
ByteVector.view(bos.toByteArray)
}
/**
@ -81,7 +82,7 @@ object ChannelRangeQueries {
* @param data
* @return a sorted set of short channel ids
*/
def decodeShortChannelIds(data: BinaryData): (Byte, SortedSet[ShortChannelId], Boolean) = {
def decodeShortChannelIds(data: ByteVector): (Byte, SortedSet[ShortChannelId], Boolean) = {
val format = data.head
if (data.tail.isEmpty) (format, SortedSet.empty[ShortChannelId], false) else {
val buffer = new Array[Byte](8)

View file

@ -17,12 +17,13 @@
package fr.acinq.eclair.router
import fr.acinq.bitcoin.Crypto.PublicKey
import scala.collection.mutable
import fr.acinq.eclair._
import fr.acinq.eclair.wire.ChannelUpdate
import Router._
import fr.acinq.eclair.channel.Channel
import fr.acinq.eclair.router.Graph.GraphStructure.{DirectedGraph, GraphEdge}
import fr.acinq.eclair.router.Router._
import fr.acinq.eclair.wire.ChannelUpdate
import scala.collection.mutable
object Graph {

View file

@ -19,6 +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.Crypto.PublicKey
import fr.acinq.bitcoin.Script.{pay2wsh, write}
import fr.acinq.eclair._
@ -32,8 +33,10 @@ import fr.acinq.eclair.router.Graph.GraphStructure.{DirectedGraph, GraphEdge}
import fr.acinq.eclair.router.Graph.{RichWeight, WeightRatios}
import fr.acinq.eclair.transactions.Scripts
import fr.acinq.eclair.wire._
import scala.collection.{SortedSet, mutable}
import scodec.bits.ByteVector
import scala.collection.immutable.{SortedMap, TreeMap}
import scala.collection.{SortedSet, mutable}
import scala.compat.Platform
import scala.concurrent.duration._
import scala.concurrent.{ExecutionContext, Promise}
@ -735,7 +738,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 = "", chainHash = "", extraHop.shortChannelId, Platform.currentTime / 1000, messageFlags = 0, channelFlags = 0, extraHop.cltvExpiryDelta, htlcMinimumMsat = 0L, extraHop.feeBaseMsat, extraHop.feeProportionalMillionths, None)
ChannelUpdate(signature = ByteVector.empty, chainHash = ByteVector32.Zeroes, extraHop.shortChannelId, Platform.currentTime / 1000, 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

View file

@ -22,7 +22,7 @@ import akka.actor.{Actor, ActorLogging, OneForOneStrategy, Props, SupervisorStra
import akka.io.{IO, Tcp}
import akka.util.ByteString
import scala.concurrent.{ExecutionContext, Promise}
import scala.concurrent.ExecutionContext
/**
* Created by rorp

View file

@ -21,12 +21,10 @@ import java.net.{Inet4Address, Inet6Address, InetAddress, InetSocketAddress}
import akka.actor.{Actor, ActorLogging, ActorRef, Props, Terminated}
import akka.io.Tcp
import akka.util.ByteString
import fr.acinq.bitcoin.toHexString
import fr.acinq.eclair.randomBytes
import fr.acinq.eclair.tor.Socks5Connection.{Credentials, Socks5Connect}
import fr.acinq.eclair.wire._
import scala.util.Success
/**
* Simple socks 5 client. It should be given a new connection, and will
@ -234,7 +232,7 @@ object Socks5ProxyParams {
def proxyCredentials(proxyParams: Socks5ProxyParams): Option[Socks5Connection.Credentials] =
if (proxyParams.randomizeCredentials) {
// randomize credentials for every proxy connection to enable Tor stream isolation
Some(Socks5Connection.Credentials(toHexString(randomBytes(16)), toHexString(randomBytes(16))))
Some(Socks5Connection.Credentials(randomBytes(16).toHex, randomBytes(16).toHex))
} else {
proxyParams.credentials_opt
}

View file

@ -16,7 +16,6 @@
package fr.acinq.eclair.tor
import java.net.InetSocketAddress
import java.nio.file.attribute.PosixFilePermissions
import java.nio.file.{Files, Path, Paths}
import java.util
@ -24,11 +23,12 @@ import java.util
import akka.actor.{Actor, ActorLogging, ActorRef, Props, Stash}
import akka.io.Tcp.Connected
import akka.util.ByteString
import fr.acinq.bitcoin.BinaryData
import fr.acinq.eclair.tor.TorProtocolHandler.{Authentication, OnionServiceVersion}
import fr.acinq.eclair.wire.{NodeAddress, Tor2, Tor3}
import javax.crypto.Mac
import javax.crypto.spec.SecretKeySpec
import scodec.bits.Bases.Alphabets
import scodec.bits.ByteVector
import scala.concurrent.Promise
import scala.util.Try
@ -86,21 +86,21 @@ class TorProtocolHandler(onionServiceVersion: OnionServiceVersion,
context become authenticate
case SafeCookie(nonce) =>
val cookieFile = Paths.get(unquote(res.getOrElse("COOKIEFILE", throw TorException("cookie file not found"))))
sendCommand(s"AUTHCHALLENGE SAFECOOKIE $nonce")
sendCommand(s"AUTHCHALLENGE SAFECOOKIE ${nonce.toHex}")
context become cookieChallenge(cookieFile, nonce)
}
}
def cookieChallenge(cookieFile: Path, clientNonce: BinaryData): Receive = {
def cookieChallenge(cookieFile: Path, clientNonce: ByteVector): Receive = {
case data: ByteString =>
val res = parseResponse(readResponse(data))
val clientHash = computeClientHash(
res.getOrElse("SERVERHASH", throw TorException("server hash not found")),
res.getOrElse("SERVERNONCE", throw TorException("server nonce not found")),
ByteVector.fromValidHex(res.getOrElse("SERVERHASH", throw TorException("server hash not found")).toLowerCase, alphabet = Alphabets.HexLowercase),
ByteVector.fromValidHex(res.getOrElse("SERVERNONCE", throw TorException("server nonce not found")).toLowerCase, alphabet = Alphabets.HexLowercase),
clientNonce,
cookieFile
)
sendCommand(s"AUTHENTICATE $clientHash")
sendCommand(s"AUTHENTICATE ${clientHash.toHex}")
context become authenticate
}
@ -166,13 +166,13 @@ class TorProtocolHandler(onionServiceVersion: OnionServiceVersion,
}
}
private def computeClientHash(serverHash: BinaryData, serverNonce: BinaryData, clientNonce: BinaryData, cookieFile: Path): BinaryData = {
private def computeClientHash(serverHash: ByteVector, serverNonce: ByteVector, clientNonce: ByteVector, cookieFile: Path): ByteVector = {
if (serverHash.length != 32)
throw TorException("invalid server hash length")
if (serverNonce.length != 32)
throw TorException("invalid server nonce length")
val cookie = Files.readAllBytes(cookieFile)
val cookie = ByteVector.view(Files.readAllBytes(cookieFile))
val message = cookie ++ clientNonce ++ serverNonce
@ -200,8 +200,8 @@ object TorProtocolHandler {
Props(new TorProtocolHandler(version, authentication, privateKeyPath, virtualPort, targetPorts, onionAdded))
// those are defined in the spec
private val ServerKey: Array[Byte] = "Tor safe cookie authentication server-to-controller hash".getBytes()
private val ClientKey: Array[Byte] = "Tor safe cookie authentication controller-to-server hash".getBytes()
private val ServerKey = ByteVector.view("Tor safe cookie authentication server-to-controller hash".getBytes())
private val ClientKey = ByteVector.view("Tor safe cookie authentication controller-to-server hash".getBytes())
// @formatter:off
sealed trait OnionServiceVersion
@ -236,7 +236,7 @@ object TorProtocolHandler {
// @formatter:off
sealed trait Authentication
case class Password(password: String) extends Authentication { override def toString = "password" }
case class SafeCookie(nonce: BinaryData = fr.acinq.eclair.randomBytes(32)) extends Authentication { override def toString = "safecookie" }
case class SafeCookie(nonce: ByteVector = fr.acinq.eclair.randomBytes32) extends Authentication { override def toString = "safecookie" }
// @formatter:on
object Authentication {
@ -302,10 +302,10 @@ object TorProtocolHandler {
}.toMap
}
def hmacSHA256(key: Array[Byte], message: Array[Byte]): BinaryData = {
def hmacSHA256(key: ByteVector, message: ByteVector): ByteVector = {
val mac = Mac.getInstance("HmacSHA256")
val secretKey = new SecretKeySpec(key, "HmacSHA256")
val secretKey = new SecretKeySpec(key.toArray, "HmacSHA256")
mac.init(secretKey)
mac.doFinal(message)
ByteVector.view(mac.doFinal(message.toArray))
}
}

View file

@ -18,7 +18,8 @@ package fr.acinq.eclair.transactions
import fr.acinq.bitcoin.Crypto.{PublicKey, ripemd160}
import fr.acinq.bitcoin.Script._
import fr.acinq.bitcoin.{BinaryData, 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, 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
/**
* Created by PM on 02/12/2016.
@ -52,11 +53,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: BinaryData, sig2: BinaryData, pubkey1: PublicKey, pubkey2: PublicKey): ScriptWitness = {
def witness2of2(sig1: ByteVector, sig2: ByteVector, pubkey1: PublicKey, pubkey2: PublicKey): ScriptWitness = {
if (LexicographicalOrdering.isLessThan(pubkey1.toBin, pubkey2.toBin))
ScriptWitness(Seq(BinaryData.empty, sig1, sig2, write(multiSig2of2(pubkey1, pubkey2))))
ScriptWitness(Seq(ByteVector.empty, sig1, sig2, write(multiSig2of2(pubkey1, pubkey2))))
else
ScriptWitness(Seq(BinaryData.empty, sig2, sig1, write(multiSig2of2(pubkey1, pubkey2))))
ScriptWitness(Seq(ByteVector.empty, sig2, sig1, write(multiSig2of2(pubkey1, pubkey2))))
}
@ -75,7 +76,7 @@ object Scripts {
case _ => OP_PUSHDATA(Script.encodeNumber(n))
}
def redeemSecretOrDelay(delayedKey: BinaryData, reltimeout: Long, keyIfSecretKnown: BinaryData, hashOfSecret: BinaryData): Seq[ScriptElt] = {
def redeemSecretOrDelay(delayedKey: ByteVector, reltimeout: Long, keyIfSecretKnown: ByteVector, hashOfSecret: ByteVector32): Seq[ScriptElt] = {
// @formatter:off
OP_HASH160 :: OP_PUSHDATA(ripemd160(hashOfSecret)) :: OP_EQUAL ::
OP_IF ::
@ -87,7 +88,7 @@ object Scripts {
// @formatter:on
}
def scriptPubKeyHtlcSend(ourkey: BinaryData, theirkey: BinaryData, abstimeout: Long, reltimeout: Long, rhash: BinaryData, commit_revoke: BinaryData): Seq[ScriptElt] = {
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
@ -104,7 +105,7 @@ object Scripts {
// @formatter:on
}
def scriptPubKeyHtlcReceive(ourkey: BinaryData, theirkey: BinaryData, abstimeout: Long, reltimeout: Long, rhash: BinaryData, commit_revoke: BinaryData): Seq[ScriptElt] = {
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
@ -187,17 +188,17 @@ object Scripts {
/**
* This witness script spends a [[toLocalDelayed]] output using a local sig after a delay
*/
def witnessToLocalDelayedAfterDelay(localSig: BinaryData, toLocalDelayedScript: BinaryData) =
ScriptWitness(localSig :: BinaryData.empty :: toLocalDelayedScript :: Nil)
def witnessToLocalDelayedAfterDelay(localSig: ByteVector, toLocalDelayedScript: ByteVector) =
ScriptWitness(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: BinaryData, toLocalScript: BinaryData) =
ScriptWitness(revocationSig :: BinaryData("01") :: toLocalScript :: Nil)
def witnessToLocalDelayedWithRevocationSig(revocationSig: ByteVector, toLocalScript: ByteVector) =
ScriptWitness(revocationSig :: ByteVector(1) :: toLocalScript :: Nil)
def htlcOffered(localHtlcPubkey: PublicKey, remoteHtlcPubkey: PublicKey, revocationPubKey: PublicKey, paymentHash: BinaryData): Seq[ScriptElt] = {
def htlcOffered(localHtlcPubkey: PublicKey, remoteHtlcPubkey: PublicKey, revocationPubKey: PublicKey, paymentHash: ByteVector): Seq[ScriptElt] = {
// @formatter:off
// To you with revocation key
OP_DUP :: OP_HASH160 :: OP_PUSHDATA(revocationPubKey.hash160) :: OP_EQUAL ::
@ -219,17 +220,17 @@ object Scripts {
/**
* This is the witness script of the 2nd-stage HTLC Success transaction (consumes htlcOffered script from commit tx)
*/
def witnessHtlcSuccess(localSig: BinaryData, remoteSig: BinaryData, paymentPreimage: BinaryData, htlcOfferedScript: BinaryData) =
ScriptWitness(BinaryData.empty :: remoteSig :: localSig :: paymentPreimage :: htlcOfferedScript :: Nil)
def witnessHtlcSuccess(localSig: ByteVector, remoteSig: ByteVector, paymentPreimage: ByteVector32, htlcOfferedScript: ByteVector) =
ScriptWitness(ByteVector.empty :: remoteSig :: 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: BinaryData, paymentPreimage: BinaryData, htlcOfferedScript: BinaryData) =
ScriptWitness(localSig :: paymentPreimage :: htlcOfferedScript :: Nil)
def witnessClaimHtlcSuccessFromCommitTx(localSig: ByteVector, paymentPreimage: ByteVector32, htlcOfferedScript: ByteVector) =
ScriptWitness(localSig :: paymentPreimage.bytes :: htlcOfferedScript :: Nil)
def htlcReceived(localHtlcPubkey: PublicKey, remoteHtlcPubkey: PublicKey, revocationPubKey: PublicKey, paymentHash: BinaryData, lockTime: Long) = {
def htlcReceived(localHtlcPubkey: PublicKey, remoteHtlcPubkey: PublicKey, revocationPubKey: PublicKey, paymentHash: ByteVector, lockTime: Long) = {
// @formatter:off
// To you with revocation key
OP_DUP :: OP_HASH160 :: OP_PUSHDATA(revocationPubKey.hash160) :: OP_EQUAL ::
@ -253,21 +254,21 @@ object Scripts {
/**
* This is the witness script of the 2nd-stage HTLC Timeout transaction (consumes htlcReceived script from commit tx)
*/
def witnessHtlcTimeout(localSig: BinaryData, remoteSig: BinaryData, htlcReceivedScript: BinaryData) =
ScriptWitness(BinaryData.empty :: remoteSig :: localSig :: BinaryData.empty :: htlcReceivedScript :: Nil)
def witnessHtlcTimeout(localSig: ByteVector, remoteSig: ByteVector, htlcReceivedScript: ByteVector) =
ScriptWitness(ByteVector.empty :: remoteSig :: 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: BinaryData, htlcReceivedScript: BinaryData) =
ScriptWitness(localSig :: BinaryData.empty :: htlcReceivedScript :: Nil)
def witnessClaimHtlcTimeoutFromCommitTx(localSig: ByteVector, htlcReceivedScript: ByteVector) =
ScriptWitness(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: BinaryData, revocationPubkey: PublicKey, htlcScript: BinaryData) =
def witnessHtlcWithRevocationSig(revocationSig: ByteVector, revocationPubkey: PublicKey, htlcScript: ByteVector) =
ScriptWitness(revocationSig :: revocationPubkey.toBin :: htlcScript :: Nil)
}

View file

@ -21,9 +21,10 @@ import java.nio.ByteOrder
import fr.acinq.bitcoin.Crypto.{Point, PrivateKey, PublicKey, ripemd160}
import fr.acinq.bitcoin.Script._
import fr.acinq.bitcoin.SigVersion._
import fr.acinq.bitcoin.{BinaryData, Crypto, LexicographicalOrdering, MilliSatoshi, OutPoint, Protocol, SIGHASH_ALL, Satoshi, Script, ScriptElt, ScriptFlags, ScriptWitness, Transaction, TxIn, TxOut, millisatoshi2satoshi}
import fr.acinq.bitcoin.{ByteVector32, 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
import scala.util.Try
@ -33,7 +34,7 @@ import scala.util.Try
object Transactions {
// @formatter:off
case class InputInfo(outPoint: OutPoint, txOut: TxOut, redeemScript: BinaryData)
case class InputInfo(outPoint: OutPoint, txOut: TxOut, redeemScript: ByteVector)
object InputInfo {
def apply(outPoint: OutPoint, txOut: TxOut, redeemScript: Seq[ScriptElt]) = new InputInfo(outPoint, txOut, Script.write(redeemScript))
}
@ -49,7 +50,7 @@ object Transactions {
}
case class CommitTx(input: InputInfo, tx: Transaction) extends TransactionWithInputInfo
case class HtlcSuccessTx(input: InputInfo, tx: Transaction, paymentHash: BinaryData) extends TransactionWithInputInfo
case class HtlcSuccessTx(input: InputInfo, tx: Transaction, paymentHash: ByteVector32) extends TransactionWithInputInfo
case class HtlcTimeoutTx(input: InputInfo, tx: Transaction) extends TransactionWithInputInfo
case class ClaimHtlcSuccessTx(input: InputInfo, tx: Transaction) extends TransactionWithInputInfo
case class ClaimHtlcTimeoutTx(input: InputInfo, tx: Transaction) extends TransactionWithInputInfo
@ -154,7 +155,7 @@ object Transactions {
else
Crypto.sha256(remotePaymentBasePoint.toBin(true) ++ localPaymentBasePoint.toBin(true))
val blind = Protocol.uint64(h.takeRight(6).reverse ++ BinaryData("0x0000"), ByteOrder.LITTLE_ENDIAN)
val blind = Protocol.uint64((h.takeRight(6).reverse ++ ByteVector.fromValidHex("0000")).toArray, ByteOrder.LITTLE_ENDIAN)
commitTxNumber ^ blind
}
@ -198,16 +199,16 @@ object Transactions {
val toRemoteOutput_opt = if (toRemoteAmount >= localDustLimit) Some(TxOut(toRemoteAmount, pay2wpkh(remotePaymentPubkey))) else None
val htlcOfferedOutputs = trimOfferedHtlcs(localDustLimit, spec)
.map(htlc => TxOut(MilliSatoshi(htlc.add.amountMsat), pay2wsh(htlcOffered(localHtlcPubkey, remoteHtlcPubkey, localRevocationPubkey, ripemd160(htlc.add.paymentHash)))))
.map(htlc => TxOut(MilliSatoshi(htlc.add.amountMsat), pay2wsh(htlcOffered(localHtlcPubkey, remoteHtlcPubkey, localRevocationPubkey, ripemd160(htlc.add.paymentHash.bytes)))))
val htlcReceivedOutputs = trimReceivedHtlcs(localDustLimit, spec)
.map(htlc => TxOut(MilliSatoshi(htlc.add.amountMsat), pay2wsh(htlcReceived(localHtlcPubkey, remoteHtlcPubkey, localRevocationPubkey, ripemd160(htlc.add.paymentHash), htlc.add.cltvExpiry))))
.map(htlc => TxOut(MilliSatoshi(htlc.add.amountMsat), pay2wsh(htlcReceived(localHtlcPubkey, remoteHtlcPubkey, localRevocationPubkey, ripemd160(htlc.add.paymentHash.bytes), htlc.add.cltvExpiry))))
val txnumber = obscuredCommitTxNumber(commitTxNumber, localIsFunder, localPaymentBasePoint, remotePaymentBasePoint)
val (sequence, locktime) = encodeTxNumber(txnumber)
val tx = Transaction(
version = 2,
txIn = TxIn(commitTxInput.outPoint, Array.emptyByteArray, sequence = sequence) :: Nil,
txIn = TxIn(commitTxInput.outPoint, ByteVector.empty, sequence = sequence) :: Nil,
txOut = toLocalDelayedOutput_opt.toSeq ++ toRemoteOutput_opt.toSeq ++ htlcOfferedOutputs ++ htlcReceivedOutputs,
lockTime = locktime)
CommitTx(commitTxInput, LexicographicalOrdering.sort(tx))
@ -215,7 +216,7 @@ object Transactions {
def makeHtlcTimeoutTx(commitTx: Transaction, outputsAlreadyUsed: Set[Int], localDustLimit: Satoshi, localRevocationPubkey: PublicKey, toLocalDelay: Int, localDelayedPaymentPubkey: PublicKey, localHtlcPubkey: PublicKey, remoteHtlcPubkey: PublicKey, feeratePerKw: Long, htlc: UpdateAddHtlc): HtlcTimeoutTx = {
val fee = weight2fee(feeratePerKw, htlcTimeoutWeight)
val redeemScript = htlcOffered(localHtlcPubkey, remoteHtlcPubkey, localRevocationPubkey, ripemd160(htlc.paymentHash))
val redeemScript = htlcOffered(localHtlcPubkey, remoteHtlcPubkey, localRevocationPubkey, ripemd160(htlc.paymentHash.bytes))
val pubkeyScript = write(pay2wsh(redeemScript))
val outputIndex = findPubKeyScriptIndex(commitTx, pubkeyScript, outputsAlreadyUsed, amount_opt = Some(Satoshi(htlc.amountMsat / 1000)))
val amount = MilliSatoshi(htlc.amountMsat) - fee
@ -225,14 +226,14 @@ object Transactions {
val input = InputInfo(OutPoint(commitTx, outputIndex), commitTx.txOut(outputIndex), write(redeemScript))
HtlcTimeoutTx(input, Transaction(
version = 2,
txIn = TxIn(input.outPoint, Array.emptyByteArray, 0x00000000L) :: Nil,
txIn = TxIn(input.outPoint, ByteVector.empty, 0x00000000L) :: Nil,
txOut = TxOut(amount, pay2wsh(toLocalDelayed(localRevocationPubkey, toLocalDelay, localDelayedPaymentPubkey))) :: Nil,
lockTime = htlc.cltvExpiry))
}
def makeHtlcSuccessTx(commitTx: Transaction, outputsAlreadyUsed: Set[Int], localDustLimit: Satoshi, localRevocationPubkey: PublicKey, toLocalDelay: Int, localDelayedPaymentPubkey: PublicKey, localHtlcPubkey: PublicKey, remoteHtlcPubkey: PublicKey, feeratePerKw: Long, htlc: UpdateAddHtlc): HtlcSuccessTx = {
val fee = weight2fee(feeratePerKw, htlcSuccessWeight)
val redeemScript = htlcReceived(localHtlcPubkey, remoteHtlcPubkey, localRevocationPubkey, ripemd160(htlc.paymentHash), htlc.cltvExpiry)
val redeemScript = htlcReceived(localHtlcPubkey, remoteHtlcPubkey, localRevocationPubkey, ripemd160(htlc.paymentHash.bytes), htlc.cltvExpiry)
val pubkeyScript = write(pay2wsh(redeemScript))
val outputIndex = findPubKeyScriptIndex(commitTx, pubkeyScript, outputsAlreadyUsed, amount_opt = Some(Satoshi(htlc.amountMsat / 1000)))
val amount = MilliSatoshi(htlc.amountMsat) - fee
@ -242,7 +243,7 @@ object Transactions {
val input = InputInfo(OutPoint(commitTx, outputIndex), commitTx.txOut(outputIndex), write(redeemScript))
HtlcSuccessTx(input, Transaction(
version = 2,
txIn = TxIn(input.outPoint, Array.emptyByteArray, 0x00000000L) :: Nil,
txIn = TxIn(input.outPoint, ByteVector.empty, 0x00000000L) :: Nil,
txOut = TxOut(amount, pay2wsh(toLocalDelayed(localRevocationPubkey, toLocalDelay, localDelayedPaymentPubkey))) :: Nil,
lockTime = 0), htlc.paymentHash)
}
@ -262,19 +263,19 @@ object Transactions {
(htlcTimeoutTxs, htlcSuccessTxs)
}
def makeClaimHtlcSuccessTx(commitTx: Transaction, outputsAlreadyUsed: Set[Int], localDustLimit: Satoshi, localHtlcPubkey: PublicKey, remoteHtlcPubkey: PublicKey, remoteRevocationPubkey: PublicKey, localFinalScriptPubKey: BinaryData, htlc: UpdateAddHtlc, feeratePerKw: Long): ClaimHtlcSuccessTx = {
val redeemScript = htlcOffered(remoteHtlcPubkey, localHtlcPubkey, remoteRevocationPubkey, ripemd160(htlc.paymentHash))
def makeClaimHtlcSuccessTx(commitTx: Transaction, outputsAlreadyUsed: Set[Int], localDustLimit: Satoshi, localHtlcPubkey: PublicKey, remoteHtlcPubkey: PublicKey, remoteRevocationPubkey: PublicKey, localFinalScriptPubKey: ByteVector, htlc: UpdateAddHtlc, feeratePerKw: Long): ClaimHtlcSuccessTx = {
val redeemScript = htlcOffered(remoteHtlcPubkey, localHtlcPubkey, remoteRevocationPubkey, ripemd160(htlc.paymentHash.bytes))
val pubkeyScript = write(pay2wsh(redeemScript))
val outputIndex = findPubKeyScriptIndex(commitTx, pubkeyScript, outputsAlreadyUsed, amount_opt = Some(Satoshi(htlc.amountMsat / 1000)))
val input = InputInfo(OutPoint(commitTx, outputIndex), commitTx.txOut(outputIndex), write(redeemScript))
val tx = Transaction(
version = 2,
txIn = TxIn(input.outPoint, Array.emptyByteArray, 0xffffffffL) :: Nil,
txIn = TxIn(input.outPoint, ByteVector.empty, 0xffffffffL) :: Nil,
txOut = TxOut(Satoshi(0), localFinalScriptPubKey) :: Nil,
lockTime = 0)
val weight = Transactions.addSigs(ClaimHtlcSuccessTx(input, tx), BinaryData("00" * 73), BinaryData("00" * 32)).tx.weight()
val weight = Transactions.addSigs(ClaimHtlcSuccessTx(input, tx), ByteVector.fill(73)(0), ByteVector32.Zeroes).tx.weight()
val fee = weight2fee(feeratePerKw, weight)
val amount = input.txOut.amount - fee
if (amount < localDustLimit) {
@ -285,8 +286,8 @@ object Transactions {
ClaimHtlcSuccessTx(input, tx1)
}
def makeClaimHtlcTimeoutTx(commitTx: Transaction, outputsAlreadyUsed: Set[Int], localDustLimit: Satoshi, localHtlcPubkey: PublicKey, remoteHtlcPubkey: PublicKey, remoteRevocationPubkey: PublicKey, localFinalScriptPubKey: BinaryData, htlc: UpdateAddHtlc, feeratePerKw: Long): ClaimHtlcTimeoutTx = {
val redeemScript = htlcReceived(remoteHtlcPubkey, localHtlcPubkey, remoteRevocationPubkey, ripemd160(htlc.paymentHash), htlc.cltvExpiry)
def makeClaimHtlcTimeoutTx(commitTx: Transaction, outputsAlreadyUsed: Set[Int], localDustLimit: Satoshi, localHtlcPubkey: PublicKey, remoteHtlcPubkey: PublicKey, remoteRevocationPubkey: PublicKey, localFinalScriptPubKey: ByteVector, htlc: UpdateAddHtlc, feeratePerKw: Long): ClaimHtlcTimeoutTx = {
val redeemScript = htlcReceived(remoteHtlcPubkey, localHtlcPubkey, remoteRevocationPubkey, ripemd160(htlc.paymentHash.bytes), htlc.cltvExpiry)
val pubkeyScript = write(pay2wsh(redeemScript))
val outputIndex = findPubKeyScriptIndex(commitTx, pubkeyScript, outputsAlreadyUsed, amount_opt = Some(Satoshi(htlc.amountMsat / 1000)))
val input = InputInfo(OutPoint(commitTx, outputIndex), commitTx.txOut(outputIndex), write(redeemScript))
@ -294,11 +295,11 @@ object Transactions {
// unsigned tx
val tx = Transaction(
version = 2,
txIn = TxIn(input.outPoint, Array.emptyByteArray, 0x00000000L) :: Nil,
txIn = TxIn(input.outPoint, ByteVector.empty, 0x00000000L) :: Nil,
txOut = TxOut(Satoshi(0), localFinalScriptPubKey) :: Nil,
lockTime = htlc.cltvExpiry)
val weight = Transactions.addSigs(ClaimHtlcTimeoutTx(input, tx), BinaryData("00" * 73)).tx.weight()
val weight = Transactions.addSigs(ClaimHtlcTimeoutTx(input, tx), ByteVector.fill(73)(0)).tx.weight()
val fee = weight2fee(feeratePerKw, weight)
val amount = input.txOut.amount - fee
@ -310,7 +311,7 @@ object Transactions {
ClaimHtlcTimeoutTx(input, tx1)
}
def makeClaimP2WPKHOutputTx(delayedOutputTx: Transaction, localDustLimit: Satoshi, localPaymentPubkey: PublicKey, localFinalScriptPubKey: BinaryData, feeratePerKw: Long): ClaimP2WPKHOutputTx = {
def makeClaimP2WPKHOutputTx(delayedOutputTx: Transaction, localDustLimit: Satoshi, localPaymentPubkey: PublicKey, localFinalScriptPubKey: ByteVector, feeratePerKw: Long): ClaimP2WPKHOutputTx = {
val redeemScript = Script.pay2pkh(localPaymentPubkey)
val pubkeyScript = write(pay2wpkh(localPaymentPubkey))
val outputIndex = findPubKeyScriptIndex(delayedOutputTx, pubkeyScript, outputsAlreadyUsed = Set.empty, amount_opt = None)
@ -319,12 +320,12 @@ object Transactions {
// unsigned tx
val tx = Transaction(
version = 2,
txIn = TxIn(input.outPoint, Array.emptyByteArray, 0x00000000L) :: Nil,
txIn = TxIn(input.outPoint, ByteVector.empty, 0x00000000L) :: Nil,
txOut = TxOut(Satoshi(0), localFinalScriptPubKey) :: Nil,
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), BinaryData("00" * 33), BinaryData("00" * 73)).tx.weight()
val weight = Transactions.addSigs(ClaimP2WPKHOutputTx(input, tx), ByteVector.fill(33)(0), ByteVector.fill(73)(0)).tx.weight()
val fee = weight2fee(feeratePerKw, weight)
val amount = input.txOut.amount - fee
@ -336,7 +337,7 @@ object Transactions {
ClaimP2WPKHOutputTx(input, tx1)
}
def makeClaimDelayedOutputTx(delayedOutputTx: Transaction, localDustLimit: Satoshi, localRevocationPubkey: PublicKey, toLocalDelay: Int, localDelayedPaymentPubkey: PublicKey, localFinalScriptPubKey: BinaryData, feeratePerKw: Long): ClaimDelayedOutputTx = {
def makeClaimDelayedOutputTx(delayedOutputTx: Transaction, localDustLimit: Satoshi, localRevocationPubkey: PublicKey, toLocalDelay: Int, localDelayedPaymentPubkey: PublicKey, localFinalScriptPubKey: ByteVector, feeratePerKw: Long): ClaimDelayedOutputTx = {
val redeemScript = toLocalDelayed(localRevocationPubkey, toLocalDelay, localDelayedPaymentPubkey)
val pubkeyScript = write(pay2wsh(redeemScript))
val outputIndex = findPubKeyScriptIndex(delayedOutputTx, pubkeyScript, outputsAlreadyUsed = Set.empty, amount_opt = None)
@ -345,12 +346,12 @@ object Transactions {
// unsigned transaction
val tx = Transaction(
version = 2,
txIn = TxIn(input.outPoint, Array.emptyByteArray, toLocalDelay) :: Nil,
txIn = TxIn(input.outPoint, ByteVector.empty, toLocalDelay) :: Nil,
txOut = TxOut(Satoshi(0), localFinalScriptPubKey) :: Nil,
lockTime = 0)
// compute weight with a dummy 73 bytes signature (the largest you can get)
val weight = Transactions.addSigs(ClaimDelayedOutputTx(input, tx), BinaryData("00" * 73)).tx.weight()
val weight = Transactions.addSigs(ClaimDelayedOutputTx(input, tx), ByteVector.fill(73)(0)).tx.weight()
val fee = weight2fee(feeratePerKw, weight)
val amount = input.txOut.amount - fee
@ -362,7 +363,7 @@ object Transactions {
ClaimDelayedOutputTx(input, tx1)
}
def makeClaimDelayedOutputPenaltyTx(delayedOutputTx: Transaction, localDustLimit: Satoshi, localRevocationPubkey: PublicKey, toLocalDelay: Int, localDelayedPaymentPubkey: PublicKey, localFinalScriptPubKey: BinaryData, feeratePerKw: Long): ClaimDelayedOutputPenaltyTx = {
def makeClaimDelayedOutputPenaltyTx(delayedOutputTx: Transaction, localDustLimit: Satoshi, localRevocationPubkey: PublicKey, toLocalDelay: Int, localDelayedPaymentPubkey: PublicKey, localFinalScriptPubKey: ByteVector, feeratePerKw: Long): ClaimDelayedOutputPenaltyTx = {
val redeemScript = toLocalDelayed(localRevocationPubkey, toLocalDelay, localDelayedPaymentPubkey)
val pubkeyScript = write(pay2wsh(redeemScript))
val outputIndex = findPubKeyScriptIndex(delayedOutputTx, pubkeyScript, outputsAlreadyUsed = Set.empty, amount_opt = None)
@ -371,12 +372,12 @@ object Transactions {
// unsigned transaction
val tx = Transaction(
version = 2,
txIn = TxIn(input.outPoint, Array.emptyByteArray, 0xffffffffL) :: Nil,
txIn = TxIn(input.outPoint, ByteVector.empty, 0xffffffffL) :: Nil,
txOut = TxOut(Satoshi(0), localFinalScriptPubKey) :: Nil,
lockTime = 0)
// compute weight with a dummy 73 bytes signature (the largest you can get)
val weight = Transactions.addSigs(ClaimDelayedOutputPenaltyTx(input, tx), BinaryData("00" * 73)).tx.weight()
val weight = Transactions.addSigs(ClaimDelayedOutputPenaltyTx(input, tx), ByteVector.fill(73)(0)).tx.weight()
val fee = weight2fee(feeratePerKw, weight)
val amount = input.txOut.amount - fee
@ -388,7 +389,7 @@ object Transactions {
ClaimDelayedOutputPenaltyTx(input, tx1)
}
def makeMainPenaltyTx(commitTx: Transaction, localDustLimit: Satoshi, remoteRevocationPubkey: PublicKey, localFinalScriptPubKey: BinaryData, toRemoteDelay: Int, remoteDelayedPaymentPubkey: PublicKey, feeratePerKw: Long): MainPenaltyTx = {
def makeMainPenaltyTx(commitTx: Transaction, localDustLimit: Satoshi, remoteRevocationPubkey: PublicKey, localFinalScriptPubKey: ByteVector, toRemoteDelay: Int, remoteDelayedPaymentPubkey: PublicKey, feeratePerKw: Long): MainPenaltyTx = {
val redeemScript = toLocalDelayed(remoteRevocationPubkey, toRemoteDelay, remoteDelayedPaymentPubkey)
val pubkeyScript = write(pay2wsh(redeemScript))
val outputIndex = findPubKeyScriptIndex(commitTx, pubkeyScript, outputsAlreadyUsed = Set.empty, amount_opt = None)
@ -397,12 +398,12 @@ object Transactions {
// unsigned transaction
val tx = Transaction(
version = 2,
txIn = TxIn(input.outPoint, Array.emptyByteArray, 0xffffffffL) :: Nil,
txIn = TxIn(input.outPoint, ByteVector.empty, 0xffffffffL) :: Nil,
txOut = TxOut(Satoshi(0), localFinalScriptPubKey) :: Nil,
lockTime = 0)
// compute weight with a dummy 73 bytes signature (the largest you can get)
val weight = Transactions.addSigs(MainPenaltyTx(input, tx), BinaryData("00" * 73)).tx.weight()
val weight = Transactions.addSigs(MainPenaltyTx(input, tx), ByteVector.fill(73)(0)).tx.weight()
val fee = weight2fee(feeratePerKw, weight)
val amount = input.txOut.amount - fee
@ -417,7 +418,7 @@ object Transactions {
/**
* We already have the redeemScript, no need to build it
*/
def makeHtlcPenaltyTx(commitTx: Transaction, outputsAlreadyUsed: Set[Int], redeemScript: BinaryData, localDustLimit: Satoshi, localFinalScriptPubKey: BinaryData, feeratePerKw: Long): HtlcPenaltyTx = {
def makeHtlcPenaltyTx(commitTx: Transaction, outputsAlreadyUsed: Set[Int], redeemScript: ByteVector, localDustLimit: Satoshi, localFinalScriptPubKey: ByteVector, feeratePerKw: Long): HtlcPenaltyTx = {
val pubkeyScript = write(pay2wsh(redeemScript))
val outputIndex = findPubKeyScriptIndex(commitTx, pubkeyScript, outputsAlreadyUsed, amount_opt = None)
val input = InputInfo(OutPoint(commitTx, outputIndex), commitTx.txOut(outputIndex), redeemScript)
@ -425,12 +426,12 @@ object Transactions {
// unsigned transaction
val tx = Transaction(
version = 2,
txIn = TxIn(input.outPoint, Array.emptyByteArray, 0xffffffffL) :: Nil,
txIn = TxIn(input.outPoint, ByteVector.empty, 0xffffffffL) :: Nil,
txOut = TxOut(Satoshi(0), localFinalScriptPubKey) :: Nil,
lockTime = 0)
// compute weight with a dummy 73 bytes signature (the largest you can get)
val weight = Transactions.addSigs(MainPenaltyTx(input, tx), BinaryData("00" * 73)).tx.weight()
val weight = Transactions.addSigs(MainPenaltyTx(input, tx), ByteVector.fill(73)(0)).tx.weight()
val fee = weight2fee(feeratePerKw, weight)
val amount = input.txOut.amount - fee
@ -442,7 +443,7 @@ object Transactions {
HtlcPenaltyTx(input, tx1)
}
def makeClosingTx(commitTxInput: InputInfo, localScriptPubKey: BinaryData, remoteScriptPubKey: BinaryData, localIsFunder: Boolean, dustLimit: Satoshi, closingFee: Satoshi, spec: CommitmentSpec): ClosingTx = {
def makeClosingTx(commitTxInput: InputInfo, localScriptPubKey: ByteVector, remoteScriptPubKey: ByteVector, localIsFunder: Boolean, dustLimit: Satoshi, closingFee: Satoshi, spec: CommitmentSpec): ClosingTx = {
require(spec.htlcs.isEmpty, "there shouldn't be any pending htlcs")
val (toLocalAmount: Satoshi, toRemoteAmount: Satoshi) = if (localIsFunder) {
@ -456,13 +457,13 @@ object Transactions {
val tx = Transaction(
version = 2,
txIn = TxIn(commitTxInput.outPoint, Array.emptyByteArray, sequence = 0xffffffffL) :: Nil,
txIn = TxIn(commitTxInput.outPoint, ByteVector.empty, sequence = 0xffffffffL) :: Nil,
txOut = toLocalOutput_opt.toSeq ++ toRemoteOutput_opt.toSeq ++ Nil,
lockTime = 0)
ClosingTx(commitTxInput, LexicographicalOrdering.sort(tx))
}
def findPubKeyScriptIndex(tx: Transaction, pubkeyScript: BinaryData, outputsAlreadyUsed: Set[Int], amount_opt: Option[Satoshi]): Int = {
def findPubKeyScriptIndex(tx: Transaction, pubkeyScript: ByteVector, outputsAlreadyUsed: Set[Int], amount_opt: Option[Satoshi]): Int = {
val outputIndex = tx.txOut
.zipWithIndex
.indexWhere { case (txOut, index) => amount_opt.map(_ == txOut.amount).getOrElse(true) && txOut.publicKeyScript == pubkeyScript && !outputsAlreadyUsed.contains(index)} // it's not enough to only resolve on pubkeyScript because we may have duplicates
@ -474,66 +475,66 @@ object Transactions {
}
def sign(tx: Transaction, inputIndex: Int, redeemScript: BinaryData, amount: Satoshi, key: PrivateKey): BinaryData = {
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)
}
def sign(txinfo: TransactionWithInputInfo, key: PrivateKey): BinaryData = {
def sign(txinfo: TransactionWithInputInfo, key: PrivateKey): ByteVector = {
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: BinaryData, remoteSig: BinaryData): CommitTx = {
def addSigs(commitTx: CommitTx, localFundingPubkey: PublicKey, remoteFundingPubkey: PublicKey, localSig: ByteVector, remoteSig: ByteVector): CommitTx = {
val witness = Scripts.witness2of2(localSig, remoteSig, localFundingPubkey, remoteFundingPubkey)
commitTx.copy(tx = commitTx.tx.updateWitness(0, witness))
}
def addSigs(mainPenaltyTx: MainPenaltyTx, revocationSig: BinaryData): MainPenaltyTx = {
def addSigs(mainPenaltyTx: MainPenaltyTx, revocationSig: ByteVector): MainPenaltyTx = {
val witness = Scripts.witnessToLocalDelayedWithRevocationSig(revocationSig, mainPenaltyTx.input.redeemScript)
mainPenaltyTx.copy(tx = mainPenaltyTx.tx.updateWitness(0, witness))
}
def addSigs(htlcPenaltyTx: HtlcPenaltyTx, revocationSig: BinaryData, revocationPubkey: PublicKey): HtlcPenaltyTx = {
def addSigs(htlcPenaltyTx: HtlcPenaltyTx, revocationSig: ByteVector, 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: BinaryData, remoteSig: BinaryData, paymentPreimage: BinaryData): HtlcSuccessTx = {
def addSigs(htlcSuccessTx: HtlcSuccessTx, localSig: ByteVector, remoteSig: ByteVector, 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: BinaryData, remoteSig: BinaryData): HtlcTimeoutTx = {
def addSigs(htlcTimeoutTx: HtlcTimeoutTx, localSig: ByteVector, remoteSig: ByteVector): HtlcTimeoutTx = {
val witness = witnessHtlcTimeout(localSig, remoteSig, htlcTimeoutTx.input.redeemScript)
htlcTimeoutTx.copy(tx = htlcTimeoutTx.tx.updateWitness(0, witness))
}
def addSigs(claimHtlcSuccessTx: ClaimHtlcSuccessTx, localSig: BinaryData, paymentPreimage: BinaryData): ClaimHtlcSuccessTx = {
def addSigs(claimHtlcSuccessTx: ClaimHtlcSuccessTx, localSig: ByteVector, paymentPreimage: ByteVector32): ClaimHtlcSuccessTx = {
val witness = witnessClaimHtlcSuccessFromCommitTx(localSig, paymentPreimage, claimHtlcSuccessTx.input.redeemScript)
claimHtlcSuccessTx.copy(tx = claimHtlcSuccessTx.tx.updateWitness(0, witness))
}
def addSigs(claimHtlcTimeoutTx: ClaimHtlcTimeoutTx, localSig: BinaryData): ClaimHtlcTimeoutTx = {
def addSigs(claimHtlcTimeoutTx: ClaimHtlcTimeoutTx, localSig: ByteVector): ClaimHtlcTimeoutTx = {
val witness = witnessClaimHtlcTimeoutFromCommitTx(localSig, claimHtlcTimeoutTx.input.redeemScript)
claimHtlcTimeoutTx.copy(tx = claimHtlcTimeoutTx.tx.updateWitness(0, witness))
}
def addSigs(claimP2WPKHOutputTx: ClaimP2WPKHOutputTx, localPaymentPubkey: BinaryData, localSig: BinaryData): ClaimP2WPKHOutputTx = {
def addSigs(claimP2WPKHOutputTx: ClaimP2WPKHOutputTx, localPaymentPubkey: ByteVector, localSig: ByteVector): ClaimP2WPKHOutputTx = {
val witness = ScriptWitness(Seq(localSig, localPaymentPubkey))
claimP2WPKHOutputTx.copy(tx = claimP2WPKHOutputTx.tx.updateWitness(0, witness))
}
def addSigs(claimHtlcDelayed: ClaimDelayedOutputTx, localSig: BinaryData): ClaimDelayedOutputTx = {
def addSigs(claimHtlcDelayed: ClaimDelayedOutputTx, localSig: ByteVector): ClaimDelayedOutputTx = {
val witness = witnessToLocalDelayedAfterDelay(localSig, claimHtlcDelayed.input.redeemScript)
claimHtlcDelayed.copy(tx = claimHtlcDelayed.tx.updateWitness(0, witness))
}
def addSigs(claimHtlcDelayedPenalty: ClaimDelayedOutputPenaltyTx, revocationSig: BinaryData): ClaimDelayedOutputPenaltyTx = {
def addSigs(claimHtlcDelayedPenalty: ClaimDelayedOutputPenaltyTx, revocationSig: ByteVector): 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: BinaryData, remoteSig: BinaryData): ClosingTx = {
def addSigs(closingTx: ClosingTx, localFundingPubkey: PublicKey, remoteFundingPubkey: PublicKey, localSig: ByteVector, remoteSig: ByteVector): ClosingTx = {
val witness = Scripts.witness2of2(localSig, remoteSig, localFundingPubkey, remoteFundingPubkey)
closingTx.copy(tx = closingTx.tx.updateWitness(0, witness))
}
@ -541,7 +542,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: BinaryData, pubKey: PublicKey): Boolean = {
def checkSig(txinfo: TransactionWithInputInfo, sig: ByteVector, 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)
}

View file

@ -17,7 +17,7 @@
package fr.acinq.eclair.wire
import fr.acinq.bitcoin.DeterministicWallet.{ExtendedPrivateKey, KeyPath}
import fr.acinq.bitcoin.{BinaryData, OutPoint, Transaction, TxOut}
import fr.acinq.bitcoin.{ByteVector32, OutPoint, Transaction, TxOut}
import fr.acinq.eclair.channel._
import fr.acinq.eclair.crypto.ShaChain
import fr.acinq.eclair.payment.{Local, Origin, Relayed}
@ -25,7 +25,7 @@ import fr.acinq.eclair.transactions.Transactions._
import fr.acinq.eclair.transactions._
import fr.acinq.eclair.wire.LightningMessageCodecs._
import grizzled.slf4j.Logging
import scodec.bits.{BitVector, ByteVector}
import scodec.bits.BitVector
import scodec.codecs._
import scodec.{Attempt, Codec}
@ -37,8 +37,8 @@ object ChannelCodecs extends Logging {
val keyPathCodec: Codec[KeyPath] = ("path" | listOfN(uint16, uint32)).xmap[KeyPath](l => new KeyPath(l), keyPath => keyPath.path.toList).as[KeyPath]
val extendedPrivateKeyCodec: Codec[ExtendedPrivateKey] = (
("secretkeybytes" | binarydata(32)) ::
("chaincode" | binarydata(32)) ::
("secretkeybytes" | bytes32) ::
("chaincode" | bytes32) ::
("depth" | uint16) ::
("path" | keyPathCodec) ::
("parent" | int64)).as[ExtendedPrivateKey]
@ -93,11 +93,11 @@ object ChannelCodecs extends Logging {
("toLocalMsat" | uint64) ::
("toRemoteMsat" | uint64)).as[CommitmentSpec]
def outPointCodec: Codec[OutPoint] = variableSizeBytes(uint16, bytes.xmap(d => OutPoint.read(d.toArray), d => ByteVector(OutPoint.write(d).data)))
def 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 => ByteVector(TxOut.write(d).data)))
def 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 => ByteVector(Transaction.write(d).data)))
def txCodec: Codec[Transaction] = variableSizeBytes(uint16, bytes.xmap(d => Transaction.read(d.toArray), d => Transaction.write(d)))
val inputInfoCodec: Codec[InputInfo] = (
("outPoint" | outPointCodec) ::
@ -106,7 +106,7 @@ object ChannelCodecs extends Logging {
val txWithInputInfoCodec: Codec[TransactionWithInputInfo] = discriminated[TransactionWithInputInfo].by(uint16)
.typecase(0x01, (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec)).as[CommitTx])
.typecase(0x02, (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec) :: ("paymentHash" | binarydata(32))).as[HtlcSuccessTx])
.typecase(0x02, (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec) :: ("paymentHash" | bytes32)).as[HtlcSuccessTx])
.typecase(0x03, (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec)).as[HtlcTimeoutTx])
.typecase(0x04, (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec)).as[ClaimHtlcSuccessTx])
.typecase(0x05, (("inputInfo" | inputInfoCodec) :: ("tx" | txCodec)).as[ClaimHtlcTimeoutTx])
@ -133,7 +133,7 @@ object ChannelCodecs extends Logging {
val remoteCommitCodec: Codec[RemoteCommit] = (
("index" | uint64) ::
("spec" | commitmentSpecCodec) ::
("txid" | binarydata(32)) ::
("txid" | bytes32) ::
("remotePerCommitmentPoint" | point)).as[RemoteCommit]
val updateMessageCodec: Codec[UpdateMessage] = lightningMessageCodec.narrow(f => Attempt.successful(f.asInstanceOf[UpdateMessage]), g => g)
@ -155,7 +155,7 @@ object ChannelCodecs extends Logging {
("reSignAsap" | bool)).as[WaitingForRevocation]
val relayedCodec: Codec[Relayed] = (
("originChannelId" | binarydata(32)) ::
("originChannelId" | bytes32) ::
("originHtlcId" | int64) ::
("amountMsatIn" | uint64) ::
("amountMsatOut" | uint64)).as[Relayed]
@ -171,10 +171,10 @@ object ChannelCodecs extends Logging {
(wire: BitVector) => originsListCodec.decode(wire).map(_.map(_.toMap))
)
val spentListCodec: Codec[List[(OutPoint, BinaryData)]] = listOfN(uint16, outPointCodec ~ binarydata(32))
val spentListCodec: Codec[List[(OutPoint, ByteVector32)]] = listOfN(uint16, outPointCodec ~ bytes32)
val spentMapCodec: Codec[Map[OutPoint, BinaryData]] = Codec[Map[OutPoint, BinaryData]](
(map: Map[OutPoint, BinaryData]) => spentListCodec.encode(map.toList),
val spentMapCodec: Codec[Map[OutPoint, ByteVector32]] = Codec[Map[OutPoint, ByteVector32]](
(map: Map[OutPoint, ByteVector32]) => spentListCodec.encode(map.toList),
(wire: BitVector) => spentListCodec.decode(wire).map(_.map(_.toMap))
)
@ -192,7 +192,7 @@ object ChannelCodecs extends Logging {
("remoteNextCommitInfo" | either(bool, waitingForRevocationCodec, point)) ::
("commitInput" | inputInfoCodec) ::
("remotePerCommitmentSecrets" | ShaChain.shaChainCodec) ::
("channelId" | binarydata(32))).as[Commitments]
("channelId" | bytes32)).as[Commitments]
val closingTxProposedCodec: Codec[ClosingTxProposed] = (
("unsignedTx" | txCodec) ::

View file

@ -26,7 +26,7 @@ object CommandCodecs {
val cmdFulfillCodec: Codec[CMD_FULFILL_HTLC] =
(("id" | int64) ::
("r" | binarydata(32)) ::
("r" | bytes32) ::
("commit" | provide(false))).as[CMD_FULFILL_HTLC]
val cmdFailCodec: Codec[CMD_FAIL_HTLC] =
@ -36,7 +36,7 @@ object CommandCodecs {
val cmdFailMalformedCodec: Codec[CMD_FAIL_MALFORMED_HTLC] =
(("id" | int64) ::
("onionHash" | binarydata(32)) ::
("onionHash" | bytes32) ::
("failureCode" | uint16) ::
("commit" | provide(false))).as[CMD_FAIL_MALFORMED_HTLC]

View file

@ -16,8 +16,8 @@
package fr.acinq.eclair.wire
import fr.acinq.bitcoin.BinaryData
import fr.acinq.eclair.wire.LightningMessageCodecs.{binarydata, channelUpdateCodec, uint64}
import fr.acinq.bitcoin.ByteVector32
import fr.acinq.eclair.wire.LightningMessageCodecs.{bytes32, channelUpdateCodec, uint64}
import scodec.codecs._
import scodec.{Attempt, Codec}
@ -28,7 +28,7 @@ import scodec.{Attempt, Codec}
// @formatter:off
sealed trait FailureMessage { def message: String }
sealed trait BadOnion extends FailureMessage { def onionHash: BinaryData }
sealed trait BadOnion extends FailureMessage { def onionHash: ByteVector32 }
sealed trait Perm extends FailureMessage
sealed trait Node extends FailureMessage
sealed trait Update extends FailureMessage { def update: ChannelUpdate }
@ -37,9 +37,9 @@ case object InvalidRealm extends Perm { def message = "realm was not understood
case object TemporaryNodeFailure extends Node { def message = "general temporary failure of the processing node" }
case object PermanentNodeFailure extends Perm with Node { def message = "general permanent failure of the processing node" }
case object RequiredNodeFeatureMissing extends Perm with Node { def message = "processing node requires features that are missing from this onion" }
case class InvalidOnionVersion(onionHash: BinaryData) extends BadOnion with Perm { def message = "onion version was not understood by the processing node" }
case class InvalidOnionHmac(onionHash: BinaryData) extends BadOnion with Perm { def message = "onion HMAC was incorrect when it reached the processing node" }
case class InvalidOnionKey(onionHash: BinaryData) extends BadOnion with Perm { def message = "ephemeral key was unparsable by the processing node" }
case class InvalidOnionVersion(onionHash: ByteVector32) extends BadOnion with Perm { def message = "onion version was not understood by the processing node" }
case class InvalidOnionHmac(onionHash: ByteVector32) extends BadOnion with Perm { def message = "onion HMAC was incorrect when it reached the processing node" }
case class InvalidOnionKey(onionHash: ByteVector32) extends BadOnion with Perm { def message = "ephemeral key was unparsable by the processing node" }
case class TemporaryChannelFailure(update: ChannelUpdate) extends Update { def message = s"channel ${update.shortChannelId} is currently unavailable" }
case object PermanentChannelFailure extends Perm { def message = "channel is permanently unavailable" }
case object RequiredChannelFeatureMissing extends Perm { def message = "channel requires features not present in the onion" }
@ -63,7 +63,7 @@ object FailureMessageCodecs {
val NODE = 0x2000
val UPDATE = 0x1000
val sha256Codec: Codec[BinaryData] = ("sha256Codec" | binarydata(32))
val sha256Codec: Codec[ByteVector32] = ("sha256Codec" | bytes32)
val channelUpdateCodecWithType = LightningMessageCodecs.lightningMessageCodec.narrow[ChannelUpdate](f => Attempt.successful(f.asInstanceOf[ChannelUpdate]), g => g)

View file

@ -21,7 +21,7 @@ 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.{BinaryData, Crypto}
import fr.acinq.bitcoin.{ByteVector32, Crypto}
import fr.acinq.eclair.crypto.{Generators, Sphinx}
import fr.acinq.eclair.wire.FixedSizeStrictCodec.bytesStrict
import fr.acinq.eclair.{ShortChannelId, UInt64, wire}
@ -29,7 +29,6 @@ import org.apache.commons.codec.binary.Base32
import scodec.bits.{BitVector, ByteVector}
import scodec.codecs._
import scodec.{Attempt, Codec, DecodeResult, Err, SizeBound}
import shapeless.nat._
import scala.util.{Failure, Success, Try}
@ -48,13 +47,13 @@ object LightningMessageCodecs {
// (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.toArray), a => ByteVector(a.toByteArray).padLeft(8))
val uint64ex: Codec[UInt64] = bytes(8).xmap(b => UInt64(b), a => a.toByteVector.padLeft(8))
def binarydata(size: Int): Codec[BinaryData] = limitedSizeBytes(size, bytesStrict(size).xmap(d => BinaryData(d.toArray), d => ByteVector(d.data)))
def bytes32: Codec[ByteVector32] = limitedSizeBytes(32, bytesStrict(32).xmap(d => ByteVector32(d), d => d.bytes))
def varsizebinarydata: Codec[BinaryData] = variableSizeBytes(uint16, bytes.xmap(d => BinaryData(d.toArray), d => ByteVector(d.data)))
def varsizebinarydata: Codec[ByteVector] = variableSizeBytes(uint16, bytes)
def listofsignatures: Codec[List[BinaryData]] = listOfN(uint16, signature)
def listofsignatures: Codec[List[ByteVector]] = listOfN(uint16, signature)
def ipv4address: Codec[Inet4Address] = bytes(4).xmap(b => InetAddress.getByAddress(b.toArray).asInstanceOf[Inet4Address], a => ByteVector(a.getAddress))
@ -76,39 +75,38 @@ object LightningMessageCodecs {
def shortchannelid: Codec[ShortChannelId] = int64.xmap(l => ShortChannelId(l), s => s.toLong)
def signature: Codec[BinaryData] = Codec[BinaryData](
(der: BinaryData) => bytes(64).encode(ByteVector(der2wire(der).toArray)),
(wire: BitVector) => bytes(64).decode(wire).map(_.map(b => wire2der(b.toArray)))
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.toArray)))
(wire: BitVector) => bytes(32).decode(wire).map(_.map(b => Scalar(b)))
)
def point: Codec[Point] = Codec[Point](
(point: Point) => bytes(33).encode(ByteVector(point.toBin(compressed = true).toArray)),
(wire: BitVector) => bytes(33).decode(wire).map(_.map(b => Point(b.toArray)))
(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(ByteVector(priv.value.toBin.toArray)),
(wire: BitVector) => bytes(32).decode(wire).map(_.map(b => PrivateKey(b.toArray, compressed = true)))
(priv: PrivateKey) => bytes(32).encode(priv.value.toBin),
(wire: BitVector) => bytes(32).decode(wire).map(_.map(b => PrivateKey(b, compressed = true)))
)
def publicKey: Codec[PublicKey] = Codec[PublicKey](
(pub: PublicKey) => bytes(33).encode(ByteVector(pub.value.toBin(compressed = true).toArray)),
(wire: BitVector) => bytes(33).decode(wire).map(_.map(b => PublicKey(b.toArray)))
(pub: PublicKey) => bytes(33).encode(pub.value.toBin(compressed = true)),
(wire: BitVector) => bytes(33).decode(wire).map(_.map(b => PublicKey(b)))
)
def optionalSignature: Codec[Option[BinaryData]] = Codec[Option[BinaryData]](
(der: Option[BinaryData]) => der match {
case Some(sig) => bytes(64).encode(ByteVector(der2wire(sig).toArray))
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 => {
val a = b.toArray
if (a.exists(_ != 0)) Some(wire2der(a)) else None
if (b.toArray.exists(_ != 0)) Some(wire2der(b)) else None
}))
)
@ -116,13 +114,13 @@ object LightningMessageCodecs {
def zeropaddedstring(size: Int): Codec[String] = fixedSizeBytes(32, utf8).xmap(s => s.takeWhile(_ != '\u0000'), s => s)
def der2wire(signature: BinaryData): BinaryData = {
def der2wire(signature: ByteVector): ByteVector = {
require(Crypto.isDERSignature(signature), s"invalid DER signature $signature")
val (r, s) = Crypto.decodeSignature(signature)
Generators.fixSize(r.toByteArray.dropWhile(_ == 0)) ++ Generators.fixSize(s.toByteArray.dropWhile(_ == 0))
Generators.fixSize(ByteVector.view(r.toByteArray.dropWhile(_ == 0))) ++ Generators.fixSize(ByteVector.view(s.toByteArray.dropWhile(_ == 0)))
}
def wire2der(sig: BinaryData): BinaryData = {
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)
@ -134,7 +132,7 @@ object LightningMessageCodecs {
("localFeatures" | varsizebinarydata)).as[Init]
val errorCodec: Codec[Error] = (
("channelId" | binarydata(32)) ::
("channelId" | bytes32) ::
("data" | varsizebinarydata)).as[Error]
val pingCodec: Codec[Ping] = (
@ -145,15 +143,15 @@ object LightningMessageCodecs {
("data" | varsizebinarydata).as[Pong]
val channelReestablishCodec: Codec[ChannelReestablish] = (
("channelId" | binarydata(32)) ::
("channelId" | bytes32) ::
("nextLocalCommitmentNumber" | uint64) ::
("nextRemoteRevocationNumber" | uint64) ::
("yourLastPerCommitmentSecret" | optional(bitsRemaining, scalar)) ::
("myCurrentPerCommitmentPoint" | optional(bitsRemaining, point))).as[ChannelReestablish]
val openChannelCodec: Codec[OpenChannel] = (
("chainHash" | binarydata(32)) ::
("temporaryChannelId" | binarydata(32)) ::
("chainHash" | bytes32) ::
("temporaryChannelId" | bytes32) ::
("fundingSatoshis" | uint64) ::
("pushMsat" | uint64) ::
("dustLimitSatoshis" | uint64) ::
@ -172,7 +170,7 @@ object LightningMessageCodecs {
("channelFlags" | byte)).as[OpenChannel]
val acceptChannelCodec: Codec[AcceptChannel] = (
("temporaryChannelId" | binarydata(32)) ::
("temporaryChannelId" | bytes32) ::
("dustLimitSatoshis" | uint64) ::
("maxHtlcValueInFlightMsat" | uint64ex) ::
("channelReserveSatoshis" | uint64) ::
@ -188,76 +186,76 @@ object LightningMessageCodecs {
("firstPerCommitmentPoint" | point)).as[AcceptChannel]
val fundingCreatedCodec: Codec[FundingCreated] = (
("temporaryChannelId" | binarydata(32)) ::
("fundingTxid" | binarydata(32)) ::
("temporaryChannelId" | bytes32) ::
("fundingTxid" | bytes32) ::
("fundingOutputIndex" | uint16) ::
("signature" | signature)).as[FundingCreated]
val fundingSignedCodec: Codec[FundingSigned] = (
("channelId" | binarydata(32)) ::
("channelId" | bytes32) ::
("signature" | signature)).as[FundingSigned]
val fundingLockedCodec: Codec[FundingLocked] = (
("channelId" | binarydata(32)) ::
("channelId" | bytes32) ::
("nextPerCommitmentPoint" | point)).as[FundingLocked]
val shutdownCodec: Codec[wire.Shutdown] = (
("channelId" | binarydata(32)) ::
("channelId" | bytes32) ::
("scriptPubKey" | varsizebinarydata)).as[Shutdown]
val closingSignedCodec: Codec[ClosingSigned] = (
("channelId" | binarydata(32)) ::
("channelId" | bytes32) ::
("feeSatoshis" | uint64) ::
("signature" | signature)).as[ClosingSigned]
val updateAddHtlcCodec: Codec[UpdateAddHtlc] = (
("channelId" | binarydata(32)) ::
("channelId" | bytes32) ::
("id" | uint64) ::
("amountMsat" | uint64) ::
("paymentHash" | binarydata(32)) ::
("paymentHash" | bytes32) ::
("expiry" | uint32) ::
("onionRoutingPacket" | binarydata(Sphinx.PacketLength))).as[UpdateAddHtlc]
("onionRoutingPacket" | bytes(Sphinx.PacketLength))).as[UpdateAddHtlc]
val updateFulfillHtlcCodec: Codec[UpdateFulfillHtlc] = (
("channelId" | binarydata(32)) ::
("channelId" | bytes32) ::
("id" | uint64) ::
("paymentPreimage" | binarydata(32))).as[UpdateFulfillHtlc]
("paymentPreimage" | bytes32)).as[UpdateFulfillHtlc]
val updateFailHtlcCodec: Codec[UpdateFailHtlc] = (
("channelId" | binarydata(32)) ::
("channelId" | bytes32) ::
("id" | uint64) ::
("reason" | varsizebinarydata)).as[UpdateFailHtlc]
val updateFailMalformedHtlcCodec: Codec[UpdateFailMalformedHtlc] = (
("channelId" | binarydata(32)) ::
("channelId" | bytes32) ::
("id" | uint64) ::
("onionHash" | binarydata(32)) ::
("onionHash" | bytes32) ::
("failureCode" | uint16)).as[UpdateFailMalformedHtlc]
val commitSigCodec: Codec[CommitSig] = (
("channelId" | binarydata(32)) ::
("channelId" | bytes32) ::
("signature" | signature) ::
("htlcSignatures" | listofsignatures)).as[CommitSig]
val revokeAndAckCodec: Codec[RevokeAndAck] = (
("channelId" | binarydata(32)) ::
("channelId" | bytes32) ::
("perCommitmentSecret" | scalar) ::
("nextPerCommitmentPoint" | point)
).as[RevokeAndAck]
val updateFeeCodec: Codec[UpdateFee] = (
("channelId" | binarydata(32)) ::
("channelId" | bytes32) ::
("feeratePerKw" | uint32)).as[UpdateFee]
val announcementSignaturesCodec: Codec[AnnouncementSignatures] = (
("channelId" | binarydata(32)) ::
("channelId" | bytes32) ::
("shortChannelId" | shortchannelid) ::
("nodeSignature" | signature) ::
("bitcoinSignature" | signature)).as[AnnouncementSignatures]
val channelAnnouncementWitnessCodec = (
("features" | varsizebinarydata) ::
("chainHash" | binarydata(32)) ::
("chainHash" | bytes32) ::
("shortChannelId" | shortchannelid) ::
("nodeId1" | publicKey) ::
("nodeId2" | publicKey) ::
@ -284,7 +282,7 @@ object LightningMessageCodecs {
nodeAnnouncementWitnessCodec).as[NodeAnnouncement]
val channelUpdateWitnessCodec =
("chainHash" | binarydata(32)) ::
("chainHash" | bytes32) ::
("shortChannelId" | shortchannelid) ::
("timestamp" | uint32) ::
(("messageFlags" | byte) >>:~ { messageFlags =>
@ -301,23 +299,23 @@ object LightningMessageCodecs {
channelUpdateWitnessCodec).as[ChannelUpdate]
val queryShortChannelIdsCodec: Codec[QueryShortChannelIds] = (
("chainHash" | binarydata(32)) ::
("chainHash" | bytes32) ::
("data" | varsizebinarydata)
).as[QueryShortChannelIds]
val replyShortChanelIdsEndCodec: Codec[ReplyShortChannelIdsEnd] = (
("chainHash" | binarydata(32)) ::
("chainHash" | bytes32) ::
("complete" | byte)
).as[ReplyShortChannelIdsEnd]
val queryChannelRangeCodec: Codec[QueryChannelRange] = (
("chainHash" | binarydata(32)) ::
("chainHash" | bytes32) ::
("firstBlockNum" | uint32) ::
("numberOfBlocks" | uint32)
).as[QueryChannelRange]
val replyChannelRangeCodec: Codec[ReplyChannelRange] = (
("chainHash" | binarydata(32)) ::
("chainHash" | bytes32) ::
("firstBlockNum" | uint32) ::
("numberOfBlocks" | uint32) ::
("complete" | byte) ::
@ -325,7 +323,7 @@ object LightningMessageCodecs {
).as[ReplyChannelRange]
val gossipTimestampFilterCodec: Codec[GossipTimestampFilter] = (
("chainHash" | binarydata(32)) ::
("chainHash" | bytes32) ::
("firstTimestamp" | uint32) ::
("timestampRange" | uint32)
).as[GossipTimestampFilter]

View file

@ -18,12 +18,13 @@ package fr.acinq.eclair.wire
import java.net.{Inet4Address, Inet6Address, InetAddress, InetSocketAddress}
import com.google.common.net.HostAndPort
import fr.acinq.bitcoin.BinaryData
import com.google.common.base.Charsets
import fr.acinq.bitcoin.ByteVector32
import fr.acinq.bitcoin.Crypto.{Point, PublicKey, Scalar}
import fr.acinq.eclair.{ShortChannelId, UInt64}
import scodec.bits.ByteVector
import scala.util.{Success, Try}
import scala.util.Try
/**
* Created by PM on 15/11/2016.
@ -36,30 +37,34 @@ sealed trait ChannelMessage extends LightningMessage
sealed trait HtlcMessage extends LightningMessage
sealed trait RoutingMessage extends LightningMessage
sealed trait HasTimestamp extends LightningMessage { def timestamp: Long }
sealed trait HasTemporaryChannelId extends LightningMessage { def temporaryChannelId: BinaryData } // <- not in the spec
sealed trait HasChannelId extends LightningMessage { def channelId: BinaryData } // <- not in the spec
sealed trait HasChainHash extends LightningMessage { def chainHash: BinaryData } // <- not in the spec
sealed trait HasTemporaryChannelId extends LightningMessage { def temporaryChannelId: ByteVector32 } // <- not in the spec
sealed trait HasChannelId extends LightningMessage { def channelId: ByteVector32 } // <- not in the spec
sealed trait HasChainHash extends LightningMessage { def chainHash: ByteVector32 } // <- not in the spec
sealed trait UpdateMessage extends HtlcMessage // <- not in the spec
// @formatter:on
case class Init(globalFeatures: BinaryData,
localFeatures: BinaryData) extends SetupMessage
case class Init(globalFeatures: ByteVector,
localFeatures: ByteVector) extends SetupMessage
case class Error(channelId: BinaryData,
data: BinaryData) extends SetupMessage with HasChannelId
case class Error(channelId: ByteVector32,
data: ByteVector) extends SetupMessage with HasChannelId
case class Ping(pongLength: Int, data: BinaryData) extends SetupMessage
object Error {
def apply(channelId: ByteVector32, msg: String): Error = Error(channelId, ByteVector.view(msg.getBytes(Charsets.US_ASCII)))
}
case class Pong(data: BinaryData) extends SetupMessage
case class Ping(pongLength: Int, data: ByteVector) extends SetupMessage
case class ChannelReestablish(channelId: BinaryData,
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
case class OpenChannel(chainHash: BinaryData,
temporaryChannelId: BinaryData,
case class OpenChannel(chainHash: ByteVector32,
temporaryChannelId: ByteVector32,
fundingSatoshis: Long,
pushMsat: Long,
dustLimitSatoshis: Long,
@ -77,7 +82,7 @@ case class OpenChannel(chainHash: BinaryData,
firstPerCommitmentPoint: Point,
channelFlags: Byte) extends ChannelMessage with HasTemporaryChannelId with HasChainHash
case class AcceptChannel(temporaryChannelId: BinaryData,
case class AcceptChannel(temporaryChannelId: ByteVector32,
dustLimitSatoshis: Long,
maxHtlcValueInFlightMsat: UInt64,
channelReserveSatoshis: Long,
@ -92,66 +97,66 @@ case class AcceptChannel(temporaryChannelId: BinaryData,
htlcBasepoint: Point,
firstPerCommitmentPoint: Point) extends ChannelMessage with HasTemporaryChannelId
case class FundingCreated(temporaryChannelId: BinaryData,
fundingTxid: BinaryData,
case class FundingCreated(temporaryChannelId: ByteVector32,
fundingTxid: ByteVector32,
fundingOutputIndex: Int,
signature: BinaryData) extends ChannelMessage with HasTemporaryChannelId
signature: ByteVector) extends ChannelMessage with HasTemporaryChannelId
case class FundingSigned(channelId: BinaryData,
signature: BinaryData) extends ChannelMessage with HasChannelId
case class FundingSigned(channelId: ByteVector32,
signature: ByteVector) extends ChannelMessage with HasChannelId
case class FundingLocked(channelId: BinaryData,
case class FundingLocked(channelId: ByteVector32,
nextPerCommitmentPoint: Point) extends ChannelMessage with HasChannelId
case class Shutdown(channelId: BinaryData,
scriptPubKey: BinaryData) extends ChannelMessage with HasChannelId
case class Shutdown(channelId: ByteVector32,
scriptPubKey: ByteVector) extends ChannelMessage with HasChannelId
case class ClosingSigned(channelId: BinaryData,
case class ClosingSigned(channelId: ByteVector32,
feeSatoshis: Long,
signature: BinaryData) extends ChannelMessage with HasChannelId
signature: ByteVector) extends ChannelMessage with HasChannelId
case class UpdateAddHtlc(channelId: BinaryData,
case class UpdateAddHtlc(channelId: ByteVector32,
id: Long,
amountMsat: Long,
paymentHash: BinaryData,
paymentHash: ByteVector32,
cltvExpiry: Long,
onionRoutingPacket: BinaryData) extends HtlcMessage with UpdateMessage with HasChannelId
onionRoutingPacket: ByteVector) extends HtlcMessage with UpdateMessage with HasChannelId
case class UpdateFulfillHtlc(channelId: BinaryData,
case class UpdateFulfillHtlc(channelId: ByteVector32,
id: Long,
paymentPreimage: BinaryData) extends HtlcMessage with UpdateMessage with HasChannelId
paymentPreimage: ByteVector32) extends HtlcMessage with UpdateMessage with HasChannelId
case class UpdateFailHtlc(channelId: BinaryData,
case class UpdateFailHtlc(channelId: ByteVector32,
id: Long,
reason: BinaryData) extends HtlcMessage with UpdateMessage with HasChannelId
reason: ByteVector) extends HtlcMessage with UpdateMessage with HasChannelId
case class UpdateFailMalformedHtlc(channelId: BinaryData,
case class UpdateFailMalformedHtlc(channelId: ByteVector32,
id: Long,
onionHash: BinaryData,
onionHash: ByteVector32,
failureCode: Int) extends HtlcMessage with UpdateMessage with HasChannelId
case class CommitSig(channelId: BinaryData,
signature: BinaryData,
htlcSignatures: List[BinaryData]) extends HtlcMessage with HasChannelId
case class CommitSig(channelId: ByteVector32,
signature: ByteVector,
htlcSignatures: List[ByteVector]) extends HtlcMessage with HasChannelId
case class RevokeAndAck(channelId: BinaryData,
case class RevokeAndAck(channelId: ByteVector32,
perCommitmentSecret: Scalar,
nextPerCommitmentPoint: Point) extends HtlcMessage with HasChannelId
case class UpdateFee(channelId: BinaryData,
case class UpdateFee(channelId: ByteVector32,
feeratePerKw: Long) extends ChannelMessage with UpdateMessage with HasChannelId
case class AnnouncementSignatures(channelId: BinaryData,
case class AnnouncementSignatures(channelId: ByteVector32,
shortChannelId: ShortChannelId,
nodeSignature: BinaryData,
bitcoinSignature: BinaryData) extends RoutingMessage with HasChannelId
nodeSignature: ByteVector,
bitcoinSignature: ByteVector) extends RoutingMessage with HasChannelId
case class ChannelAnnouncement(nodeSignature1: BinaryData,
nodeSignature2: BinaryData,
bitcoinSignature1: BinaryData,
bitcoinSignature2: BinaryData,
features: BinaryData,
chainHash: BinaryData,
case class ChannelAnnouncement(nodeSignature1: ByteVector,
nodeSignature2: ByteVector,
bitcoinSignature1: ByteVector,
bitcoinSignature2: ByteVector,
features: ByteVector,
chainHash: ByteVector32,
shortChannelId: ShortChannelId,
nodeId1: PublicKey,
nodeId2: PublicKey,
@ -196,16 +201,16 @@ case class Tor3(tor3: String, port: Int) extends OnionAddress { override def soc
// @formatter:on
case class NodeAnnouncement(signature: BinaryData,
features: BinaryData,
case class NodeAnnouncement(signature: ByteVector,
features: ByteVector,
timestamp: Long,
nodeId: PublicKey,
rgbColor: Color,
alias: String,
addresses: List[NodeAddress]) extends RoutingMessage with HasTimestamp
case class ChannelUpdate(signature: BinaryData,
chainHash: BinaryData,
case class ChannelUpdate(signature: ByteVector,
chainHash: ByteVector32,
shortChannelId: ShortChannelId,
timestamp: Long,
messageFlags: Byte,
@ -222,22 +227,22 @@ case class PerHopPayload(shortChannelId: ShortChannelId,
amtToForward: Long,
outgoingCltvValue: Long)
case class QueryShortChannelIds(chainHash: BinaryData,
data: BinaryData) extends RoutingMessage with HasChainHash
case class QueryShortChannelIds(chainHash: ByteVector32,
data: ByteVector) extends RoutingMessage with HasChainHash
case class QueryChannelRange(chainHash: BinaryData,
case class QueryChannelRange(chainHash: ByteVector32,
firstBlockNum: Long,
numberOfBlocks: Long) extends RoutingMessage with HasChainHash
case class ReplyChannelRange(chainHash: BinaryData,
case class ReplyChannelRange(chainHash: ByteVector32,
firstBlockNum: Long,
numberOfBlocks: Long,
complete: Byte,
data: BinaryData) extends RoutingMessage with HasChainHash
data: ByteVector) extends RoutingMessage with HasChainHash
case class ReplyShortChannelIdsEnd(chainHash: BinaryData,
case class ReplyShortChannelIdsEnd(chainHash: ByteVector32,
complete: Byte) extends RoutingMessage with HasChainHash
case class GossipTimestampFilter(chainHash: BinaryData,
case class GossipTimestampFilter(chainHash: ByteVector32,
firstTimestamp: Long,
timestampRange: Long) extends RoutingMessage with HasChainHash

View file

@ -21,6 +21,7 @@ import java.nio.ByteOrder
import fr.acinq.bitcoin.Protocol
import fr.acinq.eclair.Features._
import org.scalatest.FunSuite
import scodec.bits._
/**
* Created by PM on 27/01/2017.
@ -29,16 +30,16 @@ import org.scalatest.FunSuite
class FeaturesSpec extends FunSuite {
test("'initial_routing_sync' feature") {
assert(hasFeature("08", Features.INITIAL_ROUTING_SYNC_BIT_OPTIONAL))
assert(hasFeature(hex"08", Features.INITIAL_ROUTING_SYNC_BIT_OPTIONAL))
}
test("'data_loss_protect' feature") {
assert(hasFeature("01", Features.OPTION_DATA_LOSS_PROTECT_MANDATORY))
assert(hasFeature("02", Features.OPTION_DATA_LOSS_PROTECT_OPTIONAL))
assert(hasFeature(hex"01", Features.OPTION_DATA_LOSS_PROTECT_MANDATORY))
assert(hasFeature(hex"02", Features.OPTION_DATA_LOSS_PROTECT_OPTIONAL))
}
test("'initial_routing_sync' and 'data_loss_protect' feature") {
val features = "0a"
val features = hex"0a"
assert(areSupported(features) && hasFeature(features, OPTION_DATA_LOSS_PROTECT_OPTIONAL) && hasFeature(features, INITIAL_ROUTING_SYNC_BIT_OPTIONAL))
}
@ -46,8 +47,8 @@ class FeaturesSpec extends FunSuite {
assert(areSupported(Protocol.writeUInt64(1l << INITIAL_ROUTING_SYNC_BIT_OPTIONAL, ByteOrder.BIG_ENDIAN)))
assert(areSupported(Protocol.writeUInt64(1L << OPTION_DATA_LOSS_PROTECT_MANDATORY, ByteOrder.BIG_ENDIAN)))
assert(areSupported(Protocol.writeUInt64(1l << OPTION_DATA_LOSS_PROTECT_OPTIONAL, ByteOrder.BIG_ENDIAN)))
assert(areSupported("14") == false)
assert(areSupported("0141") == false)
assert(areSupported(hex"14") == false)
assert(areSupported(hex"0141") == false)
}
}

View file

@ -17,8 +17,9 @@
package fr.acinq.eclair
import fr.acinq.bitcoin.Crypto.PrivateKey
import fr.acinq.bitcoin.{Base58, Base58Check, Bech32, BinaryData, Block, Crypto, Script}
import fr.acinq.bitcoin.{Base58, Base58Check, Bech32, Block, ByteVector32, Crypto, Script}
import org.scalatest.FunSuite
import scodec.bits._
import scala.util.Try
@ -29,16 +30,17 @@ import scala.util.Try
class PackageSpec extends FunSuite {
test("compute long channel id") {
val data = (BinaryData("FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF"), 0, BinaryData("FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF")) ::
(BinaryData("FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF"), 1, BinaryData("FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFE")) ::
(BinaryData("FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF0000"), 2, BinaryData("FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF0002")) ::
(BinaryData("FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF00F0"), 0x0F00, BinaryData("FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF0FF0")) :: Nil
val data = ((hex"FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF", 0, hex"FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF") ::
(hex"FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF", 1, hex"FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFE") ::
(hex"FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF0000", 2, hex"FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF0002") ::
(hex"FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF00F0", 0x0F00, hex"FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF0FF0") :: Nil)
.map(x => (ByteVector32(x._1), x._2, ByteVector32(x._3)))
data.foreach(x => assert(toLongId(x._1, x._2) === x._3))
data.foreach(x => assert(toLongId(ByteVector32(x._1), x._2) === x._3))
}
test("decode base58 addresses") {
val priv = PrivateKey(BinaryData("01" * 32), compressed = true)
val priv = PrivateKey(ByteVector32(ByteVector.fill(32)(1)), compressed = true)
val pub = priv.publicKey
// p2pkh
@ -69,7 +71,7 @@ class PackageSpec extends FunSuite {
}
test("decode bech32 addresses") {
val priv = PrivateKey(BinaryData("01" * 32), compressed = true)
val priv = PrivateKey(ByteVector32(ByteVector.fill(32)(1)), compressed = true)
val pub = priv.publicKey
// p2wpkh

View file

@ -18,7 +18,7 @@ package fr.acinq.eclair
import org.scalatest.FunSuite
import scala.util.{Failure, Try}
import scala.util.Try

View file

@ -16,11 +16,11 @@
package fr.acinq.eclair
import java.io.{File, IOException}
import java.nio.file.attribute.BasicFileAttributes
import java.io.File
import java.nio.file._
import java.nio.file.attribute.BasicFileAttributes
import com.typesafe.config.{Config, ConfigFactory}
import com.typesafe.config.ConfigFactory
import fr.acinq.bitcoin.Block
import fr.acinq.eclair.crypto.LocalKeyManager
import org.scalatest.FunSuite

View file

@ -18,16 +18,15 @@ package fr.acinq.eclair
import java.sql.DriverManager
import com.google.common.net.HostAndPort
import fr.acinq.bitcoin.Crypto.PrivateKey
import fr.acinq.bitcoin.{BinaryData, Block, Script}
import fr.acinq.bitcoin.{Block, ByteVector32, Script}
import fr.acinq.eclair.NodeParams.BITCOIND
import fr.acinq.eclair.crypto.LocalKeyManager
import fr.acinq.eclair.db.sqlite._
import fr.acinq.eclair.io.Peer
import fr.acinq.eclair.wire.{Color, NodeAddress}
import fr.acinq.eclair.router.RouterConf
import fr.acinq.eclair.wire.Color
import fr.acinq.eclair.wire.{Color, NodeAddress}
import scodec.bits.ByteVector
import scala.concurrent.duration._
@ -40,7 +39,7 @@ object TestConstants {
val feeratePerKw = 10000L
object Alice {
val seed = BinaryData("01" * 32)
val seed = ByteVector32(ByteVector.fill(32)(1))
val keyManager = new LocalKeyManager(seed, Block.RegtestGenesisBlock.hash)
def sqlite = DriverManager.getConnection("jdbc:sqlite::memory:")
@ -51,8 +50,8 @@ object TestConstants {
alias = "alice",
color = Color(1, 2, 3),
publicAddresses = NodeAddress.fromParts("localhost", 9731).get :: Nil,
globalFeatures = "",
localFeatures = "00",
globalFeatures = ByteVector.empty,
localFeatures = ByteVector(0),
overrideFeatures = Map.empty,
dustLimitSatoshis = 1100,
maxHtlcValueInFlightMsat = UInt64(150000000),
@ -104,7 +103,7 @@ object TestConstants {
def channelParams = Peer.makeChannelParams(
nodeParams = nodeParams,
defaultFinalScriptPubKey = Script.write(Script.pay2wpkh(PrivateKey(Array.fill[Byte](32)(4), compressed = true).publicKey)),
defaultFinalScriptPubKey = Script.write(Script.pay2wpkh(PrivateKey(randomBytes32, compressed = true).publicKey)),
isFunder = true,
fundingSatoshis).copy(
channelReserveSatoshis = 10000 // Bob will need to keep that much satoshis as direct payment
@ -112,7 +111,7 @@ object TestConstants {
}
object Bob {
val seed = BinaryData("02" * 32)
val seed = ByteVector32(ByteVector.fill(32)(2))
val keyManager = new LocalKeyManager(seed, Block.RegtestGenesisBlock.hash)
def sqlite = DriverManager.getConnection("jdbc:sqlite::memory:")
@ -122,8 +121,8 @@ object TestConstants {
alias = "bob",
color = Color(4, 5, 6),
publicAddresses = NodeAddress.fromParts("localhost", 9732).get :: Nil,
globalFeatures = "",
localFeatures = "00", // no announcement
globalFeatures = ByteVector.empty,
localFeatures = ByteVector.empty, // no announcement
overrideFeatures = Map.empty,
dustLimitSatoshis = 1000,
maxHtlcValueInFlightMsat = UInt64.MaxValue, // Bob has no limit on the combined max value of in-flight htlcs
@ -175,7 +174,7 @@ object TestConstants {
def channelParams = Peer.makeChannelParams(
nodeParams = nodeParams,
defaultFinalScriptPubKey = Script.write(Script.pay2wpkh(PrivateKey(Array.fill[Byte](32)(5), compressed = true).publicKey)),
defaultFinalScriptPubKey = Script.write(Script.pay2wpkh(PrivateKey(randomBytes32, compressed = true).publicKey)),
isFunder = false,
fundingSatoshis).copy(
channelReserveSatoshis = 20000 // Alice will need to keep that much satoshis as direct payment

View file

@ -16,29 +16,28 @@
package fr.acinq.eclair
import fr.acinq.bitcoin.BinaryData
import org.scalatest.FunSuite
import scodec.bits._
class UInt64Spec extends FunSuite {
test("handle values from 0 to 2^63-1") {
val a = UInt64("0xffffffffffffffff")
val b = UInt64("0xfffffffffffffffe")
val a = UInt64(hex"0xffffffffffffffff")
val b = UInt64(hex"0xfffffffffffffffe")
val c = UInt64(42)
val z = UInt64(0)
assert(a > b)
assert(b < a)
assert(z < a && z < b && z < c)
assert(a == a)
assert(BinaryData(a.toByteArray) === BinaryData("0xffffffffffffffff"))
assert(a.toByteVector === hex"0xffffffffffffffff")
assert(a.toString === "18446744073709551615")
assert(BinaryData(b.toByteArray) === BinaryData("0xfffffffffffffffe"))
assert(b.toByteVector === hex"0xfffffffffffffffe")
assert(b.toString === "18446744073709551614")
assert(BinaryData(c.toByteArray) === BinaryData("0x2a"))
assert(c.toByteVector === hex"0x2a")
assert(c.toString === "42")
assert(BinaryData(z.toByteArray) === BinaryData("0x00"))
assert(z.toByteVector === hex"0x00")
assert(z.toString === "0")
}

View file

@ -19,25 +19,24 @@ package fr.acinq.eclair.api
import java.io.{File, FileOutputStream}
import akka.actor.{Actor, ActorSystem, Props, Scheduler}
import org.scalatest.FunSuite
import akka.http.scaladsl.model.StatusCodes._
import akka.http.scaladsl.testkit.{RouteTestTimeout, ScalatestRouteTest}
import de.heikoseeberger.akkahttpjson4s.Json4sSupport.{marshaller, unmarshaller}
import fr.acinq.eclair.blockchain.TestWallet
import fr.acinq.eclair.{Kit, TestConstants}
import fr.acinq.eclair.io.Peer.{GetPeerInfo, PeerInfo}
import TestConstants._
import akka.NotUsed
import akka.actor.{Actor, Props, Scheduler}
import akka.http.scaladsl.model.StatusCodes._
import akka.http.scaladsl.model.headers.BasicHttpCredentials
import akka.http.scaladsl.model.ws.{Message, TextMessage}
import akka.http.scaladsl.server.Route
import akka.http.scaladsl.testkit.{RouteTestTimeout, ScalatestRouteTest}
import akka.stream.scaladsl.Flow
import de.heikoseeberger.akkahttpjson4s.Json4sSupport.{marshaller, unmarshaller}
import fr.acinq.eclair.Kit
import fr.acinq.eclair.TestConstants._
import fr.acinq.eclair.blockchain.TestWallet
import fr.acinq.eclair.channel.Register.ForwardShortId
import fr.acinq.eclair.router.{Graph, Router}
import fr.acinq.eclair.io.Peer.{GetPeerInfo, PeerInfo}
import org.json4s.Formats
import org.json4s.JsonAST.{JInt, JString}
import org.json4s.jackson.Serialization
import org.scalatest.FunSuite
import scala.concurrent.Future
import scala.concurrent.duration._
@ -80,8 +79,7 @@ class JsonRpcServiceSpec extends FunSuite with ScalatestRouteTest {
test("API service should handle failures correctly"){
val mockService = new MockService
import mockService.formats
import mockService.serialization
import mockService.{formats, serialization}
// no auth
Post("/", JsonRPCBody(method = "help", params = Seq.empty)) ~>
@ -135,8 +133,7 @@ class JsonRpcServiceSpec extends FunSuite with ScalatestRouteTest {
test("'help' should respond with a help message") {
val mockService = new MockService
import mockService.formats
import mockService.serialization
import mockService.{formats, serialization}
val postBody = JsonRPCBody(method = "help", params = Seq.empty)
@ -185,8 +182,7 @@ class JsonRpcServiceSpec extends FunSuite with ScalatestRouteTest {
}))
))
import mockService.formats
import mockService.serialization
import mockService.{formats, serialization}
val postBody = JsonRPCBody(method = "peers", params = Seq.empty)
@ -217,8 +213,7 @@ class JsonRpcServiceSpec extends FunSuite with ScalatestRouteTest {
publicAddresses = Alice.nodeParams.publicAddresses
))
}
import mockService.formats
import mockService.serialization
import mockService.{formats, serialization}
val postBody = JsonRPCBody(method = "getinfo", params = Seq.empty)
@ -248,8 +243,7 @@ class JsonRpcServiceSpec extends FunSuite with ScalatestRouteTest {
}))
))
import mockService.formats
import mockService.serialization
import mockService.{formats, serialization}
val postBody = JsonRPCBody(method = "close", params = Seq(JString(shortChannelIdSerialized)))

View file

@ -16,27 +16,26 @@
package fr.acinq.eclair.api
import java.net.{InetAddress, InetSocketAddress}
import java.net.InetAddress
import com.google.common.net.HostAndPort
import fr.acinq.bitcoin.{BinaryData, OutPoint}
import fr.acinq.bitcoin.{ByteVector32, OutPoint}
import fr.acinq.eclair.payment.PaymentRequest
import fr.acinq.eclair.transactions.{IN, OUT}
import fr.acinq.eclair.wire.{NodeAddress, Tor2, Tor3}
import org.json4s.jackson.Serialization
import org.scalatest.{FunSuite, Matchers}
import scodec.bits._
class JsonSerializersSpec extends FunSuite with Matchers {
test("deserialize Map[OutPoint, BinaryData]") {
val output1 = OutPoint("11418a2d282a40461966e4f578e1fdf633ad15c1b7fb3e771d14361127233be1", 0)
val output2 = OutPoint("3d62bd4f71dc63798418e59efbc7532380c900b5e79db3a5521374b161dd0e33", 1)
test("deserialize Map[OutPoint, ByteVector]") {
val output1 = OutPoint(ByteVector32(hex"11418a2d282a40461966e4f578e1fdf633ad15c1b7fb3e771d14361127233be1"), 0)
val output2 = OutPoint(ByteVector32(hex"3d62bd4f71dc63798418e59efbc7532380c900b5e79db3a5521374b161dd0e33"), 1)
val map = Map(
output1 -> BinaryData("dead"),
output2 -> BinaryData("beef")
output1 -> hex"dead",
output2 -> hex"beef"
)
// it won't work with the default key serializer
@ -46,7 +45,7 @@ class JsonSerializersSpec extends FunSuite with Matchers {
assert(error.msg.contains("Do not know how to serialize key of type class fr.acinq.bitcoin.OutPoint."))
// but it works with our custom key serializer
val json = Serialization.write(map)(org.json4s.DefaultFormats + new BinaryDataSerializer + new OutPointKeySerializer)
val json = Serialization.write(map)(org.json4s.DefaultFormats + new ByteVectorSerializer + new OutPointKeySerializer)
assert(json === s"""{"${output1.txid}:0":"dead","${output2.txid}:1":"beef"}""")
}

View file

@ -16,7 +16,8 @@
package fr.acinq.eclair.blockchain
import fr.acinq.bitcoin.{BinaryData, Crypto, OP_PUSHDATA, OutPoint, Satoshi, Script, Transaction, TxIn, TxOut}
import fr.acinq.bitcoin.{ByteVector32, Crypto, OP_PUSHDATA, OutPoint, Satoshi, Script, Transaction, TxIn, TxOut}
import scodec.bits.ByteVector
import scala.concurrent.Future
import scala.util.Try
@ -30,7 +31,7 @@ class TestWallet extends EclairWallet {
override def getFinalAddress: Future[String] = Future.successful("2MsRZ1asG6k94m6GYUufDGaZJMoJ4EV5JKs")
override def makeFundingTx(pubkeyScript: BinaryData, amount: Satoshi, feeRatePerKw: Long): Future[MakeFundingTxResponse] =
override def makeFundingTx(pubkeyScript: ByteVector, amount: Satoshi, feeRatePerKw: Long): Future[MakeFundingTxResponse] =
Future.successful(TestWallet.makeDummyFundingTx(pubkeyScript, amount, feeRatePerKw))
override def commit(tx: Transaction): Future[Boolean] = Future.successful(true)
@ -42,9 +43,9 @@ class TestWallet extends EclairWallet {
object TestWallet {
def makeDummyFundingTx(pubkeyScript: BinaryData, amount: Satoshi, feeRatePerKw: Long): MakeFundingTxResponse = {
def makeDummyFundingTx(pubkeyScript: ByteVector, amount: Satoshi, feeRatePerKw: Long): MakeFundingTxResponse = {
val fundingTx = Transaction(version = 2,
txIn = TxIn(OutPoint("42" * 32, 42), signatureScript = Nil, sequence = TxIn.SEQUENCE_FINAL) :: Nil,
txIn = TxIn(OutPoint(ByteVector32(ByteVector.fill(32)(1)), 42), signatureScript = Nil, sequence = TxIn.SEQUENCE_FINAL) :: Nil,
txOut = TxOut(amount, pubkeyScript) :: Nil,
lockTime = 0)
MakeFundingTxResponse(fundingTx, 0, Satoshi(420))

View file

@ -24,7 +24,6 @@ import akka.actor.{Actor, ActorRef, ActorSystem, Props}
import akka.pattern.pipe
import akka.testkit.{TestKitBase, TestProbe}
import com.softwaremill.sttp.okhttp.OkHttpFutureBackend
import com.softwaremill.sttp.okhttp.OkHttpFutureBackend
import fr.acinq.eclair.TestUtils
import fr.acinq.eclair.blockchain.bitcoind.rpc.{BasicBitcoinJsonRPCClient, BitcoinJsonRPCClient}
import fr.acinq.eclair.integration.IntegrationSpec

View file

@ -18,20 +18,20 @@ package fr.acinq.eclair.blockchain.electrum
import akka.actor.{ActorRef, ActorSystem, Props}
import akka.testkit.{TestKit, TestProbe}
import fr.acinq.bitcoin.{BinaryData, Crypto, Transaction}
import fr.acinq.bitcoin.{ByteVector32, Crypto, Transaction}
import fr.acinq.eclair.blockchain.electrum.ElectrumClient._
import grizzled.slf4j.Logging
import org.scalatest.{BeforeAndAfterAll, FunSuiteLike}
import scodec.bits._
import scala.concurrent.duration._
import scala.util.Random
class ElectrumClientPoolSpec extends TestKit(ActorSystem("test")) with FunSuiteLike with Logging with BeforeAndAfterAll {
var pool: ActorRef = _
val probe = TestProbe()
val referenceTx = Transaction.read("0200000003947e307df3ab452d23f02b5a65f4ada1804ee733e168e6197b0bd6cc79932b6c010000006a473044022069346ec6526454a481690a3664609f9e8032c34553015cfa2e9b25ebb420a33002206998f21a2aa771ad92a0c1083f4181a3acdb0d42ca51d01be1309da2ffb9cecf012102b4568cc6ee751f6d39f4a908b1fcffdb878f5f784a26a48c0acb0acff9d88e3bfeffffff966d9d969cd5f95bfd53003a35fcc1a50f4fb51f211596e6472583fdc5d38470000000006b4830450221009c9757515009c5709b5b678d678185202b817ef9a69ffb954144615ab11762210220732216384da4bf79340e9c46d0effba6ba92982cca998adfc3f354cec7715f800121035f7c3e077108035026f4ebd5d6ca696ef088d4f34d45d94eab4c41202ec74f9bfefffffff8d5062f5b04455c6cfa7e3f250e5a4fb44308ba2b86baf77f9ad0d782f57071010000006a47304402207f9f7dd91fe537a26d5554105977e3949a5c8c4ef53a6a3bff6da2d36eff928f02202b9427bef487a1825fd0c3c6851d17d5f19e6d73dfee22bf06db591929a2044d012102b4568cc6ee751f6d39f4a908b1fcffdb878f5f784a26a48c0acb0acff9d88e3bfeffffff02809698000000000017a914c82753548fdf4be1c3c7b14872c90b5198e67eaa876e642500000000001976a914e2365ec29471b3e271388b22eadf0e7f54d307a788ac6f771200")
val scriptHash: BinaryData = Crypto.sha256(referenceTx.txOut(0).publicKeyScript).reverse
val scriptHash = Crypto.sha256(referenceTx.txOut(0).publicKeyScript).reverse
import scala.concurrent.ExecutionContext.Implicits.global
override protected def afterAll(): Unit = {
@ -56,18 +56,18 @@ class ElectrumClientPoolSpec extends TestKit(ActorSystem("test")) with FunSuiteL
}, max = 15 seconds, interval = 1000 millis) }
test("get transaction") {
probe.send(pool, GetTransaction("c5efb5cbd35a44ba956b18100be0a91c9c33af4c7f31be20e33741d95f04e202"))
probe.send(pool, GetTransaction(ByteVector32(hex"c5efb5cbd35a44ba956b18100be0a91c9c33af4c7f31be20e33741d95f04e202")))
val GetTransactionResponse(tx) = probe.expectMsgType[GetTransactionResponse]
assert(tx.txid == BinaryData("c5efb5cbd35a44ba956b18100be0a91c9c33af4c7f31be20e33741d95f04e202"))
assert(tx.txid == ByteVector32.fromValidHex("c5efb5cbd35a44ba956b18100be0a91c9c33af4c7f31be20e33741d95f04e202"))
}
test("get merkle tree") {
probe.send(pool, GetMerkle("c5efb5cbd35a44ba956b18100be0a91c9c33af4c7f31be20e33741d95f04e202", 1210223))
probe.send(pool, GetMerkle(ByteVector32(hex"c5efb5cbd35a44ba956b18100be0a91c9c33af4c7f31be20e33741d95f04e202"), 1210223))
val response = probe.expectMsgType[GetMerkleResponse]
assert(response.txid == BinaryData("c5efb5cbd35a44ba956b18100be0a91c9c33af4c7f31be20e33741d95f04e202"))
assert(response.txid == ByteVector32(hex"c5efb5cbd35a44ba956b18100be0a91c9c33af4c7f31be20e33741d95f04e202"))
assert(response.block_height == 1210223)
assert(response.pos == 28)
assert(response.root == BinaryData("203a52cf3cc071467e5b8780d62d5dfb672bf7bc0841bc823691961ea23402fb"))
assert(response.root == ByteVector32(hex"203a52cf3cc071467e5b8780d62d5dfb672bf7bc0841bc823691961ea23402fb"))
}
test("header subscription") {
@ -87,12 +87,12 @@ class ElectrumClientPoolSpec extends TestKit(ActorSystem("test")) with FunSuiteL
test("get scripthash history") {
probe.send(pool, GetScriptHashHistory(scriptHash))
val GetScriptHashHistoryResponse(scriptHash1, history) = probe.expectMsgType[GetScriptHashHistoryResponse]
assert(history.contains((TransactionHistoryItem(1210224, "3903726806aa044fe59f40e42eed71bded068b43aaa9e2d716e38b7825412de0"))))
assert(history.contains((TransactionHistoryItem(1210224, ByteVector32(hex"3903726806aa044fe59f40e42eed71bded068b43aaa9e2d716e38b7825412de0")))))
}
test("list script unspents") {
probe.send(pool, ScriptHashListUnspent(scriptHash))
val ScriptHashListUnspentResponse(scriptHash1, unspents) = probe.expectMsgType[ScriptHashListUnspentResponse]
assert(unspents.contains(UnspentItem("3903726806aa044fe59f40e42eed71bded068b43aaa9e2d716e38b7825412de0", 0, 10000000L, 1210224L)))
assert(unspents.contains(UnspentItem(ByteVector32(hex"3903726806aa044fe59f40e42eed71bded068b43aaa9e2d716e38b7825412de0"), 0, 10000000L, 1210224L)))
}
}

View file

@ -20,9 +20,10 @@ import java.net.InetSocketAddress
import akka.actor.{ActorRef, ActorSystem, Props}
import akka.testkit.{TestKit, TestProbe}
import fr.acinq.bitcoin.{BinaryData, Crypto, Transaction}
import fr.acinq.bitcoin.{ByteVector32, Crypto, Transaction}
import grizzled.slf4j.Logging
import org.scalatest.{BeforeAndAfterAll, FunSuiteLike}
import scodec.bits._
import scala.concurrent.ExecutionContext.Implicits.global
import scala.concurrent.duration._
@ -35,7 +36,7 @@ class ElectrumClientSpec extends TestKit(ActorSystem("test")) with FunSuiteLike
var client: ActorRef = _
val probe = TestProbe()
val referenceTx = Transaction.read("0200000003947e307df3ab452d23f02b5a65f4ada1804ee733e168e6197b0bd6cc79932b6c010000006a473044022069346ec6526454a481690a3664609f9e8032c34553015cfa2e9b25ebb420a33002206998f21a2aa771ad92a0c1083f4181a3acdb0d42ca51d01be1309da2ffb9cecf012102b4568cc6ee751f6d39f4a908b1fcffdb878f5f784a26a48c0acb0acff9d88e3bfeffffff966d9d969cd5f95bfd53003a35fcc1a50f4fb51f211596e6472583fdc5d38470000000006b4830450221009c9757515009c5709b5b678d678185202b817ef9a69ffb954144615ab11762210220732216384da4bf79340e9c46d0effba6ba92982cca998adfc3f354cec7715f800121035f7c3e077108035026f4ebd5d6ca696ef088d4f34d45d94eab4c41202ec74f9bfefffffff8d5062f5b04455c6cfa7e3f250e5a4fb44308ba2b86baf77f9ad0d782f57071010000006a47304402207f9f7dd91fe537a26d5554105977e3949a5c8c4ef53a6a3bff6da2d36eff928f02202b9427bef487a1825fd0c3c6851d17d5f19e6d73dfee22bf06db591929a2044d012102b4568cc6ee751f6d39f4a908b1fcffdb878f5f784a26a48c0acb0acff9d88e3bfeffffff02809698000000000017a914c82753548fdf4be1c3c7b14872c90b5198e67eaa876e642500000000001976a914e2365ec29471b3e271388b22eadf0e7f54d307a788ac6f771200")
val scriptHash: BinaryData = Crypto.sha256(referenceTx.txOut(0).publicKeyScript).reverse
val scriptHash = Crypto.sha256(referenceTx.txOut(0).publicKeyScript).reverse
override protected def beforeAll(): Unit = {
client = system.actorOf(Props(new ElectrumClient(new InetSocketAddress("testnet.qtornado.com", 51002), SSL.LOOSE)), "electrum-client")
@ -51,15 +52,15 @@ class ElectrumClientSpec extends TestKit(ActorSystem("test")) with FunSuiteLike
}
test("get transaction") {
probe.send(client, GetTransaction("c5efb5cbd35a44ba956b18100be0a91c9c33af4c7f31be20e33741d95f04e202"))
probe.send(client, GetTransaction(ByteVector32(hex"c5efb5cbd35a44ba956b18100be0a91c9c33af4c7f31be20e33741d95f04e202")))
val GetTransactionResponse(tx) = probe.expectMsgType[GetTransactionResponse]
assert(tx.txid == BinaryData("c5efb5cbd35a44ba956b18100be0a91c9c33af4c7f31be20e33741d95f04e202"))
assert(tx.txid == ByteVector32(hex"c5efb5cbd35a44ba956b18100be0a91c9c33af4c7f31be20e33741d95f04e202"))
}
test("get header") {
probe.send(client, GetHeader(10000))
val GetHeaderResponse(height, header) = probe.expectMsgType[GetHeaderResponse]
assert(header.blockId == BinaryData("000000000058b74204bb9d59128e7975b683ac73910660b6531e59523fb4a102"))
assert(header.blockId == ByteVector32(hex"000000000058b74204bb9d59128e7975b683ac73910660b6531e59523fb4a102"))
}
test("get headers") {
@ -71,12 +72,12 @@ class ElectrumClientSpec extends TestKit(ActorSystem("test")) with FunSuiteLike
}
test("get merkle tree") {
probe.send(client, GetMerkle("c5efb5cbd35a44ba956b18100be0a91c9c33af4c7f31be20e33741d95f04e202", 1210223))
probe.send(client, GetMerkle(ByteVector32(hex"c5efb5cbd35a44ba956b18100be0a91c9c33af4c7f31be20e33741d95f04e202"), 1210223))
val response = probe.expectMsgType[GetMerkleResponse]
assert(response.txid == BinaryData("c5efb5cbd35a44ba956b18100be0a91c9c33af4c7f31be20e33741d95f04e202"))
assert(response.txid == ByteVector32(hex"c5efb5cbd35a44ba956b18100be0a91c9c33af4c7f31be20e33741d95f04e202"))
assert(response.block_height == 1210223)
assert(response.pos == 28)
assert(response.root == BinaryData("203a52cf3cc071467e5b8780d62d5dfb672bf7bc0841bc823691961ea23402fb"))
assert(response.root == ByteVector32(hex"203a52cf3cc071467e5b8780d62d5dfb672bf7bc0841bc823691961ea23402fb"))
}
test("header subscription") {
@ -96,12 +97,12 @@ 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(1210224, "3903726806aa044fe59f40e42eed71bded068b43aaa9e2d716e38b7825412de0"))))
assert(history.contains((TransactionHistoryItem(1210224, ByteVector32(hex"3903726806aa044fe59f40e42eed71bded068b43aaa9e2d716e38b7825412de0")))))
}
test("list script unspents") {
probe.send(client, ScriptHashListUnspent(scriptHash))
val ScriptHashListUnspentResponse(scriptHash1, unspents) = probe.expectMsgType[ScriptHashListUnspentResponse]
assert(unspents.contains(UnspentItem("3903726806aa044fe59f40e42eed71bded068b43aaa9e2d716e38b7825412de0", 0, 10000000L, 1210224L)))
assert(unspents.contains(UnspentItem(ByteVector32(hex"3903726806aa044fe59f40e42eed71bded068b43aaa9e2d716e38b7825412de0"), 0, 10000000L, 1210224L)))
}
}

View file

@ -25,6 +25,7 @@ import fr.acinq.eclair.blockchain.electrum.db.sqlite.SqliteWalletDb
import fr.acinq.eclair.transactions.Transactions
import grizzled.slf4j.Logging
import org.scalatest.FunSuite
import scodec.bits.ByteVector
import scala.util.{Failure, Random, Success, Try}
@ -39,7 +40,7 @@ class ElectrumWalletBasicSpec extends FunSuite with Logging {
val feeRatePerKw = 20000
val minimumFee = Satoshi(2000)
val master = DeterministicWallet.generate(BinaryData("01" * 32))
val master = DeterministicWallet.generate(ByteVector32(ByteVector.fill(32)(1)))
val accountMaster = accountKey(master, Block.RegtestGenesisBlock.hash)
val accountIndex = 0
@ -97,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(BinaryData("01" * 32), compressed = true).publicKey
val pub = PrivateKey(ByteVector32(ByteVector.fill(32)(1)), compressed = true).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)
@ -177,7 +178,7 @@ class ElectrumWalletBasicSpec extends FunSuite with Logging {
assert(state3.utxos.length == 3)
assert(state3.balance == (Satoshi(350000000),Satoshi(0)))
val (tx, fee) = state3.spendAll(Script.pay2wpkh(BinaryData("01" * 20)), feeRatePerKw)
val (tx, fee) = state3.spendAll(Script.pay2wpkh(ByteVector.fill(20)(1)), feeRatePerKw)
val Some((received, sent, Some(fee1))) = state3.computeTransactionDelta(tx)
assert(received == Satoshi(0))
assert(fee == fee1)

View file

@ -19,14 +19,15 @@ package fr.acinq.eclair.blockchain.electrum
import java.net.InetSocketAddress
import java.sql.DriverManager
import akka.actor.{Actor, ActorRef, ActorSystem, PoisonPill, Props, Terminated}
import akka.actor.{ActorRef, ActorSystem, Terminated}
import akka.testkit
import akka.testkit.{TestActor, TestFSMRef, TestKit, TestProbe}
import fr.acinq.bitcoin.{BinaryData, Block, BlockHeader, MnemonicCode, Satoshi, Script, Transaction, TxOut}
import fr.acinq.bitcoin.{Block, BlockHeader, ByteVector32, MnemonicCode, Satoshi, Transaction, TxOut}
import fr.acinq.eclair.blockchain.electrum.ElectrumClient._
import fr.acinq.eclair.blockchain.electrum.ElectrumWallet._
import fr.acinq.eclair.blockchain.electrum.db.sqlite.SqliteWalletDb
import org.scalatest.FunSuiteLike
import scodec.bits.ByteVector
import scala.annotation.tailrec
import scala.concurrent.duration._
@ -35,7 +36,7 @@ import scala.concurrent.duration._
class ElectrumWalletSimulatedClientSpec extends TestKit(ActorSystem("test")) with FunSuiteLike {
val sender = TestProbe()
val entropy = BinaryData("01" * 32)
val entropy = ByteVector32(ByteVector.fill(32)(1))
val mnemonics = MnemonicCode.toMnemonics(entropy)
val seed = MnemonicCode.toSeed(mnemonics, "")
@ -170,7 +171,7 @@ class ElectrumWalletSimulatedClientSpec extends TestKit(ActorSystem("test")) wit
}
val key = wallet.stateData.accountKeys(0)
val scriptHash = computeScriptHashFromPublicKey(key.publicKey)
wallet ! ScriptHashSubscriptionResponse(scriptHash, "01" * 32)
wallet ! ScriptHashSubscriptionResponse(scriptHash, ByteVector32(ByteVector.fill(32)(1)).toHex)
client.expectMsg(GetScriptHashHistory(scriptHash))
val tx = Transaction(version = 2, txIn = Nil, txOut = TxOut(Satoshi(100000), ElectrumWallet.computePublicKeyScript(key.publicKey)) :: Nil, lockTime = 0)
@ -201,7 +202,7 @@ class ElectrumWalletSimulatedClientSpec extends TestKit(ActorSystem("test")) wit
TestActor.KeepRunning
}
})
probe.send(wallet, GetMerkleResponse(tx.txid, BinaryData("01" * 32) :: Nil, 2, 0))
probe.send(wallet, GetMerkleResponse(tx.txid, ByteVector32(ByteVector.fill(32)(1)) :: Nil, 2, 0))
watcher.expectTerminated(probe.ref)
awaitCond(wallet.stateName == ElectrumWallet.DISCONNECTED)

View file

@ -23,7 +23,7 @@ import java.sql.DriverManager
import akka.actor.{ActorRef, ActorSystem, Props}
import akka.testkit.{TestKit, TestProbe}
import com.whisk.docker.DockerReadyChecker
import fr.acinq.bitcoin.{BinaryData, Block, Btc, DeterministicWallet, MnemonicCode, Satoshi, Transaction, TxOut}
import fr.acinq.bitcoin.{Block, Btc, ByteVector32, DeterministicWallet, MnemonicCode, Satoshi, Transaction, TxOut}
import fr.acinq.eclair.blockchain.bitcoind.BitcoinCoreWallet.{FundTransactionResponse, SignTransactionResponse}
import fr.acinq.eclair.blockchain.bitcoind.{BitcoinCoreWallet, BitcoindService}
import fr.acinq.eclair.blockchain.electrum.ElectrumClient.{BroadcastTransaction, BroadcastTransactionResponse, SSL}
@ -32,6 +32,7 @@ import fr.acinq.eclair.blockchain.electrum.db.sqlite.SqliteWalletDb
import grizzled.slf4j.Logging
import org.json4s.JsonAST.{JDecimal, JString, JValue}
import org.scalatest.{BeforeAndAfterAll, FunSuiteLike}
import scodec.bits.ByteVector
import scala.concurrent.Await
import scala.concurrent.ExecutionContext.Implicits.global
@ -42,7 +43,7 @@ class ElectrumWalletSpec extends TestKit(ActorSystem("test")) with FunSuiteLike
import ElectrumWallet._
val entropy = BinaryData("01" * 32)
val entropy = ByteVector32(ByteVector.fill(32)(1))
val mnemonics = MnemonicCode.toMnemonics(entropy)
val seed = MnemonicCode.toSeed(mnemonics, "")
logger.info(s"mnemonic codes for our wallet: $mnemonics")
@ -197,7 +198,7 @@ class ElectrumWalletSpec extends TestKit(ActorSystem("test")) with FunSuiteLike
}, max = 30 seconds, interval = 1 second)
val TransactionReceived(tx, 0, received, sent, _, _) = listener.receiveOne(5 seconds)
assert(tx.txid === BinaryData(txid))
assert(tx.txid === ByteVector32.fromValidHex(txid))
assert(received === Satoshi(100000000))
logger.info("generating a new block")
@ -212,7 +213,7 @@ class ElectrumWalletSpec extends TestKit(ActorSystem("test")) with FunSuiteLike
awaitCond({
val msg = listener.receiveOne(5 seconds)
msg match {
case TransactionConfidenceChanged(BinaryData(txid), 1, _) => true
case TransactionConfidenceChanged(txid, 1, _) => true
case _ => false
}
}, max = 30 seconds, interval = 1 second)

View file

@ -19,7 +19,7 @@ package fr.acinq.eclair.blockchain.electrum
import com.spotify.docker.client.{DefaultDockerClient, DockerClient}
import com.whisk.docker.impl.spotify.SpotifyDockerFactory
import com.whisk.docker.scalatest.DockerTestKit
import com.whisk.docker.{DockerContainer, DockerFactory, LogLineReceiver}
import com.whisk.docker.{DockerContainer, DockerFactory}
import org.scalatest.Suite
trait ElectrumxService extends DockerTestKit {

View file

@ -18,7 +18,7 @@ package fr.acinq.eclair.blockchain.electrum.db.sqlite
import java.sql.DriverManager
import fr.acinq.bitcoin.{BinaryData, Block, BlockHeader, OutPoint, Satoshi, Transaction, TxIn, TxOut}
import fr.acinq.bitcoin.{Block, BlockHeader, OutPoint, Satoshi, Transaction, TxIn, TxOut}
import fr.acinq.eclair.blockchain.electrum.ElectrumClient
import fr.acinq.eclair.blockchain.electrum.ElectrumClient.GetMerkleResponse
import fr.acinq.eclair.blockchain.electrum.ElectrumWallet.PersistentData
@ -63,25 +63,21 @@ class SqliteWalletDbSpec extends FunSuite {
test("serialize persistent data") {
val db = new SqliteWalletDb(inmem)
def randomBytes(size: Int): BinaryData = {
val buffer = new Array[Byte](size)
random.nextBytes(buffer)
buffer
}
import fr.acinq.eclair.{randomBytes, randomBytes32}
def randomTransaction = Transaction(version = 2,
txIn = TxIn(OutPoint(randomBytes(32), random.nextInt(100)), signatureScript = Nil, sequence = TxIn.SEQUENCE_FINAL) :: Nil,
txIn = TxIn(OutPoint(randomBytes32, random.nextInt(100)), signatureScript = Nil, sequence = TxIn.SEQUENCE_FINAL) :: Nil,
txOut = TxOut(Satoshi(random.nextInt(10000000)), randomBytes(20)) :: Nil,
0L
)
def randomHeight = if (random.nextBoolean()) random.nextInt(500000) else -1
def randomHistoryItem = ElectrumClient.TransactionHistoryItem(randomHeight, randomBytes(32))
def randomHistoryItem = ElectrumClient.TransactionHistoryItem(randomHeight, randomBytes32)
def randomHistoryItems = (0 to random.nextInt(100)).map(_ => randomHistoryItem).toList
def randomProof = GetMerkleResponse(randomBytes(32), ((0 until 10).map(_ => randomBytes(32))).toList, random.nextInt(100000), 0)
def randomProof = GetMerkleResponse(randomBytes32, ((0 until 10).map(_ => randomBytes32)).toList, random.nextInt(100000), 0)
def randomPersistentData = {
val transactions = for (i <- 0 until random.nextInt(100)) yield randomTransaction
@ -89,11 +85,11 @@ class SqliteWalletDbSpec extends FunSuite {
PersistentData(
accountKeysCount = 10,
changeKeysCount = 10,
status = (for (i <- 0 until random.nextInt(100)) yield randomBytes(32) -> random.nextInt(100000).toHexString).toMap,
status = (for (i <- 0 until random.nextInt(100)) yield randomBytes32 -> random.nextInt(100000).toHexString).toMap,
transactions = transactions.map(tx => tx.hash -> tx).toMap,
heights = transactions.map(tx => tx.hash -> randomHeight).toMap,
history = (for (i <- 0 until random.nextInt(100)) yield randomBytes(32) -> randomHistoryItems).toMap,
proofs = (for (i <- 0 until random.nextInt(100)) yield randomBytes(32) -> randomProof).toMap,
history = (for (i <- 0 until random.nextInt(100)) yield randomBytes32 -> randomHistoryItems).toMap,
proofs = (for (i <- 0 until random.nextInt(100)) yield randomBytes32 -> randomProof).toMap,
pendingTransactions = transactions.toList,
locks = (for (i <- 0 until random.nextInt(10)) yield randomTransaction).toSet
)

View file

@ -17,10 +17,9 @@
package fr.acinq.eclair.channel
import akka.actor.{Actor, ActorLogging, ActorRef, Stash}
import fr.acinq.bitcoin.BinaryData
import fr.acinq.eclair.TestConstants
import fr.acinq.eclair.channel.Commitments.msg2String
import fr.acinq.eclair.wire.{Init, LightningMessage}
import scodec.bits.ByteVector
import scala.concurrent.duration._
import scala.util.Random
@ -72,7 +71,7 @@ class FuzzyPipe(fuzzy: Boolean) extends Actor with Stash with ActorLogging {
log.debug(f" X-${msg2String(msg)}%-6s--- B")
case 'reconnect =>
log.debug("RECONNECTED")
val dummyInit = Init(BinaryData.empty, BinaryData.empty)
val dummyInit = Init(ByteVector.empty, ByteVector.empty)
a ! INPUT_RECONNECTED(self, dummyInit, dummyInit)
b ! INPUT_RECONNECTED(self, dummyInit, dummyInit)
context become connected(a, b, Random.nextInt(40))

View file

@ -21,7 +21,7 @@ import java.util.concurrent.CountDownLatch
import akka.actor.{Actor, ActorLogging, ActorRef, Props, Status}
import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.bitcoin.{BinaryData, MilliSatoshi}
import fr.acinq.bitcoin.{ByteVector32, MilliSatoshi}
import fr.acinq.eclair.TestConstants.{Alice, Bob}
import fr.acinq.eclair._
import fr.acinq.eclair.blockchain._
@ -66,8 +66,8 @@ class FuzzySpec extends TestkitBaseClass with StateTestsHelperMethods with Loggi
relayerA ! alice
relayerB ! bob
// no announcements
alice ! INPUT_INIT_FUNDER("00" * 32, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Alice.channelParams, pipe, bobInit, channelFlags = 0x00.toByte)
bob ! INPUT_INIT_FUNDEE("00" * 32, Bob.channelParams, pipe, aliceInit)
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Alice.channelParams, pipe, bobInit, channelFlags = 0x00.toByte)
bob ! INPUT_INIT_FUNDEE(ByteVector32.Zeroes, Bob.channelParams, pipe, aliceInit)
pipe ! (alice, bob)
alice2blockchain.expectMsgType[WatchSpent]
alice2blockchain.expectMsgType[WatchConfirmed]
@ -88,7 +88,7 @@ class FuzzySpec extends TestkitBaseClass with StateTestsHelperMethods with Loggi
// we don't want to be below htlcMinimumMsat
val requiredAmount = 1000000
def buildCmdAdd(paymentHash: BinaryData, dest: PublicKey) = {
def buildCmdAdd(paymentHash: ByteVector32, dest: PublicKey) = {
// allow overpaying (no more than 2 times the required amount)
val amount = requiredAmount + Random.nextInt(requiredAmount)
val expiry = Globals.blockCount.get().toInt + Channel.MIN_CLTV_EXPIRY + 1

View file

@ -21,7 +21,7 @@ import java.util.concurrent.atomic.AtomicLong
import akka.actor.{Actor, ActorRef, ActorSystem, Props}
import akka.testkit.TestProbe
import fr.acinq.bitcoin.{BinaryData, Crypto}
import fr.acinq.bitcoin.{ByteVector32, Crypto}
import fr.acinq.eclair.TestConstants.{Alice, Bob}
import fr.acinq.eclair._
import fr.acinq.eclair.blockchain._
@ -42,20 +42,14 @@ class ThroughputSpec extends FunSuite {
val paymentHandler = system.actorOf(Props(new Actor() {
val random = new Random()
def generateR(): BinaryData = {
val r = Array.fill[Byte](32)(0)
random.nextBytes(r)
r
}
context.become(run(Map()))
override def receive: Receive = ???
def run(h2r: Map[BinaryData, BinaryData]): Receive = {
def run(h2r: Map[ByteVector32, ByteVector32]): Receive = {
case ('add, tgt: ActorRef) =>
val r = generateR()
val h: BinaryData = Crypto.sha256(r)
val r = randomBytes32
val h = Crypto.sha256(r)
tgt ! CMD_ADD_HTLC(1, h, 1)
context.become(run(h2r + (h -> r)))
@ -76,8 +70,8 @@ class ThroughputSpec extends FunSuite {
val bob = system.actorOf(Channel.props(Bob.nodeParams, wallet, Alice.nodeParams.nodeId, blockchain, ???, relayerB, None), "b")
val aliceInit = Init(Alice.channelParams.globalFeatures, Alice.channelParams.localFeatures)
val bobInit = Init(Bob.channelParams.globalFeatures, Bob.channelParams.localFeatures)
alice ! INPUT_INIT_FUNDER("00" * 32, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Alice.channelParams, pipe, bobInit, ChannelFlags.Empty)
bob ! INPUT_INIT_FUNDEE("00" * 32, Bob.channelParams, pipe, aliceInit)
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Alice.channelParams, pipe, bobInit, ChannelFlags.Empty)
bob ! INPUT_INIT_FUNDEE(ByteVector32.Zeroes, Bob.channelParams, pipe, aliceInit)
val latch = new CountDownLatch(2)
val listener = system.actorOf(Props(new Actor {

View file

@ -16,9 +16,8 @@
package fr.acinq.eclair.channel.states
import akka.actor.Actor
import akka.testkit.{TestFSMRef, TestKitBase, TestProbe}
import fr.acinq.bitcoin.{BinaryData, Crypto}
import fr.acinq.bitcoin.{ByteVector32, Crypto}
import fr.acinq.eclair.TestConstants.{Alice, Bob}
import fr.acinq.eclair.blockchain._
import fr.acinq.eclair.blockchain.fee.FeeratesPerKw
@ -27,16 +26,15 @@ import fr.acinq.eclair.crypto.Sphinx
import fr.acinq.eclair.payment.PaymentLifecycle
import fr.acinq.eclair.router.Hop
import fr.acinq.eclair.wire._
import fr.acinq.eclair.{Globals, NodeParams, TestConstants}
import scala.util.Random
import fr.acinq.eclair.{Globals, NodeParams, TestConstants, randomBytes32}
import scodec.bits.ByteVector
/**
* Created by PM on 23/08/2016.
*/
trait StateTestsHelperMethods extends TestKitBase {
def defaultOnion: BinaryData = "00" * Sphinx.PacketLength
def defaultOnion: ByteVector = ByteVector.fill(Sphinx.PacketLength)(0)
case class SetupFixture(alice: TestFSMRef[State, Data, Channel],
bob: TestFSMRef[State, Data, Channel],
@ -77,8 +75,8 @@ trait StateTestsHelperMethods extends TestKitBase {
val bobInit = Init(bobParams.globalFeatures, bobParams.localFeatures)
// reset global feerates (they may have been changed by previous tests)
Globals.feeratesPerKw.set(FeeratesPerKw.single(TestConstants.feeratePerKw))
alice ! INPUT_INIT_FUNDER("00" * 32, TestConstants.fundingSatoshis, pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, aliceParams, alice2bob.ref, bobInit, channelFlags)
bob ! INPUT_INIT_FUNDEE("00" * 32, bobParams, bob2alice.ref, aliceInit)
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, aliceParams, alice2bob.ref, bobInit, channelFlags)
bob ! INPUT_INIT_FUNDEE(ByteVector32.Zeroes, bobParams, bob2alice.ref, aliceInit)
alice2bob.expectMsgType[OpenChannel]
alice2bob.forward(bob)
bob2alice.expectMsgType[AcceptChannel]
@ -109,10 +107,9 @@ trait StateTestsHelperMethods extends TestKitBase {
channelUpdateListener.expectMsgType[LocalChannelUpdate]
}
def addHtlc(amountMsat: Int, s: TestFSMRef[State, Data, Channel], r: TestFSMRef[State, Data, Channel], s2r: TestProbe, r2s: TestProbe): (BinaryData, UpdateAddHtlc) = {
val R: BinaryData = Array.fill[Byte](32)(0)
Random.nextBytes(R)
val H: BinaryData = Crypto.sha256(R)
def addHtlc(amountMsat: Int, s: TestFSMRef[State, Data, Channel], r: TestFSMRef[State, Data, Channel], s2r: TestProbe, r2s: TestProbe): (ByteVector32, UpdateAddHtlc) = {
val R: ByteVector32 = randomBytes32
val H: ByteVector32 = Crypto.sha256(R)
val sender = TestProbe()
val receiverPubkey = r.underlyingActor.nodeParams.nodeId
val expiry = 400144
@ -125,7 +122,7 @@ trait StateTestsHelperMethods extends TestKitBase {
(R, htlc)
}
def fulfillHtlc(id: Long, R: BinaryData, s: TestFSMRef[State, Data, Channel], r: TestFSMRef[State, Data, Channel], s2r: TestProbe, r2s: TestProbe) = {
def fulfillHtlc(id: Long, R: ByteVector32, s: TestFSMRef[State, Data, Channel], r: TestFSMRef[State, Data, Channel], s2r: TestProbe, r2s: TestProbe) = {
val sender = TestProbe()
sender.send(s, CMD_FULFILL_HTLC(id, R))
sender.expectMsg("ok")

View file

@ -17,7 +17,7 @@
package fr.acinq.eclair.channel.states.a
import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.bitcoin.Block
import fr.acinq.bitcoin.{Block, ByteVector32}
import fr.acinq.eclair.TestConstants.{Alice, Bob}
import fr.acinq.eclair.channel.states.StateTestsHelperMethods
import fr.acinq.eclair.channel.{WAIT_FOR_FUNDING_INTERNAL, _}
@ -45,8 +45,8 @@ class WaitForAcceptChannelStateSpec extends TestkitBaseClass with StateTestsHelp
val aliceInit = Init(Alice.channelParams.globalFeatures, Alice.channelParams.localFeatures)
val bobInit = Init(Bob.channelParams.globalFeatures, Bob.channelParams.localFeatures)
within(30 seconds) {
alice ! INPUT_INIT_FUNDER("00" * 32, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Alice.channelParams, alice2bob.ref, bobInit, ChannelFlags.Empty)
bob ! INPUT_INIT_FUNDEE("00" * 32, Bob.channelParams, bob2alice.ref, aliceInit)
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Alice.channelParams, alice2bob.ref, bobInit, ChannelFlags.Empty)
bob ! INPUT_INIT_FUNDEE(ByteVector32.Zeroes, Bob.channelParams, bob2alice.ref, aliceInit)
alice2bob.expectMsgType[OpenChannel]
alice2bob.forward(bob)
awaitCond(alice.stateName == WAIT_FOR_ACCEPT_CHANNEL)
@ -68,7 +68,7 @@ class WaitForAcceptChannelStateSpec extends TestkitBaseClass with StateTestsHelp
val invalidMaxAcceptedHtlcs = 484
alice ! accept.copy(maxAcceptedHtlcs = invalidMaxAcceptedHtlcs)
val error = alice2bob.expectMsgType[Error]
assert(error === Error(accept.temporaryChannelId, InvalidMaxAcceptedHtlcs(accept.temporaryChannelId, invalidMaxAcceptedHtlcs, Channel.MAX_ACCEPTED_HTLCS).getMessage.getBytes("UTF-8")))
assert(error === Error(accept.temporaryChannelId, InvalidMaxAcceptedHtlcs(accept.temporaryChannelId, invalidMaxAcceptedHtlcs, Channel.MAX_ACCEPTED_HTLCS).getMessage))
awaitCond(alice.stateName == CLOSED)
}
@ -79,7 +79,7 @@ class WaitForAcceptChannelStateSpec extends TestkitBaseClass with StateTestsHelp
val lowDustLimitSatoshis = 545
alice ! accept.copy(dustLimitSatoshis = lowDustLimitSatoshis)
val error = alice2bob.expectMsgType[Error]
assert(error === Error(accept.temporaryChannelId, DustLimitTooSmall(accept.temporaryChannelId, lowDustLimitSatoshis, Channel.MIN_DUSTLIMIT).getMessage.getBytes("UTF-8")))
assert(error === Error(accept.temporaryChannelId, DustLimitTooSmall(accept.temporaryChannelId, lowDustLimitSatoshis, Channel.MIN_DUSTLIMIT).getMessage))
awaitCond(alice.stateName == CLOSED)
}
@ -89,7 +89,7 @@ class WaitForAcceptChannelStateSpec extends TestkitBaseClass with StateTestsHelp
val delayTooHigh = 10000
alice ! accept.copy(toSelfDelay = delayTooHigh)
val error = alice2bob.expectMsgType[Error]
assert(error === Error(accept.temporaryChannelId, ToSelfDelayTooHigh(accept.temporaryChannelId, delayTooHigh, Alice.nodeParams.maxToLocalDelayBlocks).getMessage.getBytes("UTF-8")))
assert(error === Error(accept.temporaryChannelId, ToSelfDelayTooHigh(accept.temporaryChannelId, delayTooHigh, Alice.nodeParams.maxToLocalDelayBlocks).getMessage))
awaitCond(alice.stateName == CLOSED)
}
@ -100,7 +100,7 @@ class WaitForAcceptChannelStateSpec extends TestkitBaseClass with StateTestsHelp
val reserveTooHigh = (0.3 * TestConstants.fundingSatoshis).toLong
alice ! accept.copy(channelReserveSatoshis = reserveTooHigh)
val error = alice2bob.expectMsgType[Error]
assert(error === Error(accept.temporaryChannelId, ChannelReserveTooHigh(accept.temporaryChannelId, reserveTooHigh, 0.3, 0.05).getMessage.getBytes("UTF-8")))
assert(error === Error(accept.temporaryChannelId, ChannelReserveTooHigh(accept.temporaryChannelId, reserveTooHigh, 0.3, 0.05).getMessage))
awaitCond(alice.stateName == CLOSED)
}
@ -110,7 +110,7 @@ class WaitForAcceptChannelStateSpec extends TestkitBaseClass with StateTestsHelp
val reserveTooSmall = accept.dustLimitSatoshis - 1
alice ! accept.copy(channelReserveSatoshis = reserveTooSmall)
val error = alice2bob.expectMsgType[Error]
assert(error === Error(accept.temporaryChannelId, DustLimitTooLarge(accept.temporaryChannelId, accept.dustLimitSatoshis, reserveTooSmall).getMessage.getBytes("UTF-8")))
assert(error === Error(accept.temporaryChannelId, DustLimitTooLarge(accept.temporaryChannelId, accept.dustLimitSatoshis, reserveTooSmall).getMessage))
awaitCond(alice.stateName == CLOSED)
}
@ -121,7 +121,7 @@ class WaitForAcceptChannelStateSpec extends TestkitBaseClass with StateTestsHelp
val reserveTooSmall = open.dustLimitSatoshis - 1
alice ! accept.copy(channelReserveSatoshis = reserveTooSmall)
val error = alice2bob.expectMsgType[Error]
assert(error === Error(accept.temporaryChannelId, ChannelReserveBelowOurDustLimit(accept.temporaryChannelId, reserveTooSmall, open.dustLimitSatoshis).getMessage.getBytes("UTF-8")))
assert(error === Error(accept.temporaryChannelId, ChannelReserveBelowOurDustLimit(accept.temporaryChannelId, reserveTooSmall, open.dustLimitSatoshis).getMessage))
awaitCond(alice.stateName == CLOSED)
}
@ -132,13 +132,13 @@ class WaitForAcceptChannelStateSpec extends TestkitBaseClass with StateTestsHelp
val dustTooBig = open.channelReserveSatoshis + 1
alice ! accept.copy(dustLimitSatoshis = dustTooBig)
val error = alice2bob.expectMsgType[Error]
assert(error === Error(accept.temporaryChannelId, DustLimitAboveOurChannelReserve(accept.temporaryChannelId, dustTooBig, open.channelReserveSatoshis).getMessage.getBytes("UTF-8")))
assert(error === Error(accept.temporaryChannelId, DustLimitAboveOurChannelReserve(accept.temporaryChannelId, dustTooBig, open.channelReserveSatoshis).getMessage))
awaitCond(alice.stateName == CLOSED)
}
test("recv Error") { f =>
import f._
alice ! Error("00" * 32, "oops".getBytes)
alice ! Error(ByteVector32.Zeroes, "oops")
awaitCond(alice.stateName == CLOSED)
}

View file

@ -17,7 +17,7 @@
package fr.acinq.eclair.channel.states.a
import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.bitcoin.Block
import fr.acinq.bitcoin.{Block, ByteVector32}
import fr.acinq.eclair.TestConstants.{Alice, Bob}
import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.states.StateTestsHelperMethods
@ -41,8 +41,8 @@ class WaitForOpenChannelStateSpec extends TestkitBaseClass with StateTestsHelper
val aliceInit = Init(Alice.channelParams.globalFeatures, Alice.channelParams.localFeatures)
val bobInit = Init(Bob.channelParams.globalFeatures, Bob.channelParams.localFeatures)
within(30 seconds) {
alice ! INPUT_INIT_FUNDER("00" * 32, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Alice.channelParams, alice2bob.ref, bobInit, ChannelFlags.Empty)
bob ! INPUT_INIT_FUNDEE("00" * 32, Bob.channelParams, bob2alice.ref, aliceInit)
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Alice.channelParams, alice2bob.ref, bobInit, ChannelFlags.Empty)
bob ! INPUT_INIT_FUNDEE(ByteVector32.Zeroes, Bob.channelParams, bob2alice.ref, aliceInit)
awaitCond(bob.stateName == WAIT_FOR_OPEN_CHANNEL)
withFixture(test.toNoArgTest(FixtureParam(bob, alice2bob, bob2alice, bob2blockchain)))
}
@ -62,7 +62,7 @@ class WaitForOpenChannelStateSpec extends TestkitBaseClass with StateTestsHelper
val livenetChainHash = Block.LivenetGenesisBlock.hash
bob ! open.copy(chainHash = livenetChainHash)
val error = bob2alice.expectMsgType[Error]
assert(error === Error(open.temporaryChannelId, InvalidChainHash(open.temporaryChannelId, Block.RegtestGenesisBlock.hash, livenetChainHash).getMessage.getBytes("UTF-8")))
assert(error === Error(open.temporaryChannelId, InvalidChainHash(open.temporaryChannelId, Block.RegtestGenesisBlock.hash, livenetChainHash).getMessage))
awaitCond(bob.stateName == CLOSED)
}
@ -72,7 +72,7 @@ class WaitForOpenChannelStateSpec extends TestkitBaseClass with StateTestsHelper
val lowFundingMsat = 100
bob ! open.copy(fundingSatoshis = lowFundingMsat)
val error = bob2alice.expectMsgType[Error]
assert(error === Error(open.temporaryChannelId, InvalidFundingAmount(open.temporaryChannelId, lowFundingMsat, Bob.nodeParams.minFundingSatoshis, Channel.MAX_FUNDING_SATOSHIS).getMessage.getBytes("UTF-8")))
assert(error === Error(open.temporaryChannelId, InvalidFundingAmount(open.temporaryChannelId, lowFundingMsat, Bob.nodeParams.minFundingSatoshis, Channel.MAX_FUNDING_SATOSHIS).getMessage))
awaitCond(bob.stateName == CLOSED)
}
@ -82,7 +82,7 @@ class WaitForOpenChannelStateSpec extends TestkitBaseClass with StateTestsHelper
val highFundingMsat = 100000000
bob ! open.copy(fundingSatoshis = highFundingMsat)
val error = bob2alice.expectMsgType[Error]
assert(error === Error(open.temporaryChannelId, InvalidFundingAmount(open.temporaryChannelId, highFundingMsat, Bob.nodeParams.minFundingSatoshis, Channel.MAX_FUNDING_SATOSHIS).getMessage.getBytes("UTF-8")))
assert(error === Error(open.temporaryChannelId, InvalidFundingAmount(open.temporaryChannelId, highFundingMsat, Bob.nodeParams.minFundingSatoshis, Channel.MAX_FUNDING_SATOSHIS).getMessage))
awaitCond(bob.stateName == CLOSED)
}
@ -92,7 +92,7 @@ class WaitForOpenChannelStateSpec extends TestkitBaseClass with StateTestsHelper
val invalidMaxAcceptedHtlcs = Channel.MAX_ACCEPTED_HTLCS + 1
bob ! open.copy(maxAcceptedHtlcs = invalidMaxAcceptedHtlcs)
val error = bob2alice.expectMsgType[Error]
assert(error === Error(open.temporaryChannelId, InvalidMaxAcceptedHtlcs(open.temporaryChannelId, invalidMaxAcceptedHtlcs, Channel.MAX_ACCEPTED_HTLCS).getMessage.getBytes("UTF-8")))
assert(error === Error(open.temporaryChannelId, InvalidMaxAcceptedHtlcs(open.temporaryChannelId, invalidMaxAcceptedHtlcs, Channel.MAX_ACCEPTED_HTLCS).getMessage))
awaitCond(bob.stateName == CLOSED)
}
@ -102,7 +102,7 @@ class WaitForOpenChannelStateSpec extends TestkitBaseClass with StateTestsHelper
val invalidPushMsat = 100000000000L
bob ! open.copy(pushMsat = invalidPushMsat)
val error = bob2alice.expectMsgType[Error]
assert(error === Error(open.temporaryChannelId, InvalidPushAmount(open.temporaryChannelId, invalidPushMsat, 1000 * open.fundingSatoshis).getMessage.getBytes("UTF-8")))
assert(error === Error(open.temporaryChannelId, InvalidPushAmount(open.temporaryChannelId, invalidPushMsat, 1000 * open.fundingSatoshis).getMessage))
awaitCond(bob.stateName == CLOSED)
}
@ -112,7 +112,7 @@ class WaitForOpenChannelStateSpec extends TestkitBaseClass with StateTestsHelper
val delayTooHigh = 10000
bob ! open.copy(toSelfDelay = delayTooHigh)
val error = bob2alice.expectMsgType[Error]
assert(error === Error(open.temporaryChannelId, ToSelfDelayTooHigh(open.temporaryChannelId, delayTooHigh, Alice.nodeParams.maxToLocalDelayBlocks).getMessage.getBytes("UTF-8")))
assert(error === Error(open.temporaryChannelId, ToSelfDelayTooHigh(open.temporaryChannelId, delayTooHigh, Alice.nodeParams.maxToLocalDelayBlocks).getMessage))
awaitCond(bob.stateName == CLOSED)
}
@ -123,7 +123,7 @@ class WaitForOpenChannelStateSpec extends TestkitBaseClass with StateTestsHelper
val reserveTooHigh = (0.3 * TestConstants.fundingSatoshis).toLong
bob ! open.copy(channelReserveSatoshis = reserveTooHigh)
val error = bob2alice.expectMsgType[Error]
assert(error === Error(open.temporaryChannelId, ChannelReserveTooHigh(open.temporaryChannelId, reserveTooHigh, 0.3, 0.05).getMessage.getBytes("UTF-8")))
assert(error === Error(open.temporaryChannelId, ChannelReserveTooHigh(open.temporaryChannelId, reserveTooHigh, 0.3, 0.05).getMessage))
awaitCond(bob.stateName == CLOSED)
}
@ -135,7 +135,7 @@ class WaitForOpenChannelStateSpec extends TestkitBaseClass with StateTestsHelper
bob ! open.copy(feeratePerKw = tinyFee)
val error = bob2alice.expectMsgType[Error]
// we check that the error uses the temporary channel id
assert(error === Error(open.temporaryChannelId, "local/remote feerates are too different: remoteFeeratePerKw=253 localFeeratePerKw=10000".getBytes("UTF-8")))
assert(error === Error(open.temporaryChannelId, "local/remote feerates are too different: remoteFeeratePerKw=253 localFeeratePerKw=10000"))
awaitCond(bob.stateName == CLOSED)
}
@ -147,7 +147,7 @@ class WaitForOpenChannelStateSpec extends TestkitBaseClass with StateTestsHelper
bob ! open.copy(feeratePerKw = tinyFee)
val error = bob2alice.expectMsgType[Error]
// we check that the error uses the temporary channel id
assert(error === Error(open.temporaryChannelId, "remote fee rate is too small: remoteFeeratePerKw=252".getBytes("UTF-8")))
assert(error === Error(open.temporaryChannelId, "remote fee rate is too small: remoteFeeratePerKw=252"))
awaitCond(bob.stateName == CLOSED)
}
@ -159,7 +159,7 @@ class WaitForOpenChannelStateSpec extends TestkitBaseClass with StateTestsHelper
bob ! open.copy(channelReserveSatoshis = reserveTooSmall)
val error = bob2alice.expectMsgType[Error]
// we check that the error uses the temporary channel id
assert(error === Error(open.temporaryChannelId, DustLimitTooLarge(open.temporaryChannelId, open.dustLimitSatoshis, reserveTooSmall).getMessage.getBytes("UTF-8")))
assert(error === Error(open.temporaryChannelId, DustLimitTooLarge(open.temporaryChannelId, open.dustLimitSatoshis, reserveTooSmall).getMessage))
awaitCond(bob.stateName == CLOSED)
}
@ -171,13 +171,13 @@ class WaitForOpenChannelStateSpec extends TestkitBaseClass with StateTestsHelper
bob ! open.copy(fundingSatoshis = fundingSatoshis, pushMsat = pushMsat)
val error = bob2alice.expectMsgType[Error]
// we check that the error uses the temporary channel id
assert(error === Error(open.temporaryChannelId, ChannelReserveNotMet(open.temporaryChannelId, 500 * 1000, (open.channelReserveSatoshis - 1) * 1000, open.channelReserveSatoshis).getMessage.getBytes("UTF-8")))
assert(error === Error(open.temporaryChannelId, ChannelReserveNotMet(open.temporaryChannelId, 500 * 1000, (open.channelReserveSatoshis - 1) * 1000, open.channelReserveSatoshis).getMessage))
awaitCond(bob.stateName == CLOSED)
}
test("recv Error") { f =>
import f._
bob ! Error("00" * 32, "oops".getBytes())
bob ! Error(ByteVector32.Zeroes, "oops")
awaitCond(bob.stateName == CLOSED)
}

View file

@ -17,6 +17,7 @@
package fr.acinq.eclair.channel.states.b
import akka.testkit.{TestFSMRef, TestProbe}
import fr.acinq.bitcoin.ByteVector32
import fr.acinq.eclair.TestConstants.{Alice, Bob}
import fr.acinq.eclair.channel._
import fr.acinq.eclair.channel.states.StateTestsHelperMethods
@ -40,8 +41,8 @@ class WaitForFundingCreatedInternalStateSpec extends TestkitBaseClass with State
val aliceInit = Init(Alice.channelParams.globalFeatures, Alice.channelParams.localFeatures)
val bobInit = Init(Bob.channelParams.globalFeatures, Bob.channelParams.localFeatures)
within(30 seconds) {
alice ! INPUT_INIT_FUNDER("00" * 32, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Alice.channelParams, alice2bob.ref, bobInit, ChannelFlags.Empty)
bob ! INPUT_INIT_FUNDEE("00" * 32, Bob.channelParams, bob2alice.ref, aliceInit)
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, Alice.channelParams, alice2bob.ref, bobInit, ChannelFlags.Empty)
bob ! INPUT_INIT_FUNDEE(ByteVector32.Zeroes, Bob.channelParams, bob2alice.ref, aliceInit)
alice2bob.expectMsgType[OpenChannel]
alice2bob.forward(bob)
bob2alice.expectMsgType[AcceptChannel]
@ -53,7 +54,7 @@ class WaitForFundingCreatedInternalStateSpec extends TestkitBaseClass with State
test("recv Error") { f =>
import f._
alice ! Error("00" * 32, "oops".getBytes)
alice ! Error(ByteVector32.Zeroes, "oops")
awaitCond(alice.stateName == CLOSED)
}

Some files were not shown because too many files have changed in this diff Show more