From 967404a82d670c5e6f611e0fad5055823de7eb3b Mon Sep 17 00:00:00 2001 From: sstone Date: Sun, 26 Feb 2017 19:29:17 +0100 Subject: [PATCH] persist channels, peers, and router - channels are pesisted using the transition callback --- .../src/main/resources/application.conf | 3 + .../src/main/scala/fr/acinq/eclair/Boot.scala | 31 ++++- .../fr/acinq/eclair/channel/Channel.scala | 53 +++++++- .../acinq/eclair/channel/ChannelEvents.scala | 2 +- .../acinq/eclair/channel/ChannelTypes.scala | 2 + .../fr/acinq/eclair/channel/Register.scala | 2 +- .../fr/acinq/eclair/db/ChannelState.scala | 22 +++ .../fr/acinq/eclair/db/JavaSerializer.scala | 24 ++++ .../scala/fr/acinq/eclair/db/SimpleDb.scala | 28 ++++ .../fr/acinq/eclair/db/SimpleFileDb.scala | 43 ++++++ .../scala/fr/acinq/eclair/gui/FxApp.scala | 1 + .../fr/acinq/eclair/gui/GUIUpdater.scala | 9 +- .../main/scala/fr/acinq/eclair/io/Peer.scala | 43 +++++- .../fr/acinq/eclair/io/Switchboard.scala | 24 +++- .../scala/fr/acinq/eclair/router/Router.scala | 103 ++++++++++---- .../acinq/eclair/channel/ThroughputSpec.scala | 5 +- .../eclair/channel/states/FuzzySpec.scala | 5 +- .../a/WaitForAcceptChannelStateSpec.scala | 5 +- .../a/WaitForOpenChannelStateSpec.scala | 5 +- ...itForFundingCreatedInternalStateSpec.scala | 5 +- .../b/WaitForFundingCreatedStateSpec.scala | 5 +- .../b/WaitForFundingSignedStateSpec.scala | 5 +- .../c/WaitForAnnSignaturesStateSpec.scala | 5 +- .../c/WaitForFundingConfirmedStateSpec.scala | 5 +- .../c/WaitForFundingLockedStateSpec.scala | 5 +- .../channel/states/e/NormalStateSpec.scala | 5 +- .../channel/states/e/OfflineStateSpec.scala | 5 +- .../channel/states/f/ShutdownStateSpec.scala | 5 +- .../states/g/NegotiatingStateSpec.scala | 7 +- .../channel/states/h/ClosingStateSpec.scala | 5 +- .../fr/acinq/eclair/db/ChannelStateSpec.scala | 128 ++++++++++++++++++ .../scala/fr/acinq/eclair/db/DummyDb.scala | 20 +++ .../interop/rustytests/RustyTestsSpec.scala | 5 +- .../acinq/eclair/router/BaseRouterSpec.scala | 3 +- pom.xml | 2 +- 35 files changed, 538 insertions(+), 87 deletions(-) create mode 100644 eclair-node/src/main/scala/fr/acinq/eclair/db/ChannelState.scala create mode 100644 eclair-node/src/main/scala/fr/acinq/eclair/db/JavaSerializer.scala create mode 100644 eclair-node/src/main/scala/fr/acinq/eclair/db/SimpleDb.scala create mode 100644 eclair-node/src/main/scala/fr/acinq/eclair/db/SimpleFileDb.scala create mode 100644 eclair-node/src/test/scala/fr/acinq/eclair/db/ChannelStateSpec.scala create mode 100644 eclair-node/src/test/scala/fr/acinq/eclair/db/DummyDb.scala diff --git a/eclair-node/src/main/resources/application.conf b/eclair-node/src/main/resources/application.conf index fc6c5dab9..803754b5a 100644 --- a/eclair-node/src/main/resources/application.conf +++ b/eclair-node/src/main/resources/application.conf @@ -24,6 +24,9 @@ eclair { b = 170 } } + db { + root = "~/.eclair" + } delay-blocks = 144 mindepth-blocks = 3 expiry-delta-blocks = 144 diff --git a/eclair-node/src/main/scala/fr/acinq/eclair/Boot.scala b/eclair-node/src/main/scala/fr/acinq/eclair/Boot.scala index c0346d056..b86017f05 100644 --- a/eclair-node/src/main/scala/fr/acinq/eclair/Boot.scala +++ b/eclair-node/src/main/scala/fr/acinq/eclair/Boot.scala @@ -8,14 +8,17 @@ import akka.http.scaladsl.Http import akka.stream.ActorMaterializer import akka.util.Timeout import com.typesafe.config.ConfigFactory -import fr.acinq.bitcoin.{Base58Check, OP_CHECKSIG, OP_DUP, OP_EQUALVERIFY, OP_HASH160, OP_PUSHDATA, Script} +import fr.acinq.bitcoin.Crypto.PublicKey +import fr.acinq.bitcoin.{Base58Check, BinaryData, OP_CHECKSIG, OP_DUP, OP_EQUALVERIFY, OP_HASH160, OP_PUSHDATA, Script} import fr.acinq.eclair.api.Service import fr.acinq.eclair.blockchain.peer.PeerClient import fr.acinq.eclair.blockchain.rpc.BitcoinJsonRPCClient import fr.acinq.eclair.blockchain.{ExtendedBitcoinClient, PeerWatcher} -import fr.acinq.eclair.channel.Register +import fr.acinq.eclair.channel.{Channel, Register} +import fr.acinq.eclair.crypto.TransportHandler.Serializer +import fr.acinq.eclair.db.{JavaSerializer, SimpleFileDb, SimpleTypedDb} import fr.acinq.eclair.gui.FxApp -import fr.acinq.eclair.io.{Server, Switchboard} +import fr.acinq.eclair.io.{Peer, PeerRecord, Server, Switchboard} import fr.acinq.eclair.payment._ import fr.acinq.eclair.router._ import grizzled.slf4j.Logging @@ -59,7 +62,7 @@ class Setup() extends Logging { implicit val formats = org.json4s.DefaultFormats implicit val ec = ExecutionContext.Implicits.global val (chain, blockCount, progress) = Await.result(bitcoin_client.client.invoke("getblockchaininfo").map(json => ((json \ "chain").extract[String], (json \ "blocks").extract[Long], (json \ "verificationprogress").extract[Double])), 10 seconds) - assert(chain == "testnet" || chain == "regtest" || chain == "segnet4", "you should be on testnet or regtest or segnet4") + assert(chain == "test" || chain == "regtest" || chain == "segnet4", "you should be on testnet or regtest or segnet4") assert(progress > 0.99, "bitcoind should be synchronized") Globals.blockCount.set(blockCount) val bitcoinVersion = Await.result(bitcoin_client.client.invoke("getinfo").map(json => (json \ "version").extract[String]), 10 seconds) @@ -82,6 +85,16 @@ class Setup() extends Logging { })) val fatalEventFuture = fatalEventPromise.future + val db = new SimpleFileDb(config.getString("eclair.db.root")) + val peerDb = Peer.makePeerDb(db) + val peers = peerDb.values + + val channelDb = Channel.makeChannelDb(db) + val channels = channelDb.values + + val routerDb = Router.makeRouterDb(db) + val routerStates = routerDb.values + val peer = system.actorOf(Props[PeerClient], "bitcoin-peer") val watcher = system.actorOf(PeerWatcher.props(bitcoin_client), name = "watcher") val paymentHandler = config.getString("eclair.payment-handler") match { @@ -90,9 +103,9 @@ class Setup() extends Logging { } val register = system.actorOf(Props(new Register), name = "register") val relayer = system.actorOf(Relayer.props(Globals.Node.privateKey, paymentHandler), name = "relayer") - val router = system.actorOf(Router.props(watcher), name = "router") + val router = system.actorOf(Router.props(watcher, db), name = "router") val paymentInitiator = system.actorOf(PaymentInitiator.props(Globals.Node.publicKey, router), "payment-initiator") - val switchboard = system.actorOf(Switchboard.props(watcher, router, relayer, finalScriptPubKey), name = "switchboard") + val switchboard = system.actorOf(Switchboard.props(watcher, router, relayer, finalScriptPubKey, db), name = "switchboard") val server = system.actorOf(Server.props(switchboard, new InetSocketAddress(config.getString("eclair.server.host"), config.getInt("eclair.server.port"))), "server") val _setup = this @@ -107,4 +120,10 @@ class Setup() extends Logging { Http().bindAndHandle(api.route, config.getString("eclair.api.host"), config.getInt("eclair.api.port")) onFailure { case t: Throwable => system.eventStream.publish(HTTPBindError) } + + def boostrap: Unit = { + peers.map(rec => switchboard ! rec) + channels.map(rec => switchboard ! rec) + routerStates.map(rec => router ! rec) + } } diff --git a/eclair-node/src/main/scala/fr/acinq/eclair/channel/Channel.scala b/eclair-node/src/main/scala/fr/acinq/eclair/channel/Channel.scala index 0fa190a0a..8eb76e862 100644 --- a/eclair-node/src/main/scala/fr/acinq/eclair/channel/Channel.scala +++ b/eclair-node/src/main/scala/fr/acinq/eclair/channel/Channel.scala @@ -8,7 +8,9 @@ import fr.acinq.eclair._ import fr.acinq.eclair.blockchain._ import fr.acinq.eclair.blockchain.peer.CurrentBlockCount import fr.acinq.eclair.channel.Helpers.{Closing, Funding} +import fr.acinq.eclair.crypto.TransportHandler.Serializer import fr.acinq.eclair.crypto.{Generators, ShaChain} +import fr.acinq.eclair.db.{ChannelState, JavaSerializer, SimpleDb, SimpleTypedDb} import fr.acinq.eclair.payment._ import fr.acinq.eclair.router.Announcements import fr.acinq.eclair.transactions._ @@ -19,16 +21,36 @@ import scala.concurrent.ExecutionContext import scala.concurrent.duration._ import scala.util.{Failure, Left, Success, Try} +case class ChannelRecord(id: Long, state: ChannelState) /** * Created by PM on 20/08/2015. */ object Channel { - def props(remote: ActorRef, blockchain: ActorRef, router: ActorRef, relayer: ActorRef) = Props(new Channel(remote, blockchain, router, relayer)) + def props(remote: ActorRef, blockchain: ActorRef, router: ActorRef, relayer: ActorRef, db: SimpleDb) = Props(new Channel(remote, blockchain, router, relayer, db)) + + def makeChannelDb(db: SimpleDb): SimpleTypedDb[Long, ChannelRecord] = { + def channelid2String(id: Long) = s"channel-$id" + + def string2channelid(s: String) = if (s.startsWith("channel-")) Some(s.stripPrefix("channel-").toLong) else None + + new SimpleTypedDb[Long, ChannelRecord]( + channelid2String, + string2channelid, + new Serializer[ChannelRecord] { + override def serialize(t: ChannelRecord): BinaryData = JavaSerializer.serialize(t) + + override def deserialize(bin: BinaryData): ChannelRecord = JavaSerializer.deserialize[ChannelRecord](bin) + }, + db + ) + } } -class Channel(val r: ActorRef, val blockchain: ActorRef, router: ActorRef, relayer: ActorRef)(implicit ec: ExecutionContext = ExecutionContext.Implicits.global) extends LoggingFSM[State, Data] { +class Channel(val r: ActorRef, val blockchain: ActorRef, router: ActorRef, relayer: ActorRef, db: SimpleDb)(implicit ec: ExecutionContext = ExecutionContext.Implicits.global) extends LoggingFSM[State, Data] { + import Channel._ + val channelDb = makeChannelDb(db) var remote = r var remoteNodeId: PublicKey = null @@ -96,6 +118,19 @@ class Channel(val r: ActorRef, val blockchain: ActorRef, router: ActorRef, relay case Event(inputFundee@INPUT_INIT_FUNDEE(remoteNodeId, _, localParams, _), Nothing) if !localParams.isFunder => this.remoteNodeId = remoteNodeId goto(WAIT_FOR_OPEN_CHANNEL) using DATA_WAIT_FOR_OPEN_CHANNEL(inputFundee) + + case Event(INPUT_RESTORED(channelId, cs@ChannelState(remotePubKey, state, data)), _) => + log.info(s"restoring channel $cs") + remoteNodeId = remotePubKey + data match { + case d: HasCommitments => + blockchain ! WatchSpent(self, d.commitments.commitInput.outPoint.txid, d.commitments.commitInput.outPoint.index.toInt, BITCOIN_FUNDING_SPENT) // TODO: should we wait for an acknowledgment from the watcher? + context.system.eventStream.publish(ChannelCreated(d.channelId, context.parent, self, d.commitments.localParams, remoteNodeId, Some(d.commitments))) + Register.createAlias(remoteNodeId, d.commitments.channelId) + + case _ => () + } + goto(OFFLINE) using data }) when(WAIT_FOR_OPEN_CHANNEL)(handleExceptions { @@ -828,6 +863,12 @@ class Channel(val r: ActorRef, val blockchain: ActorRef, router: ActorRef, relay } goto(NORMAL) + case Event(INPUT_RECONNECTED(r), d: DATA_NEGOTIATING) => + goto(NEGOTIATING) using (d) + + case Event(INPUT_RECONNECTED(r), d: DATA_CLOSING) => + goto(CLOSING) using (d) + case Event(c@CMD_ADD_HTLC(amountMsat, rHash, expiry, route, downstream_opt, do_commit), d@DATA_NORMAL(params, commitments, _)) => log.info(s"we are disconnected so we just include the add in our commitments") Try(Commitments.sendAdd(commitments, c)) match { @@ -897,7 +938,13 @@ class Channel(val r: ActorRef, val blockchain: ActorRef, router: ActorRef, relay } (stateData, nextStateData) match { case (from: HasCommitments, to: HasCommitments) => - // use the the transition callback to send messages + // use the the transition callback to store state and send messages + val previousChannelId = from.channelId + val nextChannelId = to.channelId + channelDb.put(nextChannelId, ChannelRecord(nextChannelId, ChannelState(remoteNodeId, currentState, nextStateData))) + if (nextChannelId != previousChannelId) { + channelDb.delete(previousChannelId) + } val nextMessages = to.commitments.unackedMessages val currentMessages = from.commitments.unackedMessages val diff = nextMessages.filterNot(c => currentMessages.contains(c)) diff --git a/eclair-node/src/main/scala/fr/acinq/eclair/channel/ChannelEvents.scala b/eclair-node/src/main/scala/fr/acinq/eclair/channel/ChannelEvents.scala index 68eece529..1e0604d83 100644 --- a/eclair-node/src/main/scala/fr/acinq/eclair/channel/ChannelEvents.scala +++ b/eclair-node/src/main/scala/fr/acinq/eclair/channel/ChannelEvents.scala @@ -10,7 +10,7 @@ import fr.acinq.bitcoin.{BinaryData, Satoshi} trait ChannelEvent -case class ChannelCreated(temporaryChannelId: Long, peer: ActorRef, channel: ActorRef, params: LocalParams, remoteNodeId: PublicKey) extends ChannelEvent +case class ChannelCreated(temporaryChannelId: Long, peer: ActorRef, channel: ActorRef, params: LocalParams, remoteNodeId: PublicKey, commitments: Option[Commitments] = None) extends ChannelEvent case class ChannelIdAssigned(channel: ActorRef, channelId: BinaryData, amount: Satoshi) extends ChannelEvent diff --git a/eclair-node/src/main/scala/fr/acinq/eclair/channel/ChannelTypes.scala b/eclair-node/src/main/scala/fr/acinq/eclair/channel/ChannelTypes.scala index 7ac44783b..d11251c7f 100644 --- a/eclair-node/src/main/scala/fr/acinq/eclair/channel/ChannelTypes.scala +++ b/eclair-node/src/main/scala/fr/acinq/eclair/channel/ChannelTypes.scala @@ -3,6 +3,7 @@ package fr.acinq.eclair.channel import akka.actor.ActorRef import fr.acinq.bitcoin.Crypto.{Point, PrivateKey, PublicKey, Scalar} import fr.acinq.bitcoin.{BinaryData, ScriptElt, Transaction} +import fr.acinq.eclair.db.ChannelState import fr.acinq.eclair.payment.{Local, Origin, Relayed} import fr.acinq.eclair.transactions.CommitmentSpec import fr.acinq.eclair.transactions.Transactions.CommitTx @@ -63,6 +64,7 @@ case object INPUT_NO_MORE_HTLCS case object INPUT_CLOSE_COMPLETE_TIMEOUT case object INPUT_DISCONNECTED case class INPUT_RECONNECTED(remote: ActorRef) +case class INPUT_RESTORED(channelId: Long, channelstate: ChannelState) sealed trait BitcoinEvent case object BITCOIN_FUNDING_DEPTHOK extends BitcoinEvent diff --git a/eclair-node/src/main/scala/fr/acinq/eclair/channel/Register.scala b/eclair-node/src/main/scala/fr/acinq/eclair/channel/Register.scala index bf3441cf2..8d584c15e 100644 --- a/eclair-node/src/main/scala/fr/acinq/eclair/channel/Register.scala +++ b/eclair-node/src/main/scala/fr/acinq/eclair/channel/Register.scala @@ -32,7 +32,7 @@ class Register extends Actor with ActorLogging { override def receive: Receive = main(Map()) def main(channels: Map[String, ActorRef]): Receive = { - case ChannelCreated(temporaryChannelId, _, channel, _, _) => + case ChannelCreated(temporaryChannelId, _, channel, _, _, _) => context.watch(channel) context become main(channels + (java.lang.Long.toHexString(temporaryChannelId) -> channel)) diff --git a/eclair-node/src/main/scala/fr/acinq/eclair/db/ChannelState.scala b/eclair-node/src/main/scala/fr/acinq/eclair/db/ChannelState.scala new file mode 100644 index 000000000..9c3e6eff1 --- /dev/null +++ b/eclair-node/src/main/scala/fr/acinq/eclair/db/ChannelState.scala @@ -0,0 +1,22 @@ +package fr.acinq.eclair.db + +/** + * Created by fabrice on 25/02/17. + */ + +import fr.acinq.bitcoin.BinaryData +import fr.acinq.bitcoin.Crypto.PublicKey +import fr.acinq.eclair.channel.{Data, State} +import fr.acinq.eclair.crypto.TransportHandler + +case class ChannelState(remotePubKey: PublicKey, state: State, data: Data) { + def serialize = ChannelState.serializer.serialize(this) +} + +object ChannelState { + val serializer = new TransportHandler.Serializer[ChannelState] { + def serialize(cs: ChannelState): BinaryData = JavaSerializer.serialize(cs) + + def deserialize(input: BinaryData): ChannelState = JavaSerializer.deserialize[ChannelState](input) + } +} \ No newline at end of file diff --git a/eclair-node/src/main/scala/fr/acinq/eclair/db/JavaSerializer.scala b/eclair-node/src/main/scala/fr/acinq/eclair/db/JavaSerializer.scala new file mode 100644 index 000000000..cf75802a0 --- /dev/null +++ b/eclair-node/src/main/scala/fr/acinq/eclair/db/JavaSerializer.scala @@ -0,0 +1,24 @@ +package fr.acinq.eclair.db + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream} + +import fr.acinq.bitcoin.BinaryData + + +/** + * Created by fabrice on 17/02/17. + */ +object JavaSerializer { + def serialize[T](cs: T): BinaryData = { + val bos = new ByteArrayOutputStream() + val oos = new ObjectOutputStream(bos) + oos.writeObject(cs) + bos.toByteArray + } + + def deserialize[T](input: BinaryData): T = { + val bis = new ByteArrayInputStream(input) + val osi = new ObjectInputStream(bis) + osi.readObject().asInstanceOf[T] + } +} \ No newline at end of file diff --git a/eclair-node/src/main/scala/fr/acinq/eclair/db/SimpleDb.scala b/eclair-node/src/main/scala/fr/acinq/eclair/db/SimpleDb.scala new file mode 100644 index 000000000..82a5811b3 --- /dev/null +++ b/eclair-node/src/main/scala/fr/acinq/eclair/db/SimpleDb.scala @@ -0,0 +1,28 @@ +package fr.acinq.eclair.db + +import fr.acinq.bitcoin.BinaryData +import fr.acinq.eclair.crypto.TransportHandler + +/** + * Created by fabrice on 25/02/17. + */ +trait SimpleDb { + // @formatter:off + def put(k: String, v: BinaryData) : Unit + def get(k: String) : Option[BinaryData] + def delete(k: String) : Boolean + def keys: Seq[String] + def values: Seq[BinaryData] + // @formatter:on +} + +class SimpleTypedDb[K,V](id2string: K => String, string2id: String => Option[K], serializer: TransportHandler.Serializer[V], db: SimpleDb) { + // @formatter:off + def put(k: K, v: V) = db.put(id2string(k), serializer.serialize(v)) + def get(k: K): Option[V] = db.get(id2string(k)).map(serializer.deserialize) + def delete(k: K) : Boolean = db.delete(id2string(k)) + def keys: Seq[K] = db.keys.map(string2id).flatten + def values: Seq[V] = keys.map(get).flatten + // @formatter:on +} + diff --git a/eclair-node/src/main/scala/fr/acinq/eclair/db/SimpleFileDb.scala b/eclair-node/src/main/scala/fr/acinq/eclair/db/SimpleFileDb.scala new file mode 100644 index 000000000..320cbe279 --- /dev/null +++ b/eclair-node/src/main/scala/fr/acinq/eclair/db/SimpleFileDb.scala @@ -0,0 +1,43 @@ +package fr.acinq.eclair.db + +import java.io.File +import java.nio.file.{Files, Paths} + +import fr.acinq.bitcoin.BinaryData +import grizzled.slf4j.Logging + +import scala.util.Try + +/** + * Created by fabrice on 25/02/17. + */ +object SimpleFileDb { + def expandUserHomeDirectory(fileName: String): String = { + if (fileName == "~") System.getProperty("user.home") + else if (fileName.startsWith("~/")) System.getProperty("user.home") + fileName.drop(1) + else fileName + } +} + +case class SimpleFileDb(directory: String) extends SimpleDb with Logging { + + import SimpleFileDb._ + + val root = expandUserHomeDirectory(directory) + + new File(root).mkdirs() + + + override def put(key: String, value: BinaryData): Unit = { + logger.debug(s"put $key -> $value") + Files.write(Paths.get(root, key), value) + } + + override def get(key: String): Option[BinaryData] = Try(Files.readAllBytes(Paths.get(root, key))).toOption.map(a => BinaryData(a)) + + override def delete(key: String): Boolean = Paths.get(root, key).toFile.delete() + + override def keys: Seq[String] = new File(root).list() + + override def values: Seq[BinaryData] = keys.map(get).flatten +} diff --git a/eclair-node/src/main/scala/fr/acinq/eclair/gui/FxApp.scala b/eclair-node/src/main/scala/fr/acinq/eclair/gui/FxApp.scala index 3753fb8e8..c32662484 100644 --- a/eclair-node/src/main/scala/fr/acinq/eclair/gui/FxApp.scala +++ b/eclair-node/src/main/scala/fr/acinq/eclair/gui/FxApp.scala @@ -42,6 +42,7 @@ class FxApp extends Application with Logging { val guiUpdater = setup.system.actorOf(Props(classOf[GUIUpdater], primaryStage, controller, setup), "gui-updater") setup.system.eventStream.subscribe(guiUpdater, classOf[ChannelEvent]) setup.system.eventStream.subscribe(guiUpdater, classOf[NetworkEvent]) + setup.boostrap import scala.concurrent.ExecutionContext.Implicits.global setup.fatalEventFuture onSuccess { diff --git a/eclair-node/src/main/scala/fr/acinq/eclair/gui/GUIUpdater.scala b/eclair-node/src/main/scala/fr/acinq/eclair/gui/GUIUpdater.scala index ef27049e0..d13f5fa7b 100644 --- a/eclair-node/src/main/scala/fr/acinq/eclair/gui/GUIUpdater.scala +++ b/eclair-node/src/main/scala/fr/acinq/eclair/gui/GUIUpdater.scala @@ -33,7 +33,7 @@ class GUIUpdater(primaryStage: Stage, mainController: MainController, setup: Set def main(m: Map[ActorRef, ChannelPaneController]): Receive = { - case ChannelCreated(_, peer, channel, params, theirNodeId) => + case ChannelCreated(_, peer, channel, params, theirNodeId, commitments_opt) => log.info(s"new channel: $channel") val loader = new FXMLLoader(getClass.getResource("/gui/main/channelPane.fxml")) @@ -43,6 +43,13 @@ class GUIUpdater(primaryStage: Stage, mainController: MainController, setup: Set channelPaneController.nodeId.setText(s"$theirNodeId") channelPaneController.funder.setText(if (params.isFunder) "Yes" else "No") + commitments_opt.map(commitments => { + val bal = commitments.localCommit.spec.toLocalMsat.toDouble / (commitments.localCommit.spec.toLocalMsat.toDouble + commitments.localCommit.spec.toRemoteMsat.toDouble) + channelPaneController.capacity.setText(s"${millisatoshi2millibtc(MilliSatoshi(commitments.localCommit.spec.totalFunds)).amount}") + channelPaneController.amountUs.setText(s"${millisatoshi2millibtc(MilliSatoshi(commitments.localCommit.spec.toLocalMsat)).amount}") + channelPaneController.balanceBar.setProgress(bal) + }) + channelPaneController.close.setOnAction(new EventHandler[ActionEvent] { override def handle(event: ActionEvent) = channel ! CMD_CLOSE(None) }) diff --git a/eclair-node/src/main/scala/fr/acinq/eclair/io/Peer.scala b/eclair-node/src/main/scala/fr/acinq/eclair/io/Peer.scala index 27d0dc88c..201f92de3 100644 --- a/eclair-node/src/main/scala/fr/acinq/eclair/io/Peer.scala +++ b/eclair-node/src/main/scala/fr/acinq/eclair/io/Peer.scala @@ -6,7 +6,8 @@ import akka.actor.{ActorRef, LoggingFSM, PoisonPill, Props, Terminated} import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey} import fr.acinq.bitcoin.{BinaryData, DeterministicWallet} import fr.acinq.eclair.channel._ -import fr.acinq.eclair.crypto.TransportHandler.{HandshakeCompleted, Listener} +import fr.acinq.eclair.crypto.TransportHandler.{HandshakeCompleted, Listener, Serializer} +import fr.acinq.eclair.db.{ChannelState, JavaSerializer, SimpleDb, SimpleTypedDb} import fr.acinq.eclair.io.Switchboard.{NewChannel, NewConnection} import fr.acinq.eclair.router.SendRoutingState import fr.acinq.eclair.wire._ @@ -34,18 +35,30 @@ case object DISCONNECTED extends State case object INITIALIZING extends State case object CONNECTED extends State +case class PeerRecord(id: PublicKey, address: Option[InetSocketAddress]) + // @formatter:on /** * Created by PM on 26/08/2016. */ -class Peer(remoteNodeId: PublicKey, address_opt: Option[InetSocketAddress], watcher: ActorRef, router: ActorRef, relayer: ActorRef, defaultFinalScriptPubKey: BinaryData) extends LoggingFSM[State, Data] { +class Peer(remoteNodeId: PublicKey, address_opt: Option[InetSocketAddress], watcher: ActorRef, router: ActorRef, relayer: ActorRef, defaultFinalScriptPubKey: BinaryData, db: SimpleDb) extends LoggingFSM[State, Data] { import Peer._ + val peerDb = makePeerDb(db) startWith(DISCONNECTED, DisconnectedData(Nil)) when(DISCONNECTED) { + case Event(c: ChannelRecord, d@DisconnectedData(offlineChannels)) if c.state.remotePubKey != remoteNodeId => + log.warning(s"received channel data for the wrong peer ${c.state.remotePubKey}") + stay + + case Event(c: ChannelRecord, d@DisconnectedData(offlineChannels)) => + val (channel, _) = createChannel(null, c.id, false, db) + channel ! INPUT_RESTORED(c.id, c.state) + stay using d.copy(offlineChannels = offlineChannels :+ HotChannel(c.id, channel)) + case Event(c: NewChannel, d@DisconnectedData(offlineChannels)) => stay using d.copy(offlineChannels = offlineChannels :+ BrandNewChannel(c)) @@ -97,7 +110,7 @@ class Peer(remoteNodeId: PublicKey, address_opt: Option[InetSocketAddress], watc case Event(c: NewChannel, d@ConnectedData(transport, remoteInit, channels)) => log.info(s"requesting a new channel to $remoteNodeId with fundingSatoshis=${c.fundingSatoshis} and pushMsat=${c.pushMsat}") val temporaryChannelId = Platform.currentTime - val (channel, localParams) = createChannel(transport, temporaryChannelId, funder = true) + val (channel, localParams) = createChannel(transport, temporaryChannelId, funder = true, db) channel ! INPUT_INIT_FUNDER(remoteNodeId, temporaryChannelId, c.fundingSatoshis.amount, c.pushMsat.amount, localParams, remoteInit) stay using d.copy(channels = channels + (temporaryChannelId -> channel)) @@ -121,7 +134,7 @@ class Peer(remoteNodeId: PublicKey, address_opt: Option[InetSocketAddress], watc case Event(msg: OpenChannel, d@ConnectedData(transport, remoteInit, channels)) => log.info(s"accepting a new channel to $remoteNodeId") val temporaryChannelId = msg.temporaryChannelId - val (channel, localParams) = createChannel(transport, temporaryChannelId, funder = false) + val (channel, localParams) = createChannel(transport, temporaryChannelId, funder = false, db) channel ! INPUT_INIT_FUNDEE(remoteNodeId, temporaryChannelId, localParams, remoteInit) channel ! msg stay using d.copy(channels = channels + (temporaryChannelId -> channel)) @@ -149,9 +162,9 @@ class Peer(remoteNodeId: PublicKey, address_opt: Option[InetSocketAddress], watc stay using d.copy(channels = channels - channelId) } - def createChannel(transport: ActorRef, temporaryChannelId: Long, funder: Boolean): (ActorRef, LocalParams) = { + def createChannel(transport: ActorRef, temporaryChannelId: Long, funder: Boolean, db: SimpleDb): (ActorRef, LocalParams) = { val localParams = makeChannelParams(temporaryChannelId, defaultFinalScriptPubKey, funder) - val channel = context.actorOf(Channel.props(transport, watcher, router, relayer), s"channel-$temporaryChannelId") + val channel = context.actorOf(Channel.props(transport, watcher, router, relayer, db), s"channel-$temporaryChannelId") context watch channel (channel, localParams) } @@ -160,7 +173,7 @@ class Peer(remoteNodeId: PublicKey, address_opt: Option[InetSocketAddress], watc object Peer { - def props(remoteNodeId: PublicKey, address_opt: Option[InetSocketAddress], watcher: ActorRef, router: ActorRef, relayer: ActorRef, defaultFinalScriptPubKey: BinaryData) = Props(classOf[Peer], remoteNodeId, address_opt, watcher, router, relayer, defaultFinalScriptPubKey) + def props(remoteNodeId: PublicKey, address_opt: Option[InetSocketAddress], watcher: ActorRef, router: ActorRef, relayer: ActorRef, defaultFinalScriptPubKey: BinaryData, db: SimpleDb) = Props(classOf[Peer], remoteNodeId, address_opt, watcher, router, relayer, defaultFinalScriptPubKey, db) def generateKey(keyPath: Seq[Long]): PrivateKey = DeterministicWallet.derivePrivateKey(Globals.Node.extendedPrivateKey, keyPath).privateKey @@ -184,4 +197,20 @@ object Peer { localFeatures = Globals.local_features ) + def makePeerDb(db: SimpleDb): SimpleTypedDb[PublicKey, PeerRecord] = { + def peerid2String(id: PublicKey) = s"peer-$id" + + def string2peerid(s: String) = if (s.startsWith("peer-")) Some(PublicKey(BinaryData(s.stripPrefix("peer-")))) else None + + new SimpleTypedDb[PublicKey, PeerRecord]( + peerid2String, + string2peerid, + new Serializer[PeerRecord] { + override def serialize(t: PeerRecord): BinaryData = JavaSerializer.serialize(t) + + override def deserialize(bin: BinaryData): PeerRecord = JavaSerializer.deserialize[PeerRecord](bin) + }, + db + ) + } } diff --git a/eclair-node/src/main/scala/fr/acinq/eclair/io/Switchboard.scala b/eclair-node/src/main/scala/fr/acinq/eclair/io/Switchboard.scala index 13e6be71f..c319c47d7 100644 --- a/eclair-node/src/main/scala/fr/acinq/eclair/io/Switchboard.scala +++ b/eclair-node/src/main/scala/fr/acinq/eclair/io/Switchboard.scala @@ -6,20 +6,35 @@ import akka.actor.{Actor, ActorLogging, ActorRef, Props, Status, Terminated} import fr.acinq.bitcoin.Crypto.PublicKey import fr.acinq.bitcoin.{BinaryData, MilliSatoshi, Satoshi, ScriptElt} import fr.acinq.eclair.Globals +import fr.acinq.eclair.channel.ChannelRecord import fr.acinq.eclair.crypto.TransportHandler.HandshakeCompleted +import fr.acinq.eclair.db.{ChannelState, SimpleDb} /** * Ties network connections to peers. * Created by PM on 14/02/2017. */ -class Switchboard(watcher: ActorRef, router: ActorRef, relayer: ActorRef, defaultFinalScriptPubKey: BinaryData) extends Actor with ActorLogging { +class Switchboard(watcher: ActorRef, router: ActorRef, relayer: ActorRef, defaultFinalScriptPubKey: BinaryData, db: SimpleDb) extends Actor with ActorLogging { import Switchboard._ + val peerDb = Peer.makePeerDb(db) def receive: Receive = main(Map(), Map()) + def main(peers: Map[PublicKey, ActorRef], connections: Map[PublicKey, ActorRef]): Receive = { + case PeerRecord(publicKey, address) if peers.contains(publicKey) => () + + case PeerRecord(publicKey, address) => + val peer = createPeer(publicKey, address) + context become main(peers + (publicKey -> peer), connections) + + case ChannelRecord(id, ChannelState(remotePubKey, _, _)) if !peers.contains(remotePubKey) => + log.warning(s"received channel data for unknown peer $remotePubKey") + + case channelRecord: ChannelRecord => peers(channelRecord.state.remotePubKey) forward channelRecord + case NewConnection(Globals.Node.publicKey, _, _) => sender ! Status.Failure(new RuntimeException("cannot open connection with oneself")) @@ -57,12 +72,15 @@ class Switchboard(watcher: ActorRef, router: ActorRef, relayer: ActorRef, defaul } - def createPeer(remoteNodeId: PublicKey, address_opt: Option[InetSocketAddress]) = context.actorOf(Peer.props(remoteNodeId, address_opt, watcher, router, relayer, defaultFinalScriptPubKey), name = s"peer-$remoteNodeId") + def createPeer(remoteNodeId: PublicKey, address_opt: Option[InetSocketAddress]) = { + peerDb.put(remoteNodeId, PeerRecord(remoteNodeId, address_opt)) + context.actorOf(Peer.props(remoteNodeId, address_opt, watcher, router, relayer, defaultFinalScriptPubKey, db), name = s"peer-$remoteNodeId") + } } object Switchboard { - def props(watcher: ActorRef, router: ActorRef, relayer: ActorRef, defaultFinalScriptPubKey: BinaryData) = Props(classOf[Switchboard], watcher, router, relayer, defaultFinalScriptPubKey) + def props(watcher: ActorRef, router: ActorRef, relayer: ActorRef, defaultFinalScriptPubKey: BinaryData, db: SimpleDb) = Props(classOf[Switchboard], watcher, router, relayer, defaultFinalScriptPubKey, db) // @formatter:off case class NewChannel(fundingSatoshis: Satoshi, pushMsat: MilliSatoshi) diff --git a/eclair-node/src/main/scala/fr/acinq/eclair/router/Router.scala b/eclair-node/src/main/scala/fr/acinq/eclair/router/Router.scala index df3d2e678..576e0b37d 100644 --- a/eclair-node/src/main/scala/fr/acinq/eclair/router/Router.scala +++ b/eclair-node/src/main/scala/fr/acinq/eclair/router/Router.scala @@ -10,6 +10,8 @@ import fr.acinq.bitcoin.Script.{pay2wsh, write} import fr.acinq.eclair._ import fr.acinq.eclair.blockchain.{GetTx, GetTxResponse, WatchEventSpent, WatchSpent} import fr.acinq.eclair.channel._ +import fr.acinq.eclair.crypto.TransportHandler.Serializer +import fr.acinq.eclair.db.{JavaSerializer, SimpleDb, SimpleTypedDb} import fr.acinq.eclair.transactions.Scripts import fr.acinq.eclair.wire._ import org.jgrapht.alg.shortestpath.DijkstraShortestPath @@ -34,14 +36,15 @@ case class SendRoutingState(to: ActorRef) * Created by PM on 24/05/2016. */ -class Router(watcher: ActorRef) extends Actor with ActorLogging { +class Router(watcher: ActorRef, db: SimpleDb) extends Actor with ActorLogging { import Router._ + val routerDb = makeRouterDb(db) import ExecutionContext.Implicits.global context.system.scheduler.schedule(10 seconds, 60 seconds, self, 'tick_broadcast) - def receive: Receive = main(nodes = Map(), channels = Map(), updates = Map(), rebroadcast = Nil, awaiting = Set(), stash = Nil) + def receive: Receive = main(State.empty) def mainWithLog(nodes: Map[BinaryData, NodeAnnouncement], channels: Map[Long, ChannelAnnouncement], @@ -50,18 +53,16 @@ class Router(watcher: ActorRef) extends Actor with ActorLogging { awaiting: Set[ChannelAnnouncement], stash: Seq[RoutingMessage]) = { log.info(s"current status channels=${channels.size} nodes=${nodes.size} updates=${updates.size}") - main(nodes, channels, updates, rebroadcast, awaiting, stash) + val state = State(nodes, channels, updates, rebroadcast, awaiting, stash) + routerDb.put("router.state", state) + main(state) } - def main( - nodes: Map[BinaryData, NodeAnnouncement], - channels: Map[Long, ChannelAnnouncement], - updates: Map[ChannelDesc, ChannelUpdate], - rebroadcast: Seq[RoutingMessage], - awaiting: Set[ChannelAnnouncement], - stash: Seq[RoutingMessage]): Receive = { + def main(state: State): Receive = { + case newState: State => context become main(newState) case SendRoutingState(remote) => + import state._ log.info(s"info sending all announcements to $remote: channels=${channels.size} nodes=${nodes.size} updates=${updates.size}") channels.values.foreach(remote ! _) updates.values.foreach(remote ! _) @@ -72,19 +73,24 @@ class Router(watcher: ActorRef) extends Actor with ActorLogging { log.error(s"bad signature for announcement $c") sender ! Error(0, "bad announcement sig!!!".getBytes()) - case c: ChannelAnnouncement if channels.containsKey(c.channelId) => + case c: ChannelAnnouncement if state.channels.containsKey(c.channelId) => log.debug(s"ignoring $c (duplicate)") case c: ChannelAnnouncement => + import state._ val (blockHeight, txIndex, outputIndex) = fromShortId(c.channelId) log.info(s"retrieving raw tx with blockHeight=$blockHeight and txIndex=$txIndex") watcher ! GetTx(blockHeight, txIndex, outputIndex, c) - context become main(nodes, channels, updates, rebroadcast, awaiting + c, stash) + val state1 = state.copy(awaiting = awaiting + c) + routerDb.put("router.state", state1) + context become main(state1) + case GetTxResponse(tx, isSpendable, c: ChannelAnnouncement) if !isSpendable => log.debug(s"ignoring $c (funding tx spent)") case GetTxResponse(tx, _, c: ChannelAnnouncement) => + import state._ // TODO: check sigs // TODO: blacklist if already received same channel id and different node ids val (_, _, outputIndex) = fromShortId(c.channelId) @@ -108,7 +114,8 @@ class Router(watcher: ActorRef) extends Actor with ActorLogging { log.error(s"bad signature for announcement $n") sender ! Error(0, "bad announcement sig!!!".getBytes()) - case WatchEventSpent(BITCOIN_FUNDING_OTHER_CHANNEL_SPENT(channelId), tx) if channels.containsKey(channelId) => + case WatchEventSpent(BITCOIN_FUNDING_OTHER_CHANNEL_SPENT(channelId), tx) if state.channels.containsKey(channelId) => + import state._ val lostChannel = channels(channelId) log.info(s"funding tx of channelId=$channelId has been spent by txid=${tx.txid}") log.info(s"removed channel channelId=$channelId") @@ -126,37 +133,42 @@ class Router(watcher: ActorRef) extends Actor with ActorLogging { val lostNodes = isNodeLost(lostChannel.nodeId1).toSeq ++ isNodeLost(lostChannel.nodeId2).toSeq context become mainWithLog(nodes -- lostNodes, channels - channelId, updates.filterKeys(_.id != channelId), rebroadcast, awaiting, stash) - case n: NodeAnnouncement if awaiting.size > 0 => - context become main(nodes, channels, updates, rebroadcast, awaiting, stash :+ n) + case n: NodeAnnouncement if state.awaiting.size > 0 => + val state1 = state.copy(stash = state.stash :+ n) + routerDb.put("router.state", state1) + context become main(state1) - case n: NodeAnnouncement if !channels.values.exists(c => c.nodeId1 == n.nodeId || c.nodeId2 == n.nodeId) => + case n: NodeAnnouncement if !state.channels.values.exists(c => c.nodeId1 == n.nodeId || c.nodeId2 == n.nodeId) => log.debug(s"ignoring $n (no related channel found)") - case n: NodeAnnouncement if nodes.containsKey(n.nodeId) && nodes(n.nodeId).timestamp >= n.timestamp => + case n: NodeAnnouncement if state.nodes.containsKey(n.nodeId) && state.nodes(n.nodeId).timestamp >= n.timestamp => log.debug(s"ignoring announcement $n (old timestamp or duplicate)") - case n: NodeAnnouncement if nodes.containsKey(n.nodeId) => + case n: NodeAnnouncement if state.nodes.containsKey(n.nodeId) => + import state._ log.info(s"updated node nodeId=${n.nodeId}") context.system.eventStream.publish(NodeUpdated(n)) context become mainWithLog(nodes + (n.nodeId -> n), channels, updates, rebroadcast :+ n, awaiting, stash) case n: NodeAnnouncement => + import state._ log.info(s"added node nodeId=${n.nodeId}") context.system.eventStream.publish(NodeDiscovered(n)) context become mainWithLog(nodes + (n.nodeId -> n), channels, updates, rebroadcast :+ n, awaiting, stash) - case u: ChannelUpdate if awaiting.size > 0 => - context become main(nodes, channels, updates, rebroadcast, awaiting, stash :+ u) + case u: ChannelUpdate if state.awaiting.size > 0 => + context become main(state.copy(stash = state.stash :+ u)) - case u: ChannelUpdate if !channels.contains(u.channelId) => + case u: ChannelUpdate if !state.channels.contains(u.channelId) => log.debug(s"ignoring $u (no related channel found)") - case u: ChannelUpdate if !Announcements.checkSig(u, getDesc(u, channels(u.channelId)).a) => + case u: ChannelUpdate if !Announcements.checkSig(u, getDesc(u, state.channels(u.channelId)).a) => // TODO: (dirty) this will make the origin channel close the connection log.error(s"bad signature for announcement $u") sender ! Error(0, "bad announcement sig!!!".getBytes()) case u: ChannelUpdate => + import state._ val channel = channels(u.channelId) val desc = getDesc(u, channel) if (updates.contains(desc) && updates(desc).timestamp >= u.timestamp) { @@ -165,23 +177,26 @@ class Router(watcher: ActorRef) extends Actor with ActorLogging { context become mainWithLog(nodes, channels, updates + (desc -> u), rebroadcast :+ u, awaiting, stash) } - case 'tick_broadcast if rebroadcast.size == 0 => + case 'tick_broadcast if state.rebroadcast.size == 0 => // no-op case 'tick_broadcast => + import state._ log.info(s"broadcasting ${rebroadcast.size} routing messages") rebroadcast.foreach(context.actorSelection(Register.actorPathToPeers) ! _) - context become main(nodes, channels, updates, Nil, awaiting, stash) + val state1 = State(nodes, channels, updates, Nil, awaiting, stash) + routerDb.put("router.state", state1) + context become main(state1) - case 'nodes => sender ! nodes.values + case 'nodes => sender ! state.nodes.values - case 'channels => sender ! channels.values + case 'channels => sender ! state.channels.values - case 'updates => sender ! updates.values + case 'updates => sender ! state.updates.values - case 'dot => graph2dot(nodes, channels) pipeTo sender + case 'dot => graph2dot(state.nodes, state.channels) pipeTo sender - case RouteRequest(start, end) => findRoute(start, end, updates).map(RouteResponse(_)) pipeTo sender + case RouteRequest(start, end) => findRoute(start, end, state.updates).map(RouteResponse(_)) pipeTo sender case other => log.warning(s"unhandled message $other") } @@ -190,8 +205,22 @@ class Router(watcher: ActorRef) extends Actor with ActorLogging { object Router { - def props(watcher: ActorRef) = Props(classOf[Router], watcher) + def props(watcher: ActorRef, db: SimpleDb) = Props(classOf[Router], watcher, db) + def makeRouterDb(db: SimpleDb) = { + // we use a single key: router.state + new SimpleTypedDb[String, State]( + _ => "router.state", + s => if (s == "router.state") Some("router.state") else None, + new Serializer[State] { + override def serialize(t: State): BinaryData = JavaSerializer.serialize(t.fixme) + + override def deserialize(bin: BinaryData): State = JavaSerializer.deserialize[State](bin) + }, + db + ) + + } def getDesc(u: ChannelUpdate, channel: ChannelAnnouncement): ChannelDesc = { require(u.flags.data.size == 2, s"invalid flags length ${u.flags.data.size} != 2") // the least significant bit tells us if it is node1 or node2 @@ -251,4 +280,18 @@ object Router { } } + + case class State(nodes: Map[BinaryData, NodeAnnouncement], + channels: Map[Long, ChannelAnnouncement], + updates: Map[ChannelDesc, ChannelUpdate], + rebroadcast: Seq[RoutingMessage], + awaiting: Set[ChannelAnnouncement], + stash: Seq[RoutingMessage]) { + // see http://stackoverflow.com/questions/32900862/map-can-not-be-serializable-in-scala :( + def fixme = this.copy(nodes = nodes.map(identity), channels = channels.map(identity), updates = updates.map(identity)) + } + + object State { + val empty = State(nodes = Map(), channels = Map(), updates = Map(), rebroadcast = Nil, awaiting = Set(), stash = Nil) + } } diff --git a/eclair-node/src/test/scala/fr/acinq/eclair/channel/ThroughputSpec.scala b/eclair-node/src/test/scala/fr/acinq/eclair/channel/ThroughputSpec.scala index 774321b5a..b24e64a41 100644 --- a/eclair-node/src/test/scala/fr/acinq/eclair/channel/ThroughputSpec.scala +++ b/eclair-node/src/test/scala/fr/acinq/eclair/channel/ThroughputSpec.scala @@ -8,6 +8,7 @@ import fr.acinq.bitcoin.{BinaryData, Crypto} import fr.acinq.eclair.TestConstants.{Alice, Bob} import fr.acinq.eclair._ import fr.acinq.eclair.blockchain._ +import fr.acinq.eclair.db.DummyDb import fr.acinq.eclair.payment.Relayer import fr.acinq.eclair.wire.{Init, UpdateAddHtlc} import org.junit.runner.RunWith @@ -53,8 +54,8 @@ class ThroughputSpec extends FunSuite { } }), "payment-handler") val relayer = system.actorOf(Relayer.props(Globals.Node.privateKey, paymentHandler)) - val alice = system.actorOf(Channel.props(pipe, blockchain, ???, relayer), "a") - val bob = system.actorOf(Channel.props(pipe, blockchain, ???, relayer), "b") + val alice = system.actorOf(Channel.props(pipe, blockchain, ???, relayer, new DummyDb()), "a") + val bob = system.actorOf(Channel.props(pipe, blockchain, ???, relayer, new DummyDb()), "b") val aliceInit = Init(Alice.channelParams.globalFeatures, Alice.channelParams.localFeatures) val bobInit = Init(Bob.channelParams.globalFeatures, Bob.channelParams.localFeatures) alice ! INPUT_INIT_FUNDER(Bob.id, 0, TestConstants.fundingSatoshis, TestConstants.pushMsat, Alice.channelParams, bobInit) diff --git a/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/FuzzySpec.scala b/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/FuzzySpec.scala index 8d1ea0293..b1d772595 100644 --- a/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/FuzzySpec.scala +++ b/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/FuzzySpec.scala @@ -7,6 +7,7 @@ import fr.acinq.eclair.TestConstants.{Alice, Bob} import fr.acinq.eclair._ import fr.acinq.eclair.blockchain._ import fr.acinq.eclair.channel.{Data, State, _} +import fr.acinq.eclair.db.DummyDb import fr.acinq.eclair.payment._ import fr.acinq.eclair.router.Hop import fr.acinq.eclair.wire._ @@ -35,8 +36,8 @@ class FuzzySpec extends TestkitBaseClass with StateTestsHelperMethods { val relayerA = system.actorOf(Relayer.props(Globals.Node.privateKey, paymentHandlerA), "relayer-a") val relayerB = system.actorOf(Relayer.props(Globals.Node.privateKey, paymentHandlerB), "relayer-b") val router = TestProbe() - val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(pipe, alice2blockchain.ref, router.ref, relayerA)) - val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(pipe, bob2blockchain.ref, router.ref, relayerB)) + val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(pipe, alice2blockchain.ref, router.ref, relayerA, new DummyDb())) + val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(pipe, bob2blockchain.ref, router.ref, relayerB, new DummyDb())) within(30 seconds) { val aliceInit = Init(Alice.channelParams.globalFeatures, Alice.channelParams.localFeatures) val bobInit = Init(Bob.channelParams.globalFeatures, Bob.channelParams.localFeatures) diff --git a/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForAcceptChannelStateSpec.scala b/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForAcceptChannelStateSpec.scala index f8c4704d0..aa086edad 100644 --- a/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForAcceptChannelStateSpec.scala +++ b/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForAcceptChannelStateSpec.scala @@ -5,6 +5,7 @@ import akka.testkit.{TestFSMRef, TestProbe} import fr.acinq.eclair.TestConstants.{Alice, Bob} import fr.acinq.eclair.blockchain.PeerWatcher import fr.acinq.eclair.channel.{WAIT_FOR_FUNDING_INTERNAL, _} +import fr.acinq.eclair.db.DummyDb import fr.acinq.eclair.wire.{AcceptChannel, Error, Init, OpenChannel} import fr.acinq.eclair.{TestBitcoinClient, TestConstants, TestkitBaseClass} import org.junit.runner.RunWith @@ -28,8 +29,8 @@ class WaitForAcceptChannelStateSpec extends TestkitBaseClass { val bob2blockchain = TestProbe() val relayer = TestProbe() val router = TestProbe() - val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(alice2bob.ref, alice2blockchain.ref, router.ref, relayer.ref)) - val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bob2alice.ref, bob2blockchain.ref, router.ref, relayer.ref)) + val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(alice2bob.ref, alice2blockchain.ref, router.ref, relayer.ref, new DummyDb())) + val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bob2alice.ref, bob2blockchain.ref, router.ref, relayer.ref, new DummyDb())) val aliceInit = Init(Alice.channelParams.globalFeatures, Alice.channelParams.localFeatures) val bobInit = Init(Bob.channelParams.globalFeatures, Bob.channelParams.localFeatures) within(30 seconds) { diff --git a/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForOpenChannelStateSpec.scala b/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForOpenChannelStateSpec.scala index b3f8c60bd..6994c42f9 100644 --- a/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForOpenChannelStateSpec.scala +++ b/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/a/WaitForOpenChannelStateSpec.scala @@ -3,6 +3,7 @@ package fr.acinq.eclair.channel.states.a import akka.testkit.{TestFSMRef, TestProbe} import fr.acinq.eclair.TestConstants.{Alice, Bob} import fr.acinq.eclair.channel._ +import fr.acinq.eclair.db.DummyDb import fr.acinq.eclair.wire.{Error, Init, OpenChannel} import fr.acinq.eclair.{TestConstants, TestkitBaseClass} import org.junit.runner.RunWith @@ -25,8 +26,8 @@ class WaitForOpenChannelStateSpec extends TestkitBaseClass { val bob2blockchain = TestProbe() val relayer = TestProbe() val router = TestProbe() - val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(alice2bob.ref, alice2blockchain.ref, router.ref, relayer.ref)) - val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bob2alice.ref, bob2blockchain.ref, router.ref, relayer.ref)) + val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(alice2bob.ref, alice2blockchain.ref, router.ref, relayer.ref, new DummyDb())) + val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bob2alice.ref, bob2blockchain.ref, router.ref, relayer.ref, new DummyDb())) val aliceInit = Init(Alice.channelParams.globalFeatures, Alice.channelParams.localFeatures) val bobInit = Init(Bob.channelParams.globalFeatures, Bob.channelParams.localFeatures) within(30 seconds) { diff --git a/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForFundingCreatedInternalStateSpec.scala b/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForFundingCreatedInternalStateSpec.scala index 86cc4e21f..e854bc3bb 100644 --- a/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForFundingCreatedInternalStateSpec.scala +++ b/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForFundingCreatedInternalStateSpec.scala @@ -5,6 +5,7 @@ import akka.testkit.{TestFSMRef, TestProbe} import fr.acinq.eclair.TestConstants.{Alice, Bob} import fr.acinq.eclair.blockchain.{MakeFundingTx, PeerWatcher} import fr.acinq.eclair.channel._ +import fr.acinq.eclair.db.DummyDb import fr.acinq.eclair.wire._ import fr.acinq.eclair.{TestBitcoinClient, TestConstants, TestkitBaseClass} import org.junit.runner.RunWith @@ -28,8 +29,8 @@ class WaitForFundingCreatedInternalStateSpec extends TestkitBaseClass { val bob2blockchain = TestProbe() val relayer = TestProbe() val router = TestProbe() - val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(alice2bob.ref, alice2blockchain.ref, router.ref, relayer.ref)) - val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bob2alice.ref, bob2blockchain.ref, router.ref, relayer.ref)) + val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(alice2bob.ref, alice2blockchain.ref, router.ref, relayer.ref, new DummyDb())) + val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bob2alice.ref, bob2blockchain.ref, router.ref, relayer.ref, new DummyDb())) val aliceInit = Init(Alice.channelParams.globalFeatures, Alice.channelParams.localFeatures) val bobInit = Init(Bob.channelParams.globalFeatures, Bob.channelParams.localFeatures) within(30 seconds) { diff --git a/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForFundingCreatedStateSpec.scala b/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForFundingCreatedStateSpec.scala index 0929a9e21..0cdbfce19 100644 --- a/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForFundingCreatedStateSpec.scala +++ b/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForFundingCreatedStateSpec.scala @@ -5,6 +5,7 @@ import akka.testkit.{TestFSMRef, TestProbe} import fr.acinq.eclair.TestConstants.{Alice, Bob} import fr.acinq.eclair.blockchain.{PeerWatcher, WatchConfirmed, WatchSpent} import fr.acinq.eclair.channel._ +import fr.acinq.eclair.db.DummyDb import fr.acinq.eclair.wire._ import fr.acinq.eclair.{TestBitcoinClient, TestConstants, TestkitBaseClass} import org.junit.runner.RunWith @@ -27,8 +28,8 @@ class WaitForFundingCreatedStateSpec extends TestkitBaseClass { val bob2blockchain = TestProbe() val relayer = TestProbe() val router = TestProbe() - val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(alice2bob.ref, blockchainA, router.ref, relayer.ref)) - val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bob2alice.ref, bob2blockchain.ref, router.ref, relayer.ref)) + val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(alice2bob.ref, blockchainA, router.ref, relayer.ref, new DummyDb())) + val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bob2alice.ref, bob2blockchain.ref, router.ref, relayer.ref, new DummyDb())) val aliceInit = Init(Alice.channelParams.globalFeatures, Alice.channelParams.localFeatures) val bobInit = Init(Bob.channelParams.globalFeatures, Bob.channelParams.localFeatures) within(30 seconds) { diff --git a/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForFundingSignedStateSpec.scala b/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForFundingSignedStateSpec.scala index 7a972f8c0..28c19721d 100644 --- a/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForFundingSignedStateSpec.scala +++ b/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/b/WaitForFundingSignedStateSpec.scala @@ -6,6 +6,7 @@ import fr.acinq.bitcoin.BinaryData import fr.acinq.eclair.TestConstants.{Alice, Bob} import fr.acinq.eclair.blockchain._ import fr.acinq.eclair.channel._ +import fr.acinq.eclair.db.DummyDb import fr.acinq.eclair.wire.{AcceptChannel, Error, FundingCreated, FundingSigned, Init, OpenChannel} import fr.acinq.eclair.{TestBitcoinClient, TestConstants, TestkitBaseClass} import org.junit.runner.RunWith @@ -29,8 +30,8 @@ class WaitForFundingSignedStateSpec extends TestkitBaseClass { val bob2blockchain = TestProbe() val relayer = TestProbe() val router = TestProbe() - val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(alice2bob.ref, alice2blockchain.ref, router.ref, relayer.ref)) - val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bob2alice.ref, bob2blockchain.ref, router.ref, relayer.ref)) + val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(alice2bob.ref, alice2blockchain.ref, router.ref, relayer.ref, new DummyDb())) + val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bob2alice.ref, bob2blockchain.ref, router.ref, relayer.ref, new DummyDb())) val aliceInit = Init(Alice.channelParams.globalFeatures, Alice.channelParams.localFeatures) val bobInit = Init(Bob.channelParams.globalFeatures, Bob.channelParams.localFeatures) within(30 seconds) { diff --git a/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForAnnSignaturesStateSpec.scala b/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForAnnSignaturesStateSpec.scala index 9dd02181c..12b1354ff 100644 --- a/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForAnnSignaturesStateSpec.scala +++ b/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForAnnSignaturesStateSpec.scala @@ -5,6 +5,7 @@ import akka.testkit.{TestFSMRef, TestProbe} import fr.acinq.eclair.TestConstants.{Alice, Bob} import fr.acinq.eclair.blockchain._ import fr.acinq.eclair.channel._ +import fr.acinq.eclair.db.DummyDb import fr.acinq.eclair.wire._ import fr.acinq.eclair.{TestBitcoinClient, TestConstants, TestkitBaseClass} import org.junit.runner.RunWith @@ -28,8 +29,8 @@ class WaitForAnnSignaturesStateSpec extends TestkitBaseClass { val bob2blockchain = TestProbe() val relayer = TestProbe() val router = TestProbe() - val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(alice2bob.ref, alice2blockchain.ref, router.ref, relayer.ref)) - val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bob2alice.ref, bob2blockchain.ref, router.ref, relayer.ref)) + val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(alice2bob.ref, alice2blockchain.ref, router.ref, relayer.ref, new DummyDb())) + val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bob2alice.ref, bob2blockchain.ref, router.ref, relayer.ref, new DummyDb())) val (aliceParams, bobParams) = (Alice.channelParams.copy(localFeatures = "01"), Bob.channelParams.copy(localFeatures = "01")) val aliceInit = Init(aliceParams.globalFeatures, aliceParams.localFeatures) val bobInit = Init(bobParams.globalFeatures, bobParams.localFeatures) diff --git a/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForFundingConfirmedStateSpec.scala b/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForFundingConfirmedStateSpec.scala index e55d54922..05daa4d2c 100644 --- a/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForFundingConfirmedStateSpec.scala +++ b/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForFundingConfirmedStateSpec.scala @@ -5,6 +5,7 @@ import akka.testkit.{TestFSMRef, TestProbe} import fr.acinq.eclair.TestConstants.{Alice, Bob} import fr.acinq.eclair.blockchain._ import fr.acinq.eclair.channel._ +import fr.acinq.eclair.db.DummyDb import fr.acinq.eclair.wire.{AcceptChannel, Error, FundingCreated, FundingLocked, FundingSigned, Init, OpenChannel} import fr.acinq.eclair.{TestBitcoinClient, TestConstants, TestkitBaseClass} import org.junit.runner.RunWith @@ -28,8 +29,8 @@ class WaitForFundingConfirmedStateSpec extends TestkitBaseClass { val bob2blockchain = TestProbe() val relayer = TestProbe() val router = TestProbe() - val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(alice2bob.ref, alice2blockchain.ref, router.ref, relayer.ref)) - val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bob2alice.ref, bob2blockchain.ref, router.ref, relayer.ref)) + val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(alice2bob.ref, alice2blockchain.ref, router.ref, relayer.ref, new DummyDb())) + val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bob2alice.ref, bob2blockchain.ref, router.ref, relayer.ref, new DummyDb())) val aliceInit = Init(Alice.channelParams.globalFeatures, Alice.channelParams.localFeatures) val bobInit = Init(Bob.channelParams.globalFeatures, Bob.channelParams.localFeatures) within(30 seconds) { diff --git a/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForFundingLockedStateSpec.scala b/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForFundingLockedStateSpec.scala index a4f6ad682..36d020f14 100644 --- a/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForFundingLockedStateSpec.scala +++ b/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/c/WaitForFundingLockedStateSpec.scala @@ -5,6 +5,7 @@ import akka.testkit.{TestFSMRef, TestProbe} import fr.acinq.eclair.TestConstants.{Alice, Bob} import fr.acinq.eclair.blockchain._ import fr.acinq.eclair.channel._ +import fr.acinq.eclair.db.DummyDb import fr.acinq.eclair.wire._ import fr.acinq.eclair.{TestBitcoinClient, TestConstants, TestkitBaseClass} import org.junit.runner.RunWith @@ -29,8 +30,8 @@ class WaitForFundingLockedStateSpec extends TestkitBaseClass { val bob2blockchain = TestProbe() val relayer = TestProbe() val router = TestProbe() - val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(alice2bob.ref, alice2blockchain.ref, router.ref, relayer.ref)) - val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bob2alice.ref, bob2blockchain.ref, router.ref, relayer.ref)) + val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(alice2bob.ref, alice2blockchain.ref, router.ref, relayer.ref, new DummyDb())) + val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bob2alice.ref, bob2blockchain.ref, router.ref, relayer.ref, new DummyDb())) val (aliceParams, bobParams) = if (test.tags.contains("public")) { (Alice.channelParams.copy(localFeatures = "01"), Bob.channelParams.copy(localFeatures = "01")) } else { diff --git a/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/e/NormalStateSpec.scala b/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/e/NormalStateSpec.scala index c5f113aed..9ada3c629 100644 --- a/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/e/NormalStateSpec.scala +++ b/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/e/NormalStateSpec.scala @@ -9,6 +9,7 @@ import fr.acinq.eclair.blockchain._ import fr.acinq.eclair.blockchain.peer.CurrentBlockCount import fr.acinq.eclair.channel.states.StateTestsHelperMethods import fr.acinq.eclair.channel.{Data, State, _} +import fr.acinq.eclair.db.DummyDb import fr.acinq.eclair.payment.{Bind, Local, Relayed} import fr.acinq.eclair.transactions.{IN, OUT} import fr.acinq.eclair.wire.{ClosingSigned, CommitSig, Error, RevokeAndAck, Shutdown, UpdateAddHtlc, UpdateFailHtlc, UpdateFulfillHtlc} @@ -34,8 +35,8 @@ class NormalStateSpec extends TestkitBaseClass with StateTestsHelperMethods { val bob2blockchain = TestProbe() val relayer = TestProbe() val router = TestProbe() - val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(alice2bob.ref, alice2blockchain.ref, router.ref, relayer.ref)) - val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bob2alice.ref, bob2blockchain.ref, router.ref, relayer.ref)) + val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(alice2bob.ref, alice2blockchain.ref, router.ref, relayer.ref, new DummyDb())) + val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bob2alice.ref, bob2blockchain.ref, router.ref, relayer.ref, new DummyDb())) within(30 seconds) { reachNormal(alice, bob, alice2bob, bob2alice, blockchainA, alice2blockchain, bob2blockchain) awaitCond(alice.stateName == NORMAL) diff --git a/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/e/OfflineStateSpec.scala b/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/e/OfflineStateSpec.scala index 458e22d2d..7a0b495ff 100644 --- a/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/e/OfflineStateSpec.scala +++ b/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/e/OfflineStateSpec.scala @@ -6,6 +6,7 @@ import fr.acinq.bitcoin.BinaryData import fr.acinq.eclair.blockchain._ import fr.acinq.eclair.channel.states.StateTestsHelperMethods import fr.acinq.eclair.channel.{Data, State, _} +import fr.acinq.eclair.db.DummyDb import fr.acinq.eclair.wire.{CommitSig, RevokeAndAck, UpdateAddHtlc} import fr.acinq.eclair.{TestBitcoinClient, TestkitBaseClass} import org.junit.runner.RunWith @@ -29,8 +30,8 @@ class OfflineStateSpec extends TestkitBaseClass with StateTestsHelperMethods { val bob2blockchain = TestProbe() val relayer = TestProbe() val router = TestProbe() - val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(alice2bob.ref, alice2blockchain.ref, router.ref, relayer.ref)) - val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bob2alice.ref, bob2blockchain.ref, router.ref, relayer.ref)) + val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(alice2bob.ref, alice2blockchain.ref, router.ref, relayer.ref, new DummyDb())) + val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bob2alice.ref, bob2blockchain.ref, router.ref, relayer.ref, new DummyDb())) within(30 seconds) { reachNormal(alice, bob, alice2bob, bob2alice, blockchainA, alice2blockchain, bob2blockchain) awaitCond(alice.stateName == NORMAL) diff --git a/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/f/ShutdownStateSpec.scala b/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/f/ShutdownStateSpec.scala index 02f8d3ac9..958b6de01 100644 --- a/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/f/ShutdownStateSpec.scala +++ b/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/f/ShutdownStateSpec.scala @@ -10,6 +10,7 @@ import fr.acinq.eclair.blockchain._ import fr.acinq.eclair.blockchain.peer.CurrentBlockCount import fr.acinq.eclair.channel.states.StateTestsHelperMethods import fr.acinq.eclair.channel.{Data, State, _} +import fr.acinq.eclair.db.DummyDb import fr.acinq.eclair.wire.{CommitSig, Error, RevokeAndAck, Shutdown, UpdateAddHtlc, UpdateFailHtlc, UpdateFulfillHtlc} import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner @@ -32,8 +33,8 @@ class ShutdownStateSpec extends TestkitBaseClass with StateTestsHelperMethods { val bob2blockchain = TestProbe() val relayer = TestProbe() val router = TestProbe() - val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(alice2bob.ref, alice2blockchain.ref, router.ref, relayer.ref)) - val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bob2alice.ref, bob2blockchain.ref, router.ref, relayer.ref)) + val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(alice2bob.ref, alice2blockchain.ref, router.ref, relayer.ref, new DummyDb())) + val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bob2alice.ref, bob2blockchain.ref, router.ref, relayer.ref, new DummyDb())) within(35 seconds) { reachNormal(alice, bob, alice2bob, bob2alice, blockchainA, alice2blockchain, bob2blockchain) val sender = TestProbe() diff --git a/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/g/NegotiatingStateSpec.scala b/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/g/NegotiatingStateSpec.scala index 62233defc..437f6a12e 100644 --- a/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/g/NegotiatingStateSpec.scala +++ b/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/g/NegotiatingStateSpec.scala @@ -2,11 +2,12 @@ package fr.acinq.eclair.channel.states.g import akka.actor.Props import akka.testkit.{TestFSMRef, TestProbe} -import fr.acinq.eclair.{TestkitBaseClass, TestBitcoinClient} +import fr.acinq.eclair.{TestBitcoinClient, TestkitBaseClass} import fr.acinq.eclair.TestConstants.{Alice, Bob} import fr.acinq.eclair.blockchain._ import fr.acinq.eclair.channel.states.StateTestsHelperMethods import fr.acinq.eclair.channel.{Data, State, _} +import fr.acinq.eclair.db.DummyDb import fr.acinq.eclair.wire.{ClosingSigned, Error, Shutdown} import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner @@ -30,8 +31,8 @@ class NegotiatingStateSpec extends TestkitBaseClass with StateTestsHelperMethods val relayer = TestProbe() // note that alice.initialFeeRate != bob.initialFeeRate val router = TestProbe() - val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(alice2bob.ref, alice2blockchain.ref, router.ref, relayer.ref)) - val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bob2alice.ref, bob2blockchain.ref, router.ref, relayer.ref)) + val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(alice2bob.ref, alice2blockchain.ref, router.ref, relayer.ref, new DummyDb())) + val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bob2alice.ref, bob2blockchain.ref, router.ref, relayer.ref, new DummyDb())) within(30 seconds) { reachNormal(alice, bob, alice2bob, bob2alice, blockchainA, alice2blockchain, bob2blockchain) val sender = TestProbe() diff --git a/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/h/ClosingStateSpec.scala b/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/h/ClosingStateSpec.scala index fadeadaae..3cc80d7d7 100644 --- a/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/h/ClosingStateSpec.scala +++ b/eclair-node/src/test/scala/fr/acinq/eclair/channel/states/h/ClosingStateSpec.scala @@ -8,6 +8,7 @@ import fr.acinq.eclair.TestConstants.{Alice, Bob} import fr.acinq.eclair.blockchain._ import fr.acinq.eclair.channel.states.StateTestsHelperMethods import fr.acinq.eclair.channel.{Data, State, _} +import fr.acinq.eclair.db.DummyDb import fr.acinq.eclair.wire._ import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner @@ -30,8 +31,8 @@ class ClosingStateSpec extends TestkitBaseClass with StateTestsHelperMethods { val bob2blockchain = TestProbe() val relayer = TestProbe() val router = TestProbe() - val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(alice2bob.ref, alice2blockchain.ref, router.ref, relayer.ref)) - val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bob2alice.ref, bob2blockchain.ref, router.ref, relayer.ref)) + val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(alice2bob.ref, alice2blockchain.ref, router.ref, relayer.ref, new DummyDb())) + val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(bob2alice.ref, bob2blockchain.ref, router.ref, relayer.ref, new DummyDb())) within(30 seconds) { reachNormal(alice, bob, alice2bob, bob2alice, blockchainA, alice2blockchain, bob2blockchain) diff --git a/eclair-node/src/test/scala/fr/acinq/eclair/db/ChannelStateSpec.scala b/eclair-node/src/test/scala/fr/acinq/eclair/db/ChannelStateSpec.scala new file mode 100644 index 000000000..a58a46107 --- /dev/null +++ b/eclair-node/src/test/scala/fr/acinq/eclair/db/ChannelStateSpec.scala @@ -0,0 +1,128 @@ +package fr.acinq.eclair.db + +import java.net.InetSocketAddress + +import fr.acinq.bitcoin.Crypto.{PrivateKey, Scalar} +import fr.acinq.bitcoin.{BinaryData, Crypto, MilliSatoshi, Satoshi, Transaction} +import fr.acinq.eclair.channel.Helpers.Funding +import fr.acinq.eclair.channel._ +import fr.acinq.eclair.crypto.{Generators, ShaChain} +import fr.acinq.eclair.transactions.Transactions.CommitTx +import fr.acinq.eclair.transactions._ +import fr.acinq.eclair.wire.{CommitSig, Init, OpenChannel, UpdateAddHtlc} +import org.junit.runner.RunWith +import org.scalatest.FunSuite +import org.scalatest.junit.JUnitRunner + +/** + * Created by fabrice on 07/02/17. + */ +@RunWith(classOf[JUnitRunner]) +class ChannelStateSpec extends FunSuite { + + import ChannelStateSpec._ + + test("basic serialization test") { + val data = openChannel + val bin = ChannelState.serializer.serialize(data) + val check = ChannelState.serializer.deserialize(bin) + assert(data == check) + } + + test("basic serialization test (NORMAL)") { + val data = normal + val bin = ChannelState.serializer.serialize(data) + val check = ChannelState.serializer.deserialize(bin) + assert(data == check) + } +} + +object ChannelStateSpec { + val localParams = LocalParams( + dustLimitSatoshis = Satoshi(546).toLong, + maxHtlcValueInFlightMsat = 50, + channelReserveSatoshis = 10000, + htlcMinimumMsat = 50000, + feeratePerKw = 15000, + toSelfDelay = 144, + maxAcceptedHtlcs = 50, + fundingPrivKey = PrivateKey(BinaryData("01" * 32) :+ 1.toByte), + revocationSecret = Scalar(BinaryData("02" * 32)), + paymentKey = PrivateKey(BinaryData("03" * 32) :+ 1.toByte), + delayedPaymentKey = Scalar(BinaryData("04" * 32)), + defaultFinalScriptPubKey = Nil, + shaSeed = BinaryData("05" * 32), + isFunder = true, + globalFeatures = "foo".getBytes(), + localFeatures = "bar".getBytes()) + + val remoteParams = RemoteParams( + dustLimitSatoshis = Satoshi(546).toLong, + maxHtlcValueInFlightMsat = 50, + channelReserveSatoshis = 10000, + htlcMinimumMsat = 50000, + feeratePerKw = 15000, + toSelfDelay = 144, + maxAcceptedHtlcs = 50, + fundingPubKey = PrivateKey(BinaryData("01" * 32) :+ 1.toByte).publicKey, + revocationBasepoint = Scalar(BinaryData("02" * 32)).toPoint, + paymentBasepoint = Scalar(BinaryData("03" * 32)).toPoint, + delayedPaymentBasepoint = Scalar(BinaryData("04" * 32)).toPoint, + globalFeatures = "foo".getBytes(), + localFeatures = "bar".getBytes()) + + val channelParams = ChannelParams(localParams, remoteParams, 42000, 100) + + val openChannel = { + val open = OpenChannel(temporaryChannelId = 1000, + fundingSatoshis = 42000, + pushMsat = 0, + dustLimitSatoshis = localParams.dustLimitSatoshis, + maxHtlcValueInFlightMsat = localParams.maxHtlcValueInFlightMsat, + channelReserveSatoshis = localParams.channelReserveSatoshis, + htlcMinimumMsat = localParams.htlcMinimumMsat, + feeratePerKw = localParams.feeratePerKw, + toSelfDelay = localParams.toSelfDelay, + maxAcceptedHtlcs = localParams.maxAcceptedHtlcs, + fundingPubkey = localParams.fundingPrivKey.publicKey, + revocationBasepoint = localParams.revocationSecret.toPoint, + paymentBasepoint = localParams.paymentKey.toPoint, + delayedPaymentBasepoint = localParams.delayedPaymentKey.toPoint, + firstPerCommitmentPoint = Generators.perCommitPoint(localParams.shaSeed, 0)) + + ChannelState( + PrivateKey(BinaryData("01" * 32) :+ 1.toByte).publicKey, + WAIT_FOR_ACCEPT_CHANNEL, + DATA_WAIT_FOR_ACCEPT_CHANNEL(INPUT_INIT_FUNDER(PrivateKey(BinaryData("01" * 32) :+ 1.toByte).publicKey, 42, 10000, 1000000, localParams, Init("foo".getBytes, "bar".getBytes)), open)) + } + + val paymentPreimages = Seq( + BinaryData("0000000000000000000000000000000000000000000000000000000000000000"), + BinaryData("0101010101010101010101010101010101010101010101010101010101010101"), + BinaryData("0202020202020202020202020202020202020202020202020202020202020202"), + BinaryData("0303030303030303030303030303030303030303030303030303030303030303"), + BinaryData("0404040404040404040404040404040404040404040404040404040404040404") + ) + + val htlcs = Seq( + Htlc(IN, UpdateAddHtlc(0, 0, MilliSatoshi(1000000).amount, 500, Crypto.sha256(paymentPreimages(0)), BinaryData("")), None), + Htlc(IN, UpdateAddHtlc(0, 0, MilliSatoshi(2000000).amount, 501, Crypto.sha256(paymentPreimages(1)), BinaryData("")), None), + Htlc(OUT, UpdateAddHtlc(0, 0, MilliSatoshi(2000000).amount, 502, Crypto.sha256(paymentPreimages(2)), BinaryData("")), None), + Htlc(OUT, UpdateAddHtlc(0, 0, MilliSatoshi(3000000).amount, 503, Crypto.sha256(paymentPreimages(3)), BinaryData("")), None), + Htlc(IN, UpdateAddHtlc(0, 0, MilliSatoshi(4000000).amount, 504, Crypto.sha256(paymentPreimages(4)), BinaryData("")), None) + ) + + val fundingTx = Transaction.read("0200000001adbb20ea41a8423ea937e76e8151636bf6093b70eaff942930d20576600521fd000000006b48304502210090587b6201e166ad6af0227d3036a9454223d49a1f11839c1a362184340ef0240220577f7cd5cca78719405cbf1de7414ac027f0239ef6e214c90fcaab0454d84b3b012103535b32d5eb0a6ed0982a0479bbadc9868d9836f6ba94dd5a63be16d875069184ffffffff028096980000000000220020c015c4a6be010e21657068fc2e6a9d02b27ebe4d490a25846f7237f104d1a3cd20256d29010000001600143ca33c2e4446f4a305f23c80df8ad1afdcf652f900000000") + val fundingAmount = fundingTx.txOut(0).amount + val commitmentInput = Funding.makeFundingInputInfo(fundingTx.hash, 0, fundingAmount, localParams.fundingPrivKey.publicKey, remoteParams.fundingPubKey) + + val localCommit = LocalCommit(0, CommitmentSpec(htlcs.toSet, 1500, 50000, 700000), PublishableTxs(CommitTx(commitmentInput, Transaction(2, Nil, Nil, 0)), Nil), CommitSig(42, BinaryData("01" * 64), Nil)) + val remoteCommit = RemoteCommit(0, CommitmentSpec(htlcs.toSet, 1500, 50000, 700000), BinaryData("0303030303030303030303030303030303030303030303030303030303030303"), Scalar(BinaryData("04" * 32)).toPoint) + val commitments = Commitments(localParams, remoteParams, localCommit, remoteCommit, LocalChanges(Nil, Nil, Nil), RemoteChanges(Nil, Nil, Nil), + localNextHtlcId = 0L, + remoteNextCommitInfo = Right(null), // TODO: we will receive their next per-commitment point in the next message, so we temporarily put an empty byte array + remoteNextHtlcId = 0L, + commitInput = commitmentInput, remotePerCommitmentSecrets = ShaChain.init, channelId = 0L, unackedMessages = Nil) + + val normal = ChannelState(PrivateKey(BinaryData("01" * 32) :+ 1.toByte).publicKey, NORMAL, DATA_NORMAL(ChannelParams(localParams, remoteParams, 42000, 10), commitments, None)) +} diff --git a/eclair-node/src/test/scala/fr/acinq/eclair/db/DummyDb.scala b/eclair-node/src/test/scala/fr/acinq/eclair/db/DummyDb.scala new file mode 100644 index 000000000..d6b983b11 --- /dev/null +++ b/eclair-node/src/test/scala/fr/acinq/eclair/db/DummyDb.scala @@ -0,0 +1,20 @@ +package fr.acinq.eclair.db + +import fr.acinq.bitcoin.BinaryData + +/** + * Created by fabrice on 20/02/17. + */ +class DummyDb extends SimpleDb { + val map = collection.mutable.HashMap.empty[String, BinaryData] + + override def put(key: String, value: BinaryData): Unit = map.put(key, value) + + override def get(key: String): Option[BinaryData] = map.get(key) + + override def delete(key: String): Boolean = map.remove(key).isDefined + + override def keys: Seq[String] = map.keys.toSeq + + override def values: Seq[BinaryData] = map.values.toSeq +} \ No newline at end of file diff --git a/eclair-node/src/test/scala/fr/acinq/eclair/interop/rustytests/RustyTestsSpec.scala b/eclair-node/src/test/scala/fr/acinq/eclair/interop/rustytests/RustyTestsSpec.scala index 02adc02ab..140773ca5 100644 --- a/eclair-node/src/test/scala/fr/acinq/eclair/interop/rustytests/RustyTestsSpec.scala +++ b/eclair-node/src/test/scala/fr/acinq/eclair/interop/rustytests/RustyTestsSpec.scala @@ -8,6 +8,7 @@ import akka.testkit.{TestFSMRef, TestKit, TestProbe} import fr.acinq.eclair.TestConstants.{Alice, Bob} import fr.acinq.eclair.blockchain.PeerWatcher import fr.acinq.eclair.channel._ +import fr.acinq.eclair.db.DummyDb import fr.acinq.eclair.payment.NoopPaymentHandler import fr.acinq.eclair.wire.Init import fr.acinq.eclair.{Globals, TestBitcoinClient} @@ -36,8 +37,8 @@ class RustyTestsSpec extends TestKit(ActorSystem("test")) with Matchers with fix // we just bypass the relayer for this test val relayer = paymentHandler val router = TestProbe() - val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(pipe, blockchainA, router.ref, relayer)) - val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(pipe, blockchainB, router.ref, relayer)) + val alice: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(pipe, blockchainA, router.ref, relayer, new DummyDb())) + val bob: TestFSMRef[State, Data, Channel] = TestFSMRef(new Channel(pipe, blockchainB, router.ref, relayer, new DummyDb())) val aliceInit = Init(Alice.channelParams.globalFeatures, Alice.channelParams.localFeatures) val bobInit = Init(Bob.channelParams.globalFeatures, Bob.channelParams.localFeatures) // alice and bob will both have 1 000 000 sat diff --git a/eclair-node/src/test/scala/fr/acinq/eclair/router/BaseRouterSpec.scala b/eclair-node/src/test/scala/fr/acinq/eclair/router/BaseRouterSpec.scala index 0a2c87231..69fbf6b3f 100644 --- a/eclair-node/src/test/scala/fr/acinq/eclair/router/BaseRouterSpec.scala +++ b/eclair-node/src/test/scala/fr/acinq/eclair/router/BaseRouterSpec.scala @@ -6,6 +6,7 @@ import fr.acinq.bitcoin.Crypto.PrivateKey import fr.acinq.bitcoin.Script.{pay2wsh, write} import fr.acinq.bitcoin.{Satoshi, Transaction, TxOut} import fr.acinq.eclair.blockchain.{GetTx, GetTxResponse, WatchSpent} +import fr.acinq.eclair.db.{DummyDb, SimpleDb} import fr.acinq.eclair.router.Announcements._ import fr.acinq.eclair.transactions.Scripts import fr.acinq.eclair.wire._ @@ -67,7 +68,7 @@ abstract class BaseRouterSpec extends TestkitBaseClass { within(30 seconds) { // first we set up the router val watcher = TestProbe() - val router = system.actorOf(Router.props(watcher.ref)) + val router = system.actorOf(Router.props(watcher.ref, new DummyDb)) // we announce channels router ! chan_ab router ! chan_bc diff --git a/pom.xml b/pom.xml index 9809e1ae9..e0255182c 100644 --- a/pom.xml +++ b/pom.xml @@ -46,7 +46,7 @@ 2.11 2.4.16 1.2 - 0.9.9 + 0.9.10-SNAPSHOT