mirror of
https://github.com/ACINQ/eclair.git
synced 2024-11-20 10:39:19 +01:00
re-enabled payment state machine and tests
This commit is contained in:
parent
287bb60d94
commit
c02c17b6b0
@ -87,7 +87,7 @@ class Setup() extends Logging {
|
||||
case "noop" => system.actorOf(Props[NoopPaymentHandler], name = "payment-handler")
|
||||
}
|
||||
val router = system.actorOf(Router.props(watcher, Globals.Node.announcement), name = "router")
|
||||
val paymentInitiator = system.actorOf(PaymentInitiator.props(router, blockCount), "payment-spawner")
|
||||
val paymentInitiator = system.actorOf(PaymentInitiator.props(Globals.Node.publicKey, router, blockCount), "payment-spawner")
|
||||
val register = system.actorOf(Register.props(watcher, router, paymentHandler, finalScriptPubKey), name = "register")
|
||||
val server = system.actorOf(Server.props(config.getString("eclair.server.host"), config.getInt("eclair.server.port"), register), "server")
|
||||
|
||||
|
@ -79,7 +79,7 @@ trait Service extends Logging {
|
||||
(register ? ListChannels).mapTo[Iterable[ActorRef]]
|
||||
.flatMap(l => Future.sequence(l.map(c => c ? CMD_GETINFO)))
|
||||
case JsonRPCBody(_, _, "network", _) =>
|
||||
(router ? 'network).mapTo[Iterable[ChannelDesc]]
|
||||
(router ? 'channels).mapTo[Iterable[ChannelDesc]]
|
||||
case JsonRPCBody(_, _, "addhtlc", JInt(amount) :: JString(rhash) :: JString(nodeId) :: Nil) =>
|
||||
(paymentInitiator ? CreatePayment(amount.toInt, BinaryData(rhash), BinaryData(nodeId))).mapTo[ChannelEvent]
|
||||
case JsonRPCBody(_, _, "genh", _) =>
|
||||
|
@ -1,9 +1,12 @@
|
||||
package fr.acinq
|
||||
|
||||
import com.google.protobuf.ByteString
|
||||
import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey}
|
||||
import fr.acinq.bitcoin._
|
||||
import lightning.bitcoin_pubkey
|
||||
|
||||
import scala.util.Random
|
||||
|
||||
package object eclair {
|
||||
|
||||
implicit def bin2pubkey(in: BinaryData) = bitcoin_pubkey(ByteString.copyFrom(in))
|
||||
@ -22,4 +25,11 @@ package object eclair {
|
||||
def fromShortId(id: Long): (Int, Int, Int) =
|
||||
(((id >> 40) & 0xFFFFFF).toInt, ((id >> 16) & 0xFFFFFF).toInt, (id & 0xFFFF).toInt)
|
||||
|
||||
def randomKey: PrivateKey = PrivateKey({
|
||||
val bin = Array.fill[Byte](32)(0)
|
||||
// TODO: use secure random
|
||||
Random.nextBytes(bin)
|
||||
bin
|
||||
}, compressed = true)
|
||||
|
||||
}
|
@ -1,12 +1,13 @@
|
||||
package fr.acinq.eclair.payment
|
||||
|
||||
import akka.actor.{Actor, ActorLogging, ActorRef, Props}
|
||||
import fr.acinq.bitcoin.BinaryData
|
||||
import fr.acinq.eclair.blockchain.peer.CurrentBlockCount
|
||||
|
||||
/**
|
||||
* Created by PM on 29/08/2016.
|
||||
*/
|
||||
class PaymentInitiator(router: ActorRef, initialBlockCount: Long) extends Actor with ActorLogging {
|
||||
class PaymentInitiator(sourceNodeId: BinaryData, router: ActorRef, initialBlockCount: Long) extends Actor with ActorLogging {
|
||||
|
||||
context.system.eventStream.subscribe(self, classOf[CurrentBlockCount])
|
||||
|
||||
@ -15,12 +16,12 @@ class PaymentInitiator(router: ActorRef, initialBlockCount: Long) extends Actor
|
||||
def main(currentBlockCount: Long): Receive = {
|
||||
case CurrentBlockCount(count) => context.become(main(currentBlockCount))
|
||||
case c: CreatePayment =>
|
||||
val payFsm = context.actorOf(PaymentLifecycle.props(router, initialBlockCount))
|
||||
val payFsm = context.actorOf(PaymentLifecycle.props(sourceNodeId, router, initialBlockCount))
|
||||
payFsm forward c
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
object PaymentInitiator {
|
||||
def props(router: ActorRef, initialBlockCount: Long) = Props(classOf[PaymentInitiator], router, initialBlockCount)
|
||||
def props(sourceNodeId: BinaryData, router: ActorRef, initialBlockCount: Long) = Props(classOf[PaymentInitiator], sourceNodeId, router, initialBlockCount)
|
||||
}
|
||||
|
@ -1,17 +1,16 @@
|
||||
package fr.acinq.eclair.payment
|
||||
|
||||
import akka.actor.Status.Failure
|
||||
import akka.actor.{ActorRef, FSM, LoggingFSM, Props, Status}
|
||||
import fr.acinq.bitcoin.BinaryData
|
||||
import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey}
|
||||
import fr.acinq.bitcoin.Crypto.PublicKey
|
||||
import fr.acinq.eclair._
|
||||
import fr.acinq.eclair.blockchain.peer.CurrentBlockCount
|
||||
import fr.acinq.eclair.channel.{CMD_ADD_HTLC, PaymentFailed, PaymentSent, Register}
|
||||
import fr.acinq.eclair.crypto.Sphinx
|
||||
import fr.acinq.eclair.router._
|
||||
import fr.acinq.eclair.wire.{Codecs, PerHopPayload}
|
||||
import scodec.Attempt.{Successful, Failure}
|
||||
|
||||
import scala.util.Random
|
||||
import scodec.Attempt
|
||||
|
||||
// @formatter:off
|
||||
|
||||
@ -32,7 +31,7 @@ case object WAITING_FOR_PAYMENT_COMPLETE extends State
|
||||
/**
|
||||
* Created by PM on 26/08/2016.
|
||||
*/
|
||||
class PaymentLifecycle(router: ActorRef, currentBlockCount: Long) extends LoggingFSM[State, Data] {
|
||||
class PaymentLifecycle(sourceNodeId: BinaryData, router: ActorRef, currentBlockCount: Long) extends LoggingFSM[State, Data] {
|
||||
|
||||
import PaymentLifecycle._
|
||||
|
||||
@ -40,7 +39,7 @@ class PaymentLifecycle(router: ActorRef, currentBlockCount: Long) extends Loggin
|
||||
|
||||
when(WAITING_FOR_REQUEST) {
|
||||
case Event(c: CreatePayment, WaitingForRequest(currentBlockCount)) =>
|
||||
router ! RouteRequest(Globals.Node.publicKey, c.targetNodeId)
|
||||
router ! RouteRequest(sourceNodeId, c.targetNodeId)
|
||||
goto(WAITING_FOR_ROUTE) using WaitingForRoute(sender, c, currentBlockCount)
|
||||
}
|
||||
|
||||
@ -76,7 +75,7 @@ class PaymentLifecycle(router: ActorRef, currentBlockCount: Long) extends Loggin
|
||||
|
||||
object PaymentLifecycle {
|
||||
|
||||
def props(router: ActorRef, initialBlockCount: Long) = Props(classOf[PaymentLifecycle], router, initialBlockCount)
|
||||
def props(sourceNodeId: BinaryData, router: ActorRef, initialBlockCount: Long) = Props(classOf[PaymentLifecycle], sourceNodeId, router, initialBlockCount)
|
||||
|
||||
/**
|
||||
*
|
||||
@ -92,18 +91,13 @@ object PaymentLifecycle {
|
||||
|
||||
val pubkeys = nodes.map(PublicKey(_))
|
||||
|
||||
val sessionKey = PrivateKey({
|
||||
val bin = Array.fill[Byte](32)(0)
|
||||
// TODO: use secure random
|
||||
Random.nextBytes(bin)
|
||||
bin
|
||||
}, compressed = true)
|
||||
val sessionKey = randomKey
|
||||
|
||||
val payloadsbin: Seq[BinaryData] = payloads
|
||||
.map(Codecs.perHopPayloadCodec.encode(_))
|
||||
.map {
|
||||
case Successful(bitVector) => BinaryData(bitVector.toByteArray)
|
||||
case Failure(cause) => throw new RuntimeException(s"serialization error: $cause")
|
||||
case Attempt.Successful(bitVector) => BinaryData(bitVector.toByteArray)
|
||||
case Attempt.Failure(cause) => throw new RuntimeException(s"serialization error: $cause")
|
||||
}
|
||||
|
||||
Sphinx.makePacket(sessionKey, pubkeys, payloadsbin, BinaryData(""))
|
||||
|
@ -123,9 +123,13 @@ class Router(watcher: ActorRef, announcement: NodeAnnouncement) extends Actor wi
|
||||
rebroadcast.foreach(context.actorSelection(Register.actorPathToTransportHandlers) ! _)
|
||||
context become main(local, nodes, channels, updates, Nil)
|
||||
|
||||
case 'network => sender ! channels.values
|
||||
case 'nodes => sender ! nodes.values
|
||||
|
||||
case RouteRequest(start, end) => findRoute(start, end, updates) pipeTo sender
|
||||
case 'channels => sender ! channels.values
|
||||
|
||||
case 'updates => sender ! updates.values
|
||||
|
||||
case RouteRequest(start, end) => findRoute(start, end, updates).map(RouteResponse(_)) pipeTo sender
|
||||
|
||||
case other => log.warning(s"unhandled message $other")
|
||||
}
|
||||
|
@ -1,4 +1,4 @@
|
||||
package fr.acinq.eclair.channel.states
|
||||
package fr.acinq.eclair
|
||||
|
||||
import akka.actor.{ActorNotFound, ActorSystem, PoisonPill}
|
||||
import akka.testkit.TestKit
|
||||
@ -7,9 +7,10 @@ import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, fixture}
|
||||
import scala.concurrent.Await
|
||||
|
||||
/**
|
||||
* This base class kills all actor between each tests.
|
||||
* Created by PM on 06/09/2016.
|
||||
*/
|
||||
abstract class StateSpecBaseClass extends TestKit(ActorSystem("test")) with fixture.FunSuiteLike with BeforeAndAfterEach with BeforeAndAfterAll {
|
||||
abstract class TestkitBaseClass extends TestKit(ActorSystem("test")) with fixture.FunSuiteLike with BeforeAndAfterEach with BeforeAndAfterAll {
|
||||
|
||||
override def afterEach() {
|
||||
system.actorSelection(system / "*") ! PoisonPill
|
@ -4,10 +4,9 @@ import akka.actor.{ActorRef, Props}
|
||||
import akka.testkit.{TestFSMRef, TestProbe}
|
||||
import fr.acinq.eclair.TestConstants.{Alice, Bob}
|
||||
import fr.acinq.eclair.blockchain.PeerWatcher
|
||||
import fr.acinq.eclair.channel.states.StateSpecBaseClass
|
||||
import fr.acinq.eclair.channel.{WAIT_FOR_FUNDING_CREATED_INTERNAL, _}
|
||||
import fr.acinq.eclair.wire.{AcceptChannel, Error, OpenChannel}
|
||||
import fr.acinq.eclair.{TestBitcoinClient, TestConstants}
|
||||
import fr.acinq.eclair.{TestkitBaseClass, TestBitcoinClient, TestConstants}
|
||||
import org.junit.runner.RunWith
|
||||
import org.scalatest.junit.JUnitRunner
|
||||
|
||||
@ -17,7 +16,7 @@ import scala.concurrent.duration._
|
||||
* Created by PM on 05/07/2016.
|
||||
*/
|
||||
@RunWith(classOf[JUnitRunner])
|
||||
class WaitForAcceptChannelStateSpec extends StateSpecBaseClass {
|
||||
class WaitForAcceptChannelTestkit extends TestkitBaseClass {
|
||||
|
||||
type FixtureParam = Tuple5[TestFSMRef[State, Data, Channel], TestProbe, TestProbe, TestProbe, ActorRef]
|
||||
|
@ -1,10 +1,9 @@
|
||||
package fr.acinq.eclair.channel.states.a
|
||||
|
||||
import akka.testkit.{TestFSMRef, TestProbe}
|
||||
import fr.acinq.eclair.TestConstants
|
||||
import fr.acinq.eclair.{TestkitBaseClass, TestConstants}
|
||||
import fr.acinq.eclair.TestConstants.{Alice, Bob}
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.channel.states.StateSpecBaseClass
|
||||
import fr.acinq.eclair.wire.{Error, OpenChannel}
|
||||
import org.junit.runner.RunWith
|
||||
import org.scalatest.junit.JUnitRunner
|
||||
@ -15,7 +14,7 @@ import scala.concurrent.duration._
|
||||
* Created by PM on 05/07/2016.
|
||||
*/
|
||||
@RunWith(classOf[JUnitRunner])
|
||||
class WaitForOpenChannelStateSpec extends StateSpecBaseClass {
|
||||
class WaitForOpenChannelTestkit extends TestkitBaseClass {
|
||||
|
||||
type FixtureParam = Tuple4[TestFSMRef[State, Data, Channel], TestProbe, TestProbe, TestProbe]
|
||||
|
@ -5,9 +5,8 @@ import akka.testkit.{TestFSMRef, TestProbe}
|
||||
import fr.acinq.eclair.TestConstants.{Alice, Bob}
|
||||
import fr.acinq.eclair.blockchain.{MakeFundingTx, PeerWatcher}
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.channel.states.StateSpecBaseClass
|
||||
import fr.acinq.eclair.wire.{AcceptChannel, Error, FundingCreated, OpenChannel}
|
||||
import fr.acinq.eclair.{TestBitcoinClient, TestConstants}
|
||||
import fr.acinq.eclair.{TestkitBaseClass, TestBitcoinClient, TestConstants}
|
||||
import org.junit.runner.RunWith
|
||||
import org.scalatest.junit.JUnitRunner
|
||||
|
||||
@ -17,7 +16,7 @@ import scala.concurrent.duration._
|
||||
* Created by PM on 05/07/2016.
|
||||
*/
|
||||
@RunWith(classOf[JUnitRunner])
|
||||
class WaitForFundingCreatedInternalStateSpec extends StateSpecBaseClass {
|
||||
class WaitForFundingCreatedInternalTestkit extends TestkitBaseClass {
|
||||
|
||||
type FixtureParam = Tuple5[TestFSMRef[State, Data, Channel], TestProbe, TestProbe, TestProbe, ActorRef]
|
||||
|
@ -5,9 +5,8 @@ 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.channel.states.StateSpecBaseClass
|
||||
import fr.acinq.eclair.wire._
|
||||
import fr.acinq.eclair.{TestBitcoinClient, TestConstants}
|
||||
import fr.acinq.eclair.{TestkitBaseClass, TestBitcoinClient, TestConstants}
|
||||
import org.junit.runner.RunWith
|
||||
import org.scalatest.junit.JUnitRunner
|
||||
|
||||
@ -17,7 +16,7 @@ import scala.concurrent.duration._
|
||||
* Created by PM on 05/07/2016.
|
||||
*/
|
||||
@RunWith(classOf[JUnitRunner])
|
||||
class WaitForFundingCreatedStateSpec extends StateSpecBaseClass {
|
||||
class WaitForFundingCreatedTestkit extends TestkitBaseClass {
|
||||
|
||||
type FixtureParam = Tuple4[TestFSMRef[State, Data, Channel], TestProbe, TestProbe, TestProbe]
|
||||
|
@ -6,9 +6,8 @@ import fr.acinq.bitcoin.BinaryData
|
||||
import fr.acinq.eclair.TestConstants.{Alice, Bob}
|
||||
import fr.acinq.eclair.blockchain._
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.channel.states.StateSpecBaseClass
|
||||
import fr.acinq.eclair.wire.{AcceptChannel, Error, FundingCreated, FundingSigned, OpenChannel}
|
||||
import fr.acinq.eclair.{TestBitcoinClient, TestConstants}
|
||||
import fr.acinq.eclair.{TestkitBaseClass, TestBitcoinClient, TestConstants}
|
||||
import org.junit.runner.RunWith
|
||||
import org.scalatest.junit.JUnitRunner
|
||||
|
||||
@ -18,7 +17,7 @@ import scala.concurrent.duration._
|
||||
* Created by PM on 05/07/2016.
|
||||
*/
|
||||
@RunWith(classOf[JUnitRunner])
|
||||
class WaitForFundingSignedStateSpec extends StateSpecBaseClass {
|
||||
class WaitForFundingSignedTestkit extends TestkitBaseClass {
|
||||
|
||||
type FixtureParam = Tuple5[TestFSMRef[State, Data, Channel], TestProbe, TestProbe, TestProbe, ActorRef]
|
||||
|
@ -5,9 +5,8 @@ import akka.testkit.{TestFSMRef, TestProbe}
|
||||
import fr.acinq.eclair.TestConstants.{Alice, Bob}
|
||||
import fr.acinq.eclair.blockchain._
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.channel.states.StateSpecBaseClass
|
||||
import fr.acinq.eclair.wire.{AcceptChannel, Error, FundingCreated, FundingLocked, FundingSigned, OpenChannel}
|
||||
import fr.acinq.eclair.{TestBitcoinClient, TestConstants}
|
||||
import fr.acinq.eclair.{TestkitBaseClass, TestBitcoinClient, TestConstants}
|
||||
import org.junit.runner.RunWith
|
||||
import org.scalatest.junit.JUnitRunner
|
||||
|
||||
@ -17,7 +16,7 @@ import scala.concurrent.duration._
|
||||
* Created by PM on 05/07/2016.
|
||||
*/
|
||||
@RunWith(classOf[JUnitRunner])
|
||||
class WaitForFundingLockedInternalStateSpec extends StateSpecBaseClass {
|
||||
class WaitForFundingLockedInternalTestkit extends TestkitBaseClass {
|
||||
|
||||
type FixtureParam = Tuple6[TestFSMRef[State, Data, Channel], TestFSMRef[State, Data, Channel], TestProbe, TestProbe, TestProbe, ActorRef]
|
||||
|
@ -5,9 +5,8 @@ import akka.testkit.{TestFSMRef, TestProbe}
|
||||
import fr.acinq.eclair.TestConstants.{Alice, Bob}
|
||||
import fr.acinq.eclair.blockchain._
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.channel.states.StateSpecBaseClass
|
||||
import fr.acinq.eclair.wire._
|
||||
import fr.acinq.eclair.{TestBitcoinClient, TestConstants}
|
||||
import fr.acinq.eclair.{TestkitBaseClass, TestBitcoinClient, TestConstants}
|
||||
import org.junit.runner.RunWith
|
||||
import org.scalatest.junit.JUnitRunner
|
||||
|
||||
@ -17,7 +16,7 @@ import scala.concurrent.duration._
|
||||
* Created by PM on 05/07/2016.
|
||||
*/
|
||||
@RunWith(classOf[JUnitRunner])
|
||||
class WaitForFundingLockedStateSpec extends StateSpecBaseClass {
|
||||
class WaitForFundingLockedTestkit extends TestkitBaseClass {
|
||||
|
||||
type FixtureParam = Tuple6[TestFSMRef[State, Data, Channel], TestFSMRef[State, Data, Channel], TestProbe, TestProbe, TestProbe, ActorRef]
|
||||
|
@ -6,11 +6,11 @@ import fr.acinq.bitcoin.Crypto.{Point, Scalar}
|
||||
import fr.acinq.bitcoin.{BinaryData, Crypto, Satoshi, Script, ScriptFlags, Transaction}
|
||||
import fr.acinq.eclair.TestConstants.{Alice, Bob}
|
||||
import fr.acinq.eclair.blockchain._
|
||||
import fr.acinq.eclair.channel.states.{StateSpecBaseClass, StateTestsHelperMethods}
|
||||
import fr.acinq.eclair.channel.states.StateTestsHelperMethods
|
||||
import fr.acinq.eclair.channel.{Data, State, _}
|
||||
import fr.acinq.eclair.transactions.{IN, OUT}
|
||||
import fr.acinq.eclair.wire.{ClosingSigned, CommitSig, Error, RevokeAndAck, Shutdown, UpdateAddHtlc, UpdateFailHtlc, UpdateFulfillHtlc}
|
||||
import fr.acinq.eclair.{TestBitcoinClient, TestConstants}
|
||||
import fr.acinq.eclair.{TestkitBaseClass, TestBitcoinClient, TestConstants}
|
||||
import org.junit.runner.RunWith
|
||||
import org.scalatest.junit.JUnitRunner
|
||||
|
||||
@ -20,7 +20,7 @@ import scala.concurrent.duration._
|
||||
* Created by PM on 05/07/2016.
|
||||
*/
|
||||
@RunWith(classOf[JUnitRunner])
|
||||
class NormalStateSpec extends StateSpecBaseClass with StateTestsHelperMethods {
|
||||
class NormalTestkit extends TestkitBaseClass with StateTestsHelperMethods {
|
||||
|
||||
type FixtureParam = Tuple6[TestFSMRef[State, Data, Channel], TestFSMRef[State, Data, Channel], TestProbe, TestProbe, TestProbe, TestProbe]
|
||||
|
@ -4,10 +4,10 @@ import akka.actor.Props
|
||||
import akka.testkit.{TestFSMRef, TestProbe}
|
||||
import fr.acinq.bitcoin.Crypto.Scalar
|
||||
import fr.acinq.bitcoin.{BinaryData, Crypto, Satoshi, ScriptFlags, Transaction}
|
||||
import fr.acinq.eclair.TestBitcoinClient
|
||||
import fr.acinq.eclair.{TestkitBaseClass, TestBitcoinClient}
|
||||
import fr.acinq.eclair.TestConstants.{Alice, Bob}
|
||||
import fr.acinq.eclair.blockchain._
|
||||
import fr.acinq.eclair.channel.states.{StateSpecBaseClass, StateTestsHelperMethods}
|
||||
import fr.acinq.eclair.channel.states.StateTestsHelperMethods
|
||||
import fr.acinq.eclair.channel.{Data, State, _}
|
||||
import fr.acinq.eclair.wire.{CommitSig, Error, RevokeAndAck, Shutdown, UpdateAddHtlc, UpdateFailHtlc, UpdateFulfillHtlc}
|
||||
import org.junit.runner.RunWith
|
||||
@ -19,7 +19,7 @@ import scala.concurrent.duration._
|
||||
* Created by PM on 05/07/2016.
|
||||
*/
|
||||
@RunWith(classOf[JUnitRunner])
|
||||
class ShutdownStateSpec extends StateSpecBaseClass with StateTestsHelperMethods {
|
||||
class ShutdownTestkit extends TestkitBaseClass with StateTestsHelperMethods {
|
||||
|
||||
type FixtureParam = Tuple6[TestFSMRef[State, Data, Channel], TestFSMRef[State, Data, Channel], TestProbe, TestProbe, TestProbe, TestProbe]
|
||||
|
@ -2,10 +2,10 @@ package fr.acinq.eclair.channel.states.g
|
||||
|
||||
import akka.actor.Props
|
||||
import akka.testkit.{TestFSMRef, TestProbe}
|
||||
import fr.acinq.eclair.TestBitcoinClient
|
||||
import fr.acinq.eclair.{TestkitBaseClass, TestBitcoinClient}
|
||||
import fr.acinq.eclair.TestConstants.{Alice, Bob}
|
||||
import fr.acinq.eclair.blockchain._
|
||||
import fr.acinq.eclair.channel.states.{StateSpecBaseClass, StateTestsHelperMethods}
|
||||
import fr.acinq.eclair.channel.states.StateTestsHelperMethods
|
||||
import fr.acinq.eclair.channel.{Data, State, _}
|
||||
import fr.acinq.eclair.wire.{ClosingSigned, Error, Shutdown}
|
||||
import org.junit.runner.RunWith
|
||||
@ -17,7 +17,7 @@ import scala.concurrent.duration._
|
||||
* Created by PM on 05/07/2016.
|
||||
*/
|
||||
@RunWith(classOf[JUnitRunner])
|
||||
class NegotiatingStateSpec extends StateSpecBaseClass with StateTestsHelperMethods {
|
||||
class NegotiatingTestkit extends TestkitBaseClass with StateTestsHelperMethods {
|
||||
|
||||
type FixtureParam = Tuple6[TestFSMRef[State, Data, Channel], TestFSMRef[State, Data, Channel], TestProbe, TestProbe, TestProbe, TestProbe]
|
||||
|
@ -3,10 +3,10 @@ package fr.acinq.eclair.channel.states.h
|
||||
import akka.actor.Props
|
||||
import akka.testkit.{TestFSMRef, TestProbe}
|
||||
import fr.acinq.bitcoin.Transaction
|
||||
import fr.acinq.eclair.TestBitcoinClient
|
||||
import fr.acinq.eclair.{TestkitBaseClass, TestBitcoinClient}
|
||||
import fr.acinq.eclair.TestConstants.{Alice, Bob}
|
||||
import fr.acinq.eclair.blockchain._
|
||||
import fr.acinq.eclair.channel.states.{StateSpecBaseClass, StateTestsHelperMethods}
|
||||
import fr.acinq.eclair.channel.states.StateTestsHelperMethods
|
||||
import fr.acinq.eclair.channel.{Data, State, _}
|
||||
import fr.acinq.eclair.wire._
|
||||
import org.junit.runner.RunWith
|
||||
@ -18,7 +18,7 @@ import scala.concurrent.duration._
|
||||
* Created by PM on 05/07/2016.
|
||||
*/
|
||||
@RunWith(classOf[JUnitRunner])
|
||||
class ClosingStateSpec extends StateSpecBaseClass with StateTestsHelperMethods {
|
||||
class ClosingTestkit extends TestkitBaseClass with StateTestsHelperMethods {
|
||||
|
||||
type FixtureParam = Tuple7[TestFSMRef[State, Data, Channel], TestFSMRef[State, Data, Channel], TestProbe, TestProbe, TestProbe, TestProbe, List[Transaction]]
|
||||
|
@ -0,0 +1,153 @@
|
||||
package fr.acinq.eclair.payment
|
||||
|
||||
import fr.acinq.bitcoin.BinaryData
|
||||
import fr.acinq.eclair.payment.PaymentLifecycle._
|
||||
import fr.acinq.eclair.randomKey
|
||||
import fr.acinq.eclair.router.Hop
|
||||
import fr.acinq.eclair.wire.{ChannelUpdate, PerHopPayload}
|
||||
import org.junit.runner.RunWith
|
||||
import org.scalatest.FunSuite
|
||||
import org.scalatest.junit.JUnitRunner
|
||||
|
||||
/**
|
||||
* Created by PM on 31/05/2016.
|
||||
*/
|
||||
@RunWith(classOf[JUnitRunner])
|
||||
class HtlcGenerationSpec extends FunSuite {
|
||||
|
||||
test("compute fees") {
|
||||
val feeBaseMsat = 150000L
|
||||
val feeProportionalMillionth = 4L
|
||||
val htlcAmountMsat = 42000000
|
||||
// spec: fee-base-msat + htlc-amount-msat * fee-proportional-millionths / 1000000
|
||||
val ref = feeBaseMsat + htlcAmountMsat * feeProportionalMillionth / 1000000
|
||||
val fee = nodeFee(feeBaseMsat, feeProportionalMillionth, htlcAmountMsat)
|
||||
assert(ref === fee)
|
||||
}
|
||||
|
||||
test("compute simple route (no fees, no expiry delta)") {
|
||||
|
||||
val (a, b, c, d, e) = (BinaryData("aa" * 33), BinaryData("bb" * 33), BinaryData("cc" * 33), BinaryData("dd" * 33), BinaryData("ee" * 33))
|
||||
|
||||
val defaultChannelUpdate = ChannelUpdate("00" * 64, 0, 0, "0000", 0, 0, 0, 0)
|
||||
|
||||
// simple route b -> c -> d -> e
|
||||
|
||||
val hops =
|
||||
Hop(b, c, defaultChannelUpdate) ::
|
||||
Hop(c, d, defaultChannelUpdate) ::
|
||||
Hop(d, e, defaultChannelUpdate) :: Nil
|
||||
|
||||
val finalAmountMsat = 42000000L
|
||||
val currentBlockCount = 420000
|
||||
|
||||
val (firstAmountMsat, firstExpiry, payloads) = buildRoute(finalAmountMsat, hops, currentBlockCount)
|
||||
assert(firstAmountMsat === finalAmountMsat)
|
||||
assert(firstExpiry === currentBlockCount + defaultHtlcExpiry)
|
||||
assert(payloads ===
|
||||
PerHopPayload(finalAmountMsat, currentBlockCount + defaultHtlcExpiry) ::
|
||||
PerHopPayload(finalAmountMsat, currentBlockCount + defaultHtlcExpiry) ::
|
||||
PerHopPayload(finalAmountMsat, currentBlockCount + defaultHtlcExpiry) :: Nil)
|
||||
}
|
||||
|
||||
test("compute route with fees and expiry delta") {
|
||||
|
||||
val (a, b, c, d, e) = (BinaryData("aa" * 33), BinaryData("bb" * 33), BinaryData("cc" * 33), BinaryData("dd" * 33), BinaryData("ee" * 33))
|
||||
|
||||
val defaultChannelUpdate = ChannelUpdate("00" * 64, 0, 0, "0000", 0, 0, 0, 0)
|
||||
val channelUpdate_bc = defaultChannelUpdate.copy(cltvExpiryDelta = 5, feeBaseMsat = 153000, feeProportionalMillionths = 4)
|
||||
val channelUpdate_cd = defaultChannelUpdate.copy(cltvExpiryDelta = 10, feeBaseMsat = 60000, feeProportionalMillionths = 1)
|
||||
val channelUpdate_de = defaultChannelUpdate.copy(cltvExpiryDelta = 7, feeBaseMsat = 766000, feeProportionalMillionths = 10)
|
||||
|
||||
// simple route b -> c -> d -> e
|
||||
|
||||
val hops =
|
||||
Hop(b, c, channelUpdate_bc) ::
|
||||
Hop(c, d, channelUpdate_cd) ::
|
||||
Hop(d, e, channelUpdate_de) :: Nil
|
||||
|
||||
val finalAmountMsat = 42000000L
|
||||
val currentBlockCount = 420000
|
||||
|
||||
val expiry_de = currentBlockCount + defaultHtlcExpiry
|
||||
val amount_de = finalAmountMsat
|
||||
val fee_d = nodeFee(channelUpdate_de.feeBaseMsat, channelUpdate_de.feeProportionalMillionths, amount_de)
|
||||
|
||||
val expiry_cd = expiry_de + channelUpdate_de.cltvExpiryDelta
|
||||
val amount_cd = amount_de + fee_d
|
||||
val fee_c = nodeFee(channelUpdate_cd.feeBaseMsat, channelUpdate_cd.feeProportionalMillionths, amount_cd)
|
||||
|
||||
val expiry_bc = expiry_cd + channelUpdate_cd.cltvExpiryDelta
|
||||
val amount_bc = amount_cd + fee_c
|
||||
val fee_b = nodeFee(channelUpdate_bc.feeBaseMsat, channelUpdate_bc.feeProportionalMillionths, amount_bc)
|
||||
|
||||
val expiry_ab = expiry_bc + channelUpdate_bc.cltvExpiryDelta
|
||||
val amount_ab = amount_bc + fee_b
|
||||
|
||||
val (firstAmountMsat, firstExpiry, payloads) = buildRoute(finalAmountMsat, hops, currentBlockCount)
|
||||
assert(firstAmountMsat === amount_ab)
|
||||
assert(firstExpiry === expiry_ab)
|
||||
assert(payloads ===
|
||||
PerHopPayload(amount_bc, expiry_bc) ::
|
||||
PerHopPayload(amount_cd, expiry_cd) ::
|
||||
PerHopPayload(amount_de, expiry_de) :: Nil)
|
||||
}
|
||||
|
||||
test("build onion") {
|
||||
|
||||
def randomPubkey = randomKey.publicKey
|
||||
val (a, b, c, d, e) = (randomPubkey, randomPubkey, randomPubkey, randomPubkey, randomPubkey)
|
||||
|
||||
val defaultChannelUpdate = ChannelUpdate("00" * 64, 0, 0, "0000", 0, 0, 0, 0)
|
||||
val channelUpdate_bc = defaultChannelUpdate.copy(cltvExpiryDelta = 5, feeBaseMsat = 153000, feeProportionalMillionths = 4)
|
||||
val channelUpdate_cd = defaultChannelUpdate.copy(cltvExpiryDelta = 10, feeBaseMsat = 60000, feeProportionalMillionths = 1)
|
||||
val channelUpdate_de = defaultChannelUpdate.copy(cltvExpiryDelta = 7, feeBaseMsat = 766000, feeProportionalMillionths = 10)
|
||||
|
||||
// simple route b -> c -> d -> e
|
||||
|
||||
val hops =
|
||||
Hop(b, c, channelUpdate_bc) ::
|
||||
Hop(c, d, channelUpdate_cd) ::
|
||||
Hop(d, e, channelUpdate_de) :: Nil
|
||||
|
||||
val finalAmountMsat = 42000000L
|
||||
val currentBlockCount = 420000
|
||||
|
||||
val (_, _, payloads) = buildRoute(finalAmountMsat, hops, currentBlockCount)
|
||||
val nodes = hops.map(_.nextNodeId)
|
||||
val onion = buildOnion(nodes, payloads)
|
||||
|
||||
assert(onion.size === 1254)
|
||||
|
||||
}
|
||||
|
||||
test("build command including the onion") {
|
||||
|
||||
def randomPubkey = randomKey.publicKey
|
||||
val (a, b, c, d, e) = (randomPubkey, randomPubkey, randomPubkey, randomPubkey, randomPubkey)
|
||||
|
||||
val defaultChannelUpdate = ChannelUpdate("00" * 64, 0, 0, "0000", 0, 0, 0, 0)
|
||||
val channelUpdate_bc = defaultChannelUpdate.copy(cltvExpiryDelta = 5, feeBaseMsat = 153000, feeProportionalMillionths = 4)
|
||||
val channelUpdate_cd = defaultChannelUpdate.copy(cltvExpiryDelta = 10, feeBaseMsat = 60000, feeProportionalMillionths = 1)
|
||||
val channelUpdate_de = defaultChannelUpdate.copy(cltvExpiryDelta = 7, feeBaseMsat = 766000, feeProportionalMillionths = 10)
|
||||
|
||||
// simple route b -> c -> d -> e
|
||||
|
||||
val hops =
|
||||
Hop(b, c, channelUpdate_bc) ::
|
||||
Hop(c, d, channelUpdate_cd) ::
|
||||
Hop(d, e, channelUpdate_de) :: Nil
|
||||
|
||||
val finalAmountMsat = 42000000L
|
||||
val paymentHash = BinaryData("42" * 32)
|
||||
val currentBlockCount = 420000
|
||||
|
||||
val add = buildCommand(finalAmountMsat, paymentHash, hops, currentBlockCount)
|
||||
assert(add.amountMsat > finalAmountMsat)
|
||||
assert(add.expiry === currentBlockCount + defaultHtlcExpiry + channelUpdate_de.cltvExpiryDelta + channelUpdate_cd.cltvExpiryDelta + channelUpdate_bc.cltvExpiryDelta)
|
||||
assert(add.paymentHash === paymentHash)
|
||||
assert(add.onion.size === 1254)
|
||||
}
|
||||
|
||||
|
||||
}
|
@ -1,119 +0,0 @@
|
||||
package fr.acinq.eclair.payment
|
||||
|
||||
import akka.actor.ActorSystem
|
||||
import akka.testkit.TestKit
|
||||
import org.junit.runner.RunWith
|
||||
import org.scalatest.junit.JUnitRunner
|
||||
import org.scalatest.{BeforeAndAfterAll, FunSuiteLike}
|
||||
|
||||
/**
|
||||
* Created by PM on 29/08/2016.
|
||||
*/
|
||||
@RunWith(classOf[JUnitRunner])
|
||||
class PaymentFSMSpec extends TestKit(ActorSystem("test")) with FunSuiteLike with BeforeAndAfterAll {
|
||||
|
||||
override def afterAll {
|
||||
TestKit.shutdownActorSystem(system)
|
||||
}
|
||||
|
||||
// TODO: reenable
|
||||
/*test("route not available") {
|
||||
val router = system.actorOf(Props[Router])
|
||||
val selector = system.actorOf(Props[ChannelSelector])
|
||||
val channel00 = TestProbe()
|
||||
val channel01 = TestProbe()
|
||||
|
||||
// network: aaaa -> bbbbbbb -> cccc
|
||||
val node_a = Globals.Node.publicKey
|
||||
val node_b = PrivateKey(BinaryData("bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb"), compressed = true).publicKey
|
||||
val node_c = PrivateKey(BinaryData("cccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccc"), compressed = true).publicKey
|
||||
val node_d = PrivateKey(BinaryData("dddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddd"), compressed = true).publicKey
|
||||
|
||||
// no route b -> c
|
||||
router ! ChannelDiscovered(ChannelDesc("01", node_a, node_b))
|
||||
router ! ChannelDiscovered(ChannelDesc("02", node_c, node_d))
|
||||
|
||||
val paymentFsm = system.actorOf(PaymentLifecycle.props(router, selector, 1440))
|
||||
|
||||
val monitor = TestProbe()
|
||||
paymentFsm ! SubscribeTransitionCallBack(monitor.ref)
|
||||
val CurrentState(_, WAITING_FOR_REQUEST) = monitor.expectMsgClass(classOf[CurrentState[_]])
|
||||
|
||||
val sender = TestProbe()
|
||||
sender.send(paymentFsm, CreatePayment(42000000, BinaryData("00112233445566778899aabbccddeeff"), node_c))
|
||||
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
sender.expectMsgType[Status.Failure]
|
||||
}*/
|
||||
|
||||
//TODO re-enable
|
||||
/*test("payment succeeded") {
|
||||
val router = system.actorOf(Props[Router])
|
||||
val selector = system.actorOf(Props[ChannelSelector])
|
||||
val channel00 = TestProbe()
|
||||
val channel01 = TestProbe()
|
||||
|
||||
// network: aaaa -> bbbbbbb -> cccc
|
||||
val node_a = Globals.Node.publicKey
|
||||
val node_b = BinaryData("bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb")
|
||||
val node_c = BinaryData("ccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccc")
|
||||
|
||||
router ! ChannelDiscovered(ChannelDesc("01", node_a, node_b))
|
||||
router ! ChannelDiscovered(ChannelDesc("02", node_b, node_c))
|
||||
|
||||
selector ! ChannelChangedState(channel00.ref, node_b, OPEN_WAIT_FOR_COMPLETE_OURFUNDING, NORMAL, DATA_NORMAL_2(0, Commitments(null, null, null, TheirCommit(0L, CommitmentSpec(Set(), 0L, 0L, 100000), null, null), null, null, 0L, null, null, null, null, null), null, null))
|
||||
selector ! ChannelChangedState(channel01.ref, node_b, OPEN_WAIT_FOR_COMPLETE_OURFUNDING, NORMAL, DATA_NORMAL(Commitments(null, null, null, TheirCommit(0L, CommitmentSpec(Set(), 0L, 0L, 100000000), null, null), null, null, 0L, null, null, null, null, null), null, null))
|
||||
|
||||
val paymentFsm = system.actorOf(PaymentLifecycle.props(router, selector, 1440))
|
||||
|
||||
val monitor = TestProbe()
|
||||
paymentFsm ! SubscribeTransitionCallBack(monitor.ref)
|
||||
val CurrentState(_, WAITING_FOR_REQUEST) = monitor.expectMsgClass(classOf[CurrentState[_]])
|
||||
|
||||
val sender = TestProbe()
|
||||
val req = CreatePayment(42000000, BinaryData("00112233445566778899aabbccddeeff"), node_c)
|
||||
sender.send(paymentFsm, req)
|
||||
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
val Transition(_, WAITING_FOR_ROUTE, WAITING_FOR_CHANNEL) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
val Transition(_, WAITING_FOR_CHANNEL, WAITING_FOR_PAYMENT_COMPLETE) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
channel01.expectMsgType[CMD_ADD_HTLC]
|
||||
sender.send(paymentFsm, PaymentSent(channel01.ref, req.h))
|
||||
sender.expectMsg("sent")
|
||||
|
||||
}*/
|
||||
|
||||
//TODO re-enable
|
||||
/*test("payment failed") {
|
||||
val router = system.actorOf(Props[Router])
|
||||
val selector = system.actorOf(Props[ChannelSelector])
|
||||
val channel00 = TestProbe()
|
||||
val channel01 = TestProbe()
|
||||
|
||||
// network: aaaa -> bbbbbbb -> cccc
|
||||
val node_a = Globals.Node.publicKey
|
||||
val node_b = BinaryData("bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb")
|
||||
val node_c = BinaryData("ccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccc")
|
||||
|
||||
router ! ChannelDiscovered(ChannelDesc("01", node_a, node_b))
|
||||
router ! ChannelDiscovered(ChannelDesc("02", node_b, node_c))
|
||||
|
||||
selector ! ChannelChangedState(channel00.ref, node_b, OPEN_WAIT_FOR_COMPLETE_OURFUNDING, NORMAL, DATA_NORMAL(Commitments(null, null, null, TheirCommit(0L, CommitmentSpec(Set(), 0L, 0L, 100000), null, null), null, null, 0L, null, null, null, null, null), null, null))
|
||||
selector ! ChannelChangedState(channel01.ref, node_b, OPEN_WAIT_FOR_COMPLETE_OURFUNDING, NORMAL, DATA_NORMAL(Commitments(null, null, null, TheirCommit(0L, CommitmentSpec(Set(), 0L, 0L, 100000000), null, null), null, null, 0L, null, null, null, null, null), null, null))
|
||||
|
||||
val paymentFsm = system.actorOf(PaymentLifecycle.props(router, selector, 1440))
|
||||
|
||||
val monitor = TestProbe()
|
||||
paymentFsm ! SubscribeTransitionCallBack(monitor.ref)
|
||||
val CurrentState(_, WAITING_FOR_REQUEST) = monitor.expectMsgClass(classOf[CurrentState[_]])
|
||||
|
||||
val sender = TestProbe()
|
||||
val req = CreatePayment(42000000, BinaryData("00112233445566778899aabbccddeeff"), node_c)
|
||||
sender.send(paymentFsm, req)
|
||||
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
val Transition(_, WAITING_FOR_ROUTE, WAITING_FOR_CHANNEL) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
val Transition(_, WAITING_FOR_CHANNEL, WAITING_FOR_PAYMENT_COMPLETE) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
channel01.expectMsgType[CMD_ADD_HTLC]
|
||||
sender.send(paymentFsm, PaymentFailed(channel01.ref, req.h, "some reason"))
|
||||
sender.expectMsgType[Status.Failure]
|
||||
}*/
|
||||
|
||||
}
|
@ -1,160 +1,73 @@
|
||||
package fr.acinq.eclair.payment
|
||||
|
||||
import fr.acinq.bitcoin.BinaryData
|
||||
import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey}
|
||||
import fr.acinq.eclair.payment.PaymentLifecycle._
|
||||
import fr.acinq.eclair.router.Hop
|
||||
import fr.acinq.eclair.wire.{ChannelUpdate, PerHopPayload}
|
||||
import akka.actor.FSM.{CurrentState, SubscribeTransitionCallBack, Transition}
|
||||
import akka.actor.Status.Failure
|
||||
import akka.testkit.TestProbe
|
||||
import fr.acinq.eclair.channel.{PaymentFailed, PaymentSent}
|
||||
import fr.acinq.eclair.router.BaseRouterSpec
|
||||
import org.junit.runner.RunWith
|
||||
import org.scalatest.FunSuite
|
||||
import org.scalatest.junit.JUnitRunner
|
||||
|
||||
import scala.util.Random
|
||||
|
||||
/**
|
||||
* Created by PM on 31/05/2016.
|
||||
* Created by PM on 29/08/2016.
|
||||
*/
|
||||
@RunWith(classOf[JUnitRunner])
|
||||
class PaymentLifecycleSpec extends FunSuite {
|
||||
class PaymentLifecycleSpec extends BaseRouterSpec {
|
||||
|
||||
test("compute fees") {
|
||||
val feeBaseMsat = 150000L
|
||||
val feeProportionalMillionth = 4L
|
||||
val htlcAmountMsat = 42000000
|
||||
// spec: fee-base-msat + htlc-amount-msat * fee-proportional-millionths / 1000000
|
||||
val ref = feeBaseMsat + htlcAmountMsat * feeProportionalMillionth / 1000000
|
||||
val fee = nodeFee(feeBaseMsat, feeProportionalMillionth, htlcAmountMsat)
|
||||
assert(ref === fee)
|
||||
val initialBlockCount = 420000
|
||||
|
||||
test("payment failed (route not found)") { case (router, _) =>
|
||||
val paymentFSM = system.actorOf(PaymentLifecycle.props(a, router, initialBlockCount))
|
||||
val monitor = TestProbe()
|
||||
val sender = TestProbe()
|
||||
|
||||
paymentFSM ! SubscribeTransitionCallBack(monitor.ref)
|
||||
val CurrentState(_, WAITING_FOR_REQUEST) = monitor.expectMsgClass(classOf[CurrentState[_]])
|
||||
|
||||
val request = CreatePayment(142000, "42" * 32, f)
|
||||
sender.send(paymentFSM, request)
|
||||
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
|
||||
val res = sender.expectMsgType[Failure]
|
||||
assert(res.cause.getMessage === "route not found")
|
||||
}
|
||||
|
||||
test("compute simple route (no fees, no expiry delta)") {
|
||||
test("payment failed (htlc failed)") { case (router, _) =>
|
||||
val paymentFSM = system.actorOf(PaymentLifecycle.props(a, router, initialBlockCount))
|
||||
val monitor = TestProbe()
|
||||
val sender = TestProbe()
|
||||
|
||||
val (a, b, c, d, e) = (BinaryData("aa" * 33), BinaryData("bb" * 33), BinaryData("cc" * 33), BinaryData("dd" * 33), BinaryData("ee" * 33))
|
||||
paymentFSM ! SubscribeTransitionCallBack(monitor.ref)
|
||||
val CurrentState(_, WAITING_FOR_REQUEST) = monitor.expectMsgClass(classOf[CurrentState[_]])
|
||||
|
||||
val defaultChannelUpdate = ChannelUpdate("00" * 64, 0, 0, "0000", 0, 0, 0, 0)
|
||||
val request = CreatePayment(142000, "42" * 32, d)
|
||||
sender.send(paymentFSM, request)
|
||||
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
val Transition(_, WAITING_FOR_ROUTE, WAITING_FOR_PAYMENT_COMPLETE) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
|
||||
// simple route b -> c -> d -> e
|
||||
sender.send(paymentFSM, PaymentFailed(null, request.paymentHash, "some reason"))
|
||||
|
||||
val hops =
|
||||
Hop(b, c, defaultChannelUpdate) ::
|
||||
Hop(c, d, defaultChannelUpdate) ::
|
||||
Hop(d, e, defaultChannelUpdate) :: Nil
|
||||
|
||||
val finalAmountMsat = 42000000L
|
||||
val currentBlockCount = 420000
|
||||
|
||||
val (firstAmountMsat, firstExpiry, payloads) = buildRoute(finalAmountMsat, hops, currentBlockCount)
|
||||
assert(firstAmountMsat === finalAmountMsat)
|
||||
assert(firstExpiry === currentBlockCount + defaultHtlcExpiry)
|
||||
assert(payloads ===
|
||||
PerHopPayload(finalAmountMsat, currentBlockCount + defaultHtlcExpiry) ::
|
||||
PerHopPayload(finalAmountMsat, currentBlockCount + defaultHtlcExpiry) ::
|
||||
PerHopPayload(finalAmountMsat, currentBlockCount + defaultHtlcExpiry) :: Nil)
|
||||
val res = sender.expectMsgType[Failure]
|
||||
assert(res.cause.getMessage === "some reason")
|
||||
}
|
||||
|
||||
test("compute route with fees and expiry delta") {
|
||||
test("payment succeeded") { case (router, _) =>
|
||||
val paymentFSM = system.actorOf(PaymentLifecycle.props(a, router, initialBlockCount))
|
||||
val monitor = TestProbe()
|
||||
val sender = TestProbe()
|
||||
|
||||
val (a, b, c, d, e) = (BinaryData("aa" * 33), BinaryData("bb" * 33), BinaryData("cc" * 33), BinaryData("dd" * 33), BinaryData("ee" * 33))
|
||||
paymentFSM ! SubscribeTransitionCallBack(monitor.ref)
|
||||
val CurrentState(_, WAITING_FOR_REQUEST) = monitor.expectMsgClass(classOf[CurrentState[_]])
|
||||
|
||||
val defaultChannelUpdate = ChannelUpdate("00" * 64, 0, 0, "0000", 0, 0, 0, 0)
|
||||
val channelUpdate_bc = defaultChannelUpdate.copy(cltvExpiryDelta = 5, feeBaseMsat = 153000, feeProportionalMillionths = 4)
|
||||
val channelUpdate_cd = defaultChannelUpdate.copy(cltvExpiryDelta = 10, feeBaseMsat = 60000, feeProportionalMillionths = 1)
|
||||
val channelUpdate_de = defaultChannelUpdate.copy(cltvExpiryDelta = 7, feeBaseMsat = 766000, feeProportionalMillionths = 10)
|
||||
val request = CreatePayment(142000, "42" * 32, d)
|
||||
sender.send(paymentFSM, request)
|
||||
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
val Transition(_, WAITING_FOR_ROUTE, WAITING_FOR_PAYMENT_COMPLETE) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
|
||||
// simple route b -> c -> d -> e
|
||||
sender.send(paymentFSM, PaymentSent(null, request.paymentHash))
|
||||
|
||||
val hops =
|
||||
Hop(b, c, channelUpdate_bc) ::
|
||||
Hop(c, d, channelUpdate_cd) ::
|
||||
Hop(d, e, channelUpdate_de) :: Nil
|
||||
|
||||
val finalAmountMsat = 42000000L
|
||||
val currentBlockCount = 420000
|
||||
|
||||
val expiry_de = currentBlockCount + defaultHtlcExpiry
|
||||
val amount_de = finalAmountMsat
|
||||
val fee_d = nodeFee(channelUpdate_de.feeBaseMsat, channelUpdate_de.feeProportionalMillionths, amount_de)
|
||||
|
||||
val expiry_cd = expiry_de + channelUpdate_de.cltvExpiryDelta
|
||||
val amount_cd = amount_de + fee_d
|
||||
val fee_c = nodeFee(channelUpdate_cd.feeBaseMsat, channelUpdate_cd.feeProportionalMillionths, amount_cd)
|
||||
|
||||
val expiry_bc = expiry_cd + channelUpdate_cd.cltvExpiryDelta
|
||||
val amount_bc = amount_cd + fee_c
|
||||
val fee_b = nodeFee(channelUpdate_bc.feeBaseMsat, channelUpdate_bc.feeProportionalMillionths, amount_bc)
|
||||
|
||||
val expiry_ab = expiry_bc + channelUpdate_bc.cltvExpiryDelta
|
||||
val amount_ab = amount_bc + fee_b
|
||||
|
||||
val (firstAmountMsat, firstExpiry, payloads) = buildRoute(finalAmountMsat, hops, currentBlockCount)
|
||||
assert(firstAmountMsat === amount_ab)
|
||||
assert(firstExpiry === expiry_ab)
|
||||
assert(payloads ===
|
||||
PerHopPayload(amount_bc, expiry_bc) ::
|
||||
PerHopPayload(amount_cd, expiry_cd) ::
|
||||
PerHopPayload(amount_de, expiry_de) :: Nil)
|
||||
val res = sender.expectMsgType[String]
|
||||
assert(res === "sent")
|
||||
}
|
||||
|
||||
def randomPubkey: PublicKey = PrivateKey({
|
||||
val bin = Array.fill[Byte](32)(0)
|
||||
// TODO: use secure random
|
||||
Random.nextBytes(bin)
|
||||
bin
|
||||
}, compressed = true).publicKey
|
||||
|
||||
test("build onion") {
|
||||
|
||||
val (a, b, c, d, e) = (randomPubkey, randomPubkey, randomPubkey, randomPubkey, randomPubkey)
|
||||
|
||||
val defaultChannelUpdate = ChannelUpdate("00" * 64, 0, 0, "0000", 0, 0, 0, 0)
|
||||
val channelUpdate_bc = defaultChannelUpdate.copy(cltvExpiryDelta = 5, feeBaseMsat = 153000, feeProportionalMillionths = 4)
|
||||
val channelUpdate_cd = defaultChannelUpdate.copy(cltvExpiryDelta = 10, feeBaseMsat = 60000, feeProportionalMillionths = 1)
|
||||
val channelUpdate_de = defaultChannelUpdate.copy(cltvExpiryDelta = 7, feeBaseMsat = 766000, feeProportionalMillionths = 10)
|
||||
|
||||
// simple route b -> c -> d -> e
|
||||
|
||||
val hops =
|
||||
Hop(b, c, channelUpdate_bc) ::
|
||||
Hop(c, d, channelUpdate_cd) ::
|
||||
Hop(d, e, channelUpdate_de) :: Nil
|
||||
|
||||
val finalAmountMsat = 42000000L
|
||||
val currentBlockCount = 420000
|
||||
|
||||
val (firstAmountMsat, firstExpiry, payloads) = buildRoute(finalAmountMsat, hops, currentBlockCount)
|
||||
val nodes = hops.map(_.nextNodeId)
|
||||
val onion = buildOnion(nodes, payloads)
|
||||
|
||||
assert(onion.size === 1254)
|
||||
|
||||
}
|
||||
|
||||
test("build command including the onion") {
|
||||
|
||||
val (a, b, c, d, e) = (randomPubkey, randomPubkey, randomPubkey, randomPubkey, randomPubkey)
|
||||
|
||||
val defaultChannelUpdate = ChannelUpdate("00" * 64, 0, 0, "0000", 0, 0, 0, 0)
|
||||
val channelUpdate_bc = defaultChannelUpdate.copy(cltvExpiryDelta = 5, feeBaseMsat = 153000, feeProportionalMillionths = 4)
|
||||
val channelUpdate_cd = defaultChannelUpdate.copy(cltvExpiryDelta = 10, feeBaseMsat = 60000, feeProportionalMillionths = 1)
|
||||
val channelUpdate_de = defaultChannelUpdate.copy(cltvExpiryDelta = 7, feeBaseMsat = 766000, feeProportionalMillionths = 10)
|
||||
|
||||
// simple route b -> c -> d -> e
|
||||
|
||||
val hops =
|
||||
Hop(b, c, channelUpdate_bc) ::
|
||||
Hop(c, d, channelUpdate_cd) ::
|
||||
Hop(d, e, channelUpdate_de) :: Nil
|
||||
|
||||
val finalAmountMsat = 42000000L
|
||||
val paymentHash = BinaryData("42" * 32)
|
||||
val currentBlockCount = 420000
|
||||
|
||||
val add = buildCommand(finalAmountMsat, paymentHash, hops, currentBlockCount)
|
||||
assert(add.amountMsat > finalAmountMsat)
|
||||
assert(add.expiry === currentBlockCount + defaultHtlcExpiry + channelUpdate_de.cltvExpiryDelta + channelUpdate_cd.cltvExpiryDelta + channelUpdate_bc.cltvExpiryDelta)
|
||||
assert(add.paymentHash === paymentHash)
|
||||
assert(add.onion.size === 1254)
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -0,0 +1,89 @@
|
||||
package fr.acinq.eclair.router
|
||||
|
||||
import akka.actor.ActorRef
|
||||
import akka.testkit.TestProbe
|
||||
import fr.acinq.eclair.router.Router.DUMMY_SIG
|
||||
import fr.acinq.eclair.wire._
|
||||
import fr.acinq.eclair.{TestkitBaseClass, randomKey}
|
||||
import org.junit.runner.RunWith
|
||||
import org.scalatest.junit.JUnitRunner
|
||||
|
||||
import scala.concurrent.duration._
|
||||
|
||||
/**
|
||||
* Base class for router testing.
|
||||
* It is re-used in payment FSM tests
|
||||
* Created by PM on 29/08/2016.
|
||||
*/
|
||||
@RunWith(classOf[JUnitRunner])
|
||||
abstract class BaseRouterSpec extends TestkitBaseClass {
|
||||
|
||||
type FixtureParam = Tuple2[ActorRef, TestProbe]
|
||||
|
||||
def randomPubkey = randomKey.publicKey
|
||||
|
||||
val (a, b, c, d, e, f) = (randomPubkey, randomPubkey, randomPubkey, randomPubkey, randomPubkey, randomPubkey)
|
||||
|
||||
val ann_a = NodeAnnouncement(DUMMY_SIG, 0, a, (0, 0, 0), "node-A", "0000", Nil)
|
||||
val ann_b = NodeAnnouncement(DUMMY_SIG, 0, b, (0, 0, 0), "node-B", "0000", Nil)
|
||||
val ann_c = NodeAnnouncement(DUMMY_SIG, 0, c, (0, 0, 0), "node-C", "0000", Nil)
|
||||
val ann_d = NodeAnnouncement(DUMMY_SIG, 0, d, (0, 0, 0), "node-D", "0000", Nil)
|
||||
val ann_e = NodeAnnouncement(DUMMY_SIG, 0, e, (0, 0, 0), "node-E", "0000", Nil)
|
||||
val ann_f = NodeAnnouncement(DUMMY_SIG, 0, f, (0, 0, 0), "node-F", "0000", Nil)
|
||||
|
||||
val chan_ab = ChannelAnnouncement(DUMMY_SIG, DUMMY_SIG, channelId = 1, DUMMY_SIG, DUMMY_SIG, a, b, "", "")
|
||||
val chan_bc = ChannelAnnouncement(DUMMY_SIG, DUMMY_SIG, channelId = 2, DUMMY_SIG, DUMMY_SIG, b, c, "", "")
|
||||
val chan_cd = ChannelAnnouncement(DUMMY_SIG, DUMMY_SIG, channelId = 3, DUMMY_SIG, DUMMY_SIG, c, d, "", "")
|
||||
val chan_ef = ChannelAnnouncement(DUMMY_SIG, DUMMY_SIG, channelId = 4, DUMMY_SIG, DUMMY_SIG, e, f, "", "")
|
||||
|
||||
val defaultChannelUpdate = ChannelUpdate(Router.DUMMY_SIG, 0, 0, "0000", 0, 0, 0, 0)
|
||||
val channelUpdate_ab = ChannelUpdate(Router.DUMMY_SIG, channelId = 1, 0, "0000", cltvExpiryDelta = 7, 0, feeBaseMsat = 766000, feeProportionalMillionths = 10)
|
||||
val channelUpdate_bc = ChannelUpdate(Router.DUMMY_SIG, channelId = 2, 0, "0000", cltvExpiryDelta = 5, 0, feeBaseMsat = 233000, feeProportionalMillionths = 1)
|
||||
val channelUpdate_cd = ChannelUpdate(Router.DUMMY_SIG, channelId = 3, 0, "0000", cltvExpiryDelta = 3, 0, feeBaseMsat = 153000, feeProportionalMillionths = 4)
|
||||
val channelUpdate_ef = ChannelUpdate(Router.DUMMY_SIG, channelId = 4, 0, "0000", cltvExpiryDelta = 9, 0, feeBaseMsat = 786000, feeProportionalMillionths = 8)
|
||||
|
||||
|
||||
override def withFixture(test: OneArgTest) = {
|
||||
// the network will be a --(1)--> b ---(2)--> c --(3)--> d and e --(4)--> f (we are a)
|
||||
|
||||
within(30 seconds) {
|
||||
// first we set up the router
|
||||
val watcher = TestProbe()
|
||||
val router = system.actorOf(Router.props(watcher.ref, ann_a))
|
||||
// we announce channels
|
||||
router ! chan_ab
|
||||
router ! chan_bc
|
||||
router ! chan_cd
|
||||
router ! chan_ef
|
||||
// then nodes
|
||||
router ! ann_a
|
||||
router ! ann_b
|
||||
router ! ann_c
|
||||
router ! ann_d
|
||||
router ! ann_e
|
||||
router ! ann_f
|
||||
// then channel updates
|
||||
router ! channelUpdate_ab
|
||||
router ! channelUpdate_bc
|
||||
router ! channelUpdate_cd
|
||||
router ! channelUpdate_ef
|
||||
|
||||
val sender = TestProbe()
|
||||
|
||||
sender.send(router, 'nodes)
|
||||
val nodes = sender.expectMsgType[Iterable[NodeAnnouncement]]
|
||||
assert(nodes.size === 6)
|
||||
|
||||
sender.send(router, 'channels)
|
||||
val channels = sender.expectMsgType[Iterable[ChannelAnnouncement]]
|
||||
assert(channels.size === 4)
|
||||
|
||||
sender.send(router, 'updates)
|
||||
val updates = sender.expectMsgType[Iterable[ChannelUpdate]]
|
||||
assert(updates.size === 4)
|
||||
|
||||
test((router, watcher))
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,127 @@
|
||||
package fr.acinq.eclair.router
|
||||
|
||||
import fr.acinq.bitcoin.Crypto.PrivateKey
|
||||
import fr.acinq.bitcoin.{BinaryData, Crypto}
|
||||
import fr.acinq.eclair.wire.ChannelUpdate
|
||||
import org.junit.runner.RunWith
|
||||
import org.scalatest.FunSuite
|
||||
import org.scalatest.junit.JUnitRunner
|
||||
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
|
||||
/**
|
||||
* Created by PM on 31/05/2016.
|
||||
*/
|
||||
@RunWith(classOf[JUnitRunner])
|
||||
class RouteCalculationSpec extends FunSuite {
|
||||
|
||||
val (a, b, c, d, e) = (BinaryData("aa" * 33), BinaryData("bb" * 33), BinaryData("cc" * 33), BinaryData("dd" * 33), BinaryData("ee" * 33))
|
||||
|
||||
test("calculate simple route") {
|
||||
|
||||
val channels = List(
|
||||
ChannelDesc(1L, a, b),
|
||||
ChannelDesc(2L, b, c),
|
||||
ChannelDesc(3L, c, d),
|
||||
ChannelDesc(4L, d, e)
|
||||
)
|
||||
|
||||
val route = Router.findRouteDijkstra(a, e, channels)
|
||||
assert(route.map(_.id) === 1 :: 2 :: 3 :: 4 :: Nil)
|
||||
|
||||
}
|
||||
|
||||
test("route not found") {
|
||||
|
||||
val channels = List(
|
||||
ChannelDesc(1L, a, b),
|
||||
ChannelDesc(2L, b, c),
|
||||
ChannelDesc(4L, d, e)
|
||||
)
|
||||
|
||||
intercept[RuntimeException] {
|
||||
Router.findRouteDijkstra(a, e, channels)
|
||||
}
|
||||
}
|
||||
|
||||
test("route to self") {
|
||||
|
||||
val channels = List(
|
||||
ChannelDesc(1L, a, b),
|
||||
ChannelDesc(2L, b, c),
|
||||
ChannelDesc(3L, c, d),
|
||||
ChannelDesc(4L, d, e)
|
||||
)
|
||||
|
||||
intercept[RuntimeException] {
|
||||
Router.findRouteDijkstra(a, a, channels)
|
||||
}
|
||||
}
|
||||
|
||||
test("route to immediate neighbor") {
|
||||
val channels = List(
|
||||
ChannelDesc(1L, a, b),
|
||||
ChannelDesc(2L, b, c),
|
||||
ChannelDesc(3L, c, d),
|
||||
ChannelDesc(4L, d, e)
|
||||
)
|
||||
|
||||
val route = Router.findRouteDijkstra(a, b, channels)
|
||||
assert(route.map(_.id) === 1 :: Nil)
|
||||
}
|
||||
|
||||
test("directed graph") {
|
||||
val channels = List(
|
||||
ChannelDesc(1L, a, b),
|
||||
ChannelDesc(2L, b, c),
|
||||
ChannelDesc(3L, c, d),
|
||||
ChannelDesc(4L, d, e)
|
||||
)
|
||||
|
||||
// a->e works, e->a fails
|
||||
|
||||
Router.findRouteDijkstra(a, e, channels)
|
||||
|
||||
intercept[RuntimeException] {
|
||||
Router.findRouteDijkstra(e, a, channels)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
test("compute an example sig") {
|
||||
val data = BinaryData("00" * 32)
|
||||
val key = PrivateKey(BinaryData("11" * 32))
|
||||
val sig = Crypto.encodeSignature(Crypto.sign(data, key))
|
||||
assert(Crypto.isDERSignature(sig :+ 1.toByte))
|
||||
}
|
||||
|
||||
test("calculate route and return metadata") {
|
||||
|
||||
val uab = ChannelUpdate(Router.DUMMY_SIG, 1L, 0L, "0000", 1, 42, 2500, 140)
|
||||
val uba = ChannelUpdate(Router.DUMMY_SIG, 1L, 1L, "0001", 1, 43, 2501, 141)
|
||||
val ubc = ChannelUpdate(Router.DUMMY_SIG, 2L, 1L, "0000", 1, 44, 2502, 142)
|
||||
val ucb = ChannelUpdate(Router.DUMMY_SIG, 2L, 1L, "0001", 1, 45, 2503, 143)
|
||||
val ucd = ChannelUpdate(Router.DUMMY_SIG, 3L, 1L, "0000", 1, 46, 2504, 144)
|
||||
val udc = ChannelUpdate(Router.DUMMY_SIG, 3L, 1L, "0001", 1, 47, 2505, 145)
|
||||
val ude = ChannelUpdate(Router.DUMMY_SIG, 4L, 1L, "0000", 1, 48, 2506, 146)
|
||||
val ued = ChannelUpdate(Router.DUMMY_SIG, 4L, 1L, "0001", 1, 49, 2507, 147)
|
||||
|
||||
val updates = Map(
|
||||
ChannelDesc(1L, a, b) -> uab,
|
||||
ChannelDesc(1L, b, a) -> uba,
|
||||
ChannelDesc(2L, b, c) -> ubc,
|
||||
ChannelDesc(2L, c, b) -> ucb,
|
||||
ChannelDesc(3L, c, d) -> ucd,
|
||||
ChannelDesc(3L, d, c) -> udc,
|
||||
ChannelDesc(4L, d, e) -> ude,
|
||||
ChannelDesc(4L, e, d) -> ued
|
||||
)
|
||||
|
||||
import scala.concurrent.ExecutionContext.Implicits.global
|
||||
val hops = Await.result(Router.findRoute(a, e, updates), 3 seconds)
|
||||
|
||||
assert(hops === Hop(a, b, uab) :: Hop(b, c, ubc) :: Hop(c, d, ucd) :: Hop(d, e, ude) :: Nil)
|
||||
}
|
||||
|
||||
}
|
@ -1,127 +1,141 @@
|
||||
package fr.acinq.eclair.router
|
||||
|
||||
import fr.acinq.bitcoin.Crypto.PrivateKey
|
||||
import fr.acinq.bitcoin.{BinaryData, Crypto}
|
||||
import fr.acinq.eclair.wire.ChannelUpdate
|
||||
import akka.actor.Status.Failure
|
||||
import akka.testkit.TestProbe
|
||||
import org.junit.runner.RunWith
|
||||
import org.scalatest.FunSuite
|
||||
import org.scalatest.junit.JUnitRunner
|
||||
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
|
||||
/**
|
||||
* Created by PM on 31/05/2016.
|
||||
* Created by PM on 29/08/2016.
|
||||
*/
|
||||
@RunWith(classOf[JUnitRunner])
|
||||
class RouterSpec extends FunSuite {
|
||||
|
||||
val (a, b, c, d, e) = (BinaryData("aa" * 33), BinaryData("bb" * 33), BinaryData("cc" * 33), BinaryData("dd" * 33), BinaryData("ee" * 33))
|
||||
|
||||
test("calculate simple route") {
|
||||
|
||||
val channels = List(
|
||||
ChannelDesc(1L, a, b),
|
||||
ChannelDesc(2L, b, c),
|
||||
ChannelDesc(3L, c, d),
|
||||
ChannelDesc(4L, d, e)
|
||||
)
|
||||
|
||||
val route = Router.findRouteDijkstra(a, e, channels)
|
||||
assert(route.map(_.id) === 1 :: 2 :: 3 :: 4 :: Nil)
|
||||
class RouterSpec extends BaseRouterSpec {
|
||||
|
||||
test("route not found (unreachable target)") { case (router, watcher) =>
|
||||
val sender = TestProbe()
|
||||
// no route a->f
|
||||
sender.send(router, RouteRequest(a, f))
|
||||
val res = sender.expectMsgType[Failure]
|
||||
assert(res.cause.getMessage === "route not found")
|
||||
}
|
||||
|
||||
test("route not found") {
|
||||
|
||||
val channels = List(
|
||||
ChannelDesc(1L, a, b),
|
||||
ChannelDesc(2L, b, c),
|
||||
ChannelDesc(4L, d, e)
|
||||
)
|
||||
|
||||
intercept[RuntimeException] {
|
||||
Router.findRouteDijkstra(a, e, channels)
|
||||
}
|
||||
test("route not found (non-existing source)") { case (router, watcher) =>
|
||||
val sender = TestProbe()
|
||||
// no route a->f
|
||||
sender.send(router, RouteRequest(randomPubkey, f))
|
||||
val res = sender.expectMsgType[Failure]
|
||||
assert(res.cause.getMessage === "graph must contain the source vertex")
|
||||
}
|
||||
|
||||
test("route to self") {
|
||||
|
||||
val channels = List(
|
||||
ChannelDesc(1L, a, b),
|
||||
ChannelDesc(2L, b, c),
|
||||
ChannelDesc(3L, c, d),
|
||||
ChannelDesc(4L, d, e)
|
||||
)
|
||||
|
||||
intercept[RuntimeException] {
|
||||
Router.findRouteDijkstra(a, a, channels)
|
||||
}
|
||||
test("route not found (non-existing target)") { case (router, watcher) =>
|
||||
val sender = TestProbe()
|
||||
// no route a->f
|
||||
sender.send(router, RouteRequest(a, randomPubkey))
|
||||
val res = sender.expectMsgType[Failure]
|
||||
assert(res.cause.getMessage === "graph must contain the sink vertex")
|
||||
}
|
||||
|
||||
test("route to immediate neighbor") {
|
||||
val channels = List(
|
||||
ChannelDesc(1L, a, b),
|
||||
ChannelDesc(2L, b, c),
|
||||
ChannelDesc(3L, c, d),
|
||||
ChannelDesc(4L, d, e)
|
||||
)
|
||||
|
||||
val route = Router.findRouteDijkstra(a, b, channels)
|
||||
assert(route.map(_.id) === 1 :: Nil)
|
||||
test("route found") { case (router, watcher) =>
|
||||
val sender = TestProbe()
|
||||
sender.send(router, RouteRequest(a, d))
|
||||
val res = sender.expectMsgType[RouteResponse]
|
||||
assert(res.hops.map(_.nodeId).toList === a.toBin :: b.toBin :: c.toBin :: Nil)
|
||||
assert(res.hops.last.nextNodeId === d.toBin)
|
||||
}
|
||||
|
||||
test("directed graph") {
|
||||
val channels = List(
|
||||
ChannelDesc(1L, a, b),
|
||||
ChannelDesc(2L, b, c),
|
||||
ChannelDesc(3L, c, d),
|
||||
ChannelDesc(4L, d, e)
|
||||
)
|
||||
/*val finalAmountMsat = 42000000L
|
||||
val currentBlockCount = 420000
|
||||
|
||||
// a->e works, e->a fails
|
||||
val node_a = Globals.Node.publicKey
|
||||
val node_b = PrivateKey(BinaryData("bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb"), compressed = true).publicKey
|
||||
val node_c = PrivateKey(BinaryData("cccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccc"), compressed = true).publicKey
|
||||
val node_d = PrivateKey(BinaryData("dddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddd"), compressed = true).publicKey
|
||||
|
||||
Router.findRouteDijkstra(a, e, channels)
|
||||
// no route b -> c
|
||||
router ! ChannelDiscovered(ChannelDesc("01", node_a, node_b))
|
||||
router ! ChannelDiscovered(ChannelDesc("02", node_c, node_d))
|
||||
|
||||
intercept[RuntimeException] {
|
||||
Router.findRouteDijkstra(e, a, channels)
|
||||
}
|
||||
val paymentFsm = system.actorOf(PaymentLifecycle.props(router, selector, 1440))
|
||||
|
||||
}
|
||||
val monitor = TestProbe()
|
||||
paymentFsm ! SubscribeTransitionCallBack(monitor.ref)
|
||||
val CurrentState(_, WAITING_FOR_REQUEST) = monitor.expectMsgClass(classOf[CurrentState[_]])
|
||||
|
||||
test("compute an example sig") {
|
||||
val data = BinaryData("00" * 32)
|
||||
val key = PrivateKey(BinaryData("11" * 32))
|
||||
val sig = Crypto.encodeSignature(Crypto.sign(data, key))
|
||||
assert(Crypto.isDERSignature(sig :+ 1.toByte))
|
||||
}
|
||||
val sender = TestProbe()
|
||||
sender.send(paymentFsm, CreatePayment(42000000, BinaryData("00112233445566778899aabbccddeeff"), node_c))
|
||||
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
sender.expectMsgType[Status.Failure]*/
|
||||
|
||||
test("calculate route and return metadata") {
|
||||
|
||||
val uab = ChannelUpdate(Router.DUMMY_SIG, 1L, 0L, "0000", 1, 42, 2500, 140)
|
||||
val uba = ChannelUpdate(Router.DUMMY_SIG, 1L, 1L, "0001", 1, 43, 2501, 141)
|
||||
val ubc = ChannelUpdate(Router.DUMMY_SIG, 2L, 1L, "0000", 1, 44, 2502, 142)
|
||||
val ucb = ChannelUpdate(Router.DUMMY_SIG, 2L, 1L, "0001", 1, 45, 2503, 143)
|
||||
val ucd = ChannelUpdate(Router.DUMMY_SIG, 3L, 1L, "0000", 1, 46, 2504, 144)
|
||||
val udc = ChannelUpdate(Router.DUMMY_SIG, 3L, 1L, "0001", 1, 47, 2505, 145)
|
||||
val ude = ChannelUpdate(Router.DUMMY_SIG, 4L, 1L, "0000", 1, 48, 2506, 146)
|
||||
val ued = ChannelUpdate(Router.DUMMY_SIG, 4L, 1L, "0001", 1, 49, 2507, 147)
|
||||
//TODO re-enable
|
||||
/*test("payment succeeded") {
|
||||
val router = system.actorOf(Props[Router])
|
||||
val selector = system.actorOf(Props[ChannelSelector])
|
||||
val channel00 = TestProbe()
|
||||
val channel01 = TestProbe()
|
||||
|
||||
val updates = Map(
|
||||
ChannelDesc(1L, a, b) -> uab,
|
||||
ChannelDesc(1L, b, a) -> uba,
|
||||
ChannelDesc(2L, b, c) -> ubc,
|
||||
ChannelDesc(2L, c, b) -> ucb,
|
||||
ChannelDesc(3L, c, d) -> ucd,
|
||||
ChannelDesc(3L, d, c) -> udc,
|
||||
ChannelDesc(4L, d, e) -> ude,
|
||||
ChannelDesc(4L, e, d) -> ued
|
||||
)
|
||||
// network: aaaa -> bbbbbbb -> cccc
|
||||
val node_a = Globals.Node.publicKey
|
||||
val node_b = BinaryData("bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb")
|
||||
val node_c = BinaryData("ccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccc")
|
||||
|
||||
import scala.concurrent.ExecutionContext.Implicits.global
|
||||
val hops = Await.result(Router.findRoute(a, e, updates), 3 seconds)
|
||||
router ! ChannelDiscovered(ChannelDesc("01", node_a, node_b))
|
||||
router ! ChannelDiscovered(ChannelDesc("02", node_b, node_c))
|
||||
|
||||
assert(hops === Hop(a, b, uab) :: Hop(b, c, ubc) :: Hop(c, d, ucd) :: Hop(d, e, ude) :: Nil)
|
||||
}
|
||||
selector ! ChannelChangedState(channel00.ref, node_b, OPEN_WAIT_FOR_COMPLETE_OURFUNDING, NORMAL, DATA_NORMAL_2(0, Commitments(null, null, null, TheirCommit(0L, CommitmentSpec(Set(), 0L, 0L, 100000), null, null), null, null, 0L, null, null, null, null, null), null, null))
|
||||
selector ! ChannelChangedState(channel01.ref, node_b, OPEN_WAIT_FOR_COMPLETE_OURFUNDING, NORMAL, DATA_NORMAL(Commitments(null, null, null, TheirCommit(0L, CommitmentSpec(Set(), 0L, 0L, 100000000), null, null), null, null, 0L, null, null, null, null, null), null, null))
|
||||
|
||||
val paymentFsm = system.actorOf(PaymentLifecycle.props(router, selector, 1440))
|
||||
|
||||
val monitor = TestProbe()
|
||||
paymentFsm ! SubscribeTransitionCallBack(monitor.ref)
|
||||
val CurrentState(_, WAITING_FOR_REQUEST) = monitor.expectMsgClass(classOf[CurrentState[_]])
|
||||
|
||||
val sender = TestProbe()
|
||||
val req = CreatePayment(42000000, BinaryData("00112233445566778899aabbccddeeff"), node_c)
|
||||
sender.send(paymentFsm, req)
|
||||
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
val Transition(_, WAITING_FOR_ROUTE, WAITING_FOR_CHANNEL) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
val Transition(_, WAITING_FOR_CHANNEL, WAITING_FOR_PAYMENT_COMPLETE) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
channel01.expectMsgType[CMD_ADD_HTLC]
|
||||
sender.send(paymentFsm, PaymentSent(channel01.ref, req.h))
|
||||
sender.expectMsg("sent")
|
||||
|
||||
}*/
|
||||
|
||||
//TODO re-enable
|
||||
/*test("payment failed") {
|
||||
val router = system.actorOf(Props[Router])
|
||||
val selector = system.actorOf(Props[ChannelSelector])
|
||||
val channel00 = TestProbe()
|
||||
val channel01 = TestProbe()
|
||||
|
||||
// network: aaaa -> bbbbbbb -> cccc
|
||||
val node_a = Globals.Node.publicKey
|
||||
val node_b = BinaryData("bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb")
|
||||
val node_c = BinaryData("ccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccc")
|
||||
|
||||
router ! ChannelDiscovered(ChannelDesc("01", node_a, node_b))
|
||||
router ! ChannelDiscovered(ChannelDesc("02", node_b, node_c))
|
||||
|
||||
selector ! ChannelChangedState(channel00.ref, node_b, OPEN_WAIT_FOR_COMPLETE_OURFUNDING, NORMAL, DATA_NORMAL(Commitments(null, null, null, TheirCommit(0L, CommitmentSpec(Set(), 0L, 0L, 100000), null, null), null, null, 0L, null, null, null, null, null), null, null))
|
||||
selector ! ChannelChangedState(channel01.ref, node_b, OPEN_WAIT_FOR_COMPLETE_OURFUNDING, NORMAL, DATA_NORMAL(Commitments(null, null, null, TheirCommit(0L, CommitmentSpec(Set(), 0L, 0L, 100000000), null, null), null, null, 0L, null, null, null, null, null), null, null))
|
||||
|
||||
val paymentFsm = system.actorOf(PaymentLifecycle.props(router, selector, 1440))
|
||||
|
||||
val monitor = TestProbe()
|
||||
paymentFsm ! SubscribeTransitionCallBack(monitor.ref)
|
||||
val CurrentState(_, WAITING_FOR_REQUEST) = monitor.expectMsgClass(classOf[CurrentState[_]])
|
||||
|
||||
val sender = TestProbe()
|
||||
val req = CreatePayment(42000000, BinaryData("00112233445566778899aabbccddeeff"), node_c)
|
||||
sender.send(paymentFsm, req)
|
||||
val Transition(_, WAITING_FOR_REQUEST, WAITING_FOR_ROUTE) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
val Transition(_, WAITING_FOR_ROUTE, WAITING_FOR_CHANNEL) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
val Transition(_, WAITING_FOR_CHANNEL, WAITING_FOR_PAYMENT_COMPLETE) = monitor.expectMsgClass(classOf[Transition[_]])
|
||||
channel01.expectMsgType[CMD_ADD_HTLC]
|
||||
sender.send(paymentFsm, PaymentFailed(channel01.ref, req.h, "some reason"))
|
||||
sender.expectMsgType[Status.Failure]
|
||||
}*/
|
||||
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user