mirror of
https://github.com/ACINQ/eclair.git
synced 2025-02-23 06:35:11 +01:00
Wumbo support (#1323)
* Support wumbo channels: - use bits 18, 19 - compute the min depth for the funding transaction according to the channel size - update routing heuristics for a wumbo world: - the lower bound is the 25th percentile of current channel capacity on the network - the higher bound is the most common capacity for wumbo channels - add 'max-funding-satoshis' configuration key to allow to set the maximum channel size that will be accepted
This commit is contained in:
parent
35051d0535
commit
84edf645d0
17 changed files with 295 additions and 62 deletions
|
@ -109,6 +109,7 @@ eclair {
|
|||
payment-request-expiry = 1 hour // default expiry for payment requests generated by this node
|
||||
multi-part-payment-expiry = 60 seconds // default expiry for receiving all parts of a multi-part payment
|
||||
min-funding-satoshis = 100000
|
||||
max-funding-satoshis = 16777215 // to open channels larger than 16777215 you must enable the large_channel_support feature in 'eclair.features'
|
||||
max-payment-attempts = 5
|
||||
|
||||
autoprobe-count = 0 // number of parallel tasks that send test payments to detect invalid channels
|
||||
|
|
|
@ -79,6 +79,11 @@ object Features {
|
|||
val mandatory = 16
|
||||
}
|
||||
|
||||
case object Wumbo extends Feature {
|
||||
val rfcName = "large_channel_support"
|
||||
val mandatory = 18
|
||||
}
|
||||
|
||||
// TODO: @t-bast: update feature bits once spec-ed (currently reserved here: https://github.com/lightningnetwork/lightning-rfc/issues/605)
|
||||
// We're not advertising these bits yet in our announcements, clients have to assume support.
|
||||
// This is why we haven't added them yet to `areSupported`.
|
||||
|
@ -132,7 +137,8 @@ object Features {
|
|||
VariableLengthOnion,
|
||||
ChannelRangeQueriesExtended,
|
||||
PaymentSecret,
|
||||
BasicMultiPartPayment
|
||||
BasicMultiPartPayment,
|
||||
Wumbo
|
||||
).map(_.mandatory.toLong)
|
||||
val reversed = features.reverse
|
||||
for (i <- 0L until reversed.length by 2) {
|
||||
|
|
|
@ -77,6 +77,7 @@ case class NodeParams(keyManager: KeyManager,
|
|||
paymentRequestExpiry: FiniteDuration,
|
||||
multiPartPaymentExpiry: FiniteDuration,
|
||||
minFundingSatoshis: Satoshi,
|
||||
maxFundingSatoshis: Satoshi,
|
||||
routerConf: RouterConf,
|
||||
socksProxy_opt: Option[Socks5ProxyParams],
|
||||
maxPaymentAttempts: Int,
|
||||
|
@ -263,6 +264,7 @@ object NodeParams {
|
|||
paymentRequestExpiry = FiniteDuration(config.getDuration("payment-request-expiry").getSeconds, TimeUnit.SECONDS),
|
||||
multiPartPaymentExpiry = FiniteDuration(config.getDuration("multi-part-payment-expiry").getSeconds, TimeUnit.SECONDS),
|
||||
minFundingSatoshis = Satoshi(config.getLong("min-funding-satoshis")),
|
||||
maxFundingSatoshis = Satoshi(config.getLong("max-funding-satoshis")),
|
||||
routerConf = RouterConf(
|
||||
channelExcludeDuration = FiniteDuration(config.getDuration("router.channel-exclude-duration").getSeconds, TimeUnit.SECONDS),
|
||||
routerBroadcastInterval = FiniteDuration(config.getDuration("router.broadcast-interval").getSeconds, TimeUnit.SECONDS),
|
||||
|
|
|
@ -282,7 +282,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
|
|||
val channelVersion = ChannelVersion.STANDARD
|
||||
val channelKeyPath = keyManager.channelKeyPath(localParams, channelVersion)
|
||||
// TODO: maybe also check uniqueness of temporary channel id
|
||||
val minimumDepth = nodeParams.minDepthBlocks
|
||||
val minimumDepth = Helpers.minDepthForFunding(nodeParams, open.fundingSatoshis)
|
||||
val accept = AcceptChannel(temporaryChannelId = open.temporaryChannelId,
|
||||
dustLimitSatoshis = localParams.dustLimit,
|
||||
maxHtlcValueInFlightMsat = localParams.maxHtlcValueInFlightMsat,
|
||||
|
@ -443,8 +443,9 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
|
|||
context.system.eventStream.publish(ChannelSignatureReceived(self, commitments))
|
||||
// NB: we don't send a ChannelSignatureSent for the first commit
|
||||
log.info(s"waiting for them to publish the funding tx for channelId=$channelId fundingTxid=${commitInput.outPoint.txid}")
|
||||
val fundingMinDepth = Helpers.minDepthForFunding(nodeParams, fundingAmount)
|
||||
blockchain ! WatchSpent(self, commitInput.outPoint.txid, commitInput.outPoint.index.toInt, commitments.commitInput.txOut.publicKeyScript, BITCOIN_FUNDING_SPENT) // TODO: should we wait for an acknowledgment from the watcher?
|
||||
blockchain ! WatchConfirmed(self, commitInput.outPoint.txid, commitments.commitInput.txOut.publicKeyScript, nodeParams.minDepthBlocks, BITCOIN_FUNDING_DEPTHOK)
|
||||
blockchain ! WatchConfirmed(self, commitInput.outPoint.txid, commitments.commitInput.txOut.publicKeyScript, fundingMinDepth, BITCOIN_FUNDING_DEPTHOK)
|
||||
context.system.scheduler.scheduleOnce(FUNDING_TIMEOUT_FUNDEE, self, BITCOIN_FUNDING_TIMEOUT)
|
||||
goto(WAIT_FOR_FUNDING_CONFIRMED) using DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments, None, now, None, Right(fundingSigned)) storing() sending fundingSigned
|
||||
}
|
||||
|
@ -1442,8 +1443,14 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId
|
|||
|
||||
when(SYNCING)(handleExceptions {
|
||||
case Event(_: ChannelReestablish, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) =>
|
||||
val minDepth = if (d.commitments.localParams.isFunder) {
|
||||
nodeParams.minDepthBlocks
|
||||
} else {
|
||||
// when we're fundee we scale the min_depth confirmations depending on the funding amount
|
||||
Helpers.minDepthForFunding(nodeParams, d.commitments.commitInput.txOut.amount)
|
||||
}
|
||||
// we put back the watch (operation is idempotent) because the event may have been fired while we were in OFFLINE
|
||||
blockchain ! WatchConfirmed(self, d.commitments.commitInput.outPoint.txid, d.commitments.commitInput.txOut.publicKeyScript, nodeParams.minDepthBlocks, BITCOIN_FUNDING_DEPTHOK)
|
||||
blockchain ! WatchConfirmed(self, d.commitments.commitInput.outPoint.txid, d.commitments.commitInput.txOut.publicKeyScript, minDepth, BITCOIN_FUNDING_DEPTHOK)
|
||||
goto(WAIT_FOR_FUNDING_CONFIRMED)
|
||||
|
||||
case Event(_: ChannelReestablish, d: DATA_WAIT_FOR_FUNDING_LOCKED) =>
|
||||
|
|
|
@ -20,6 +20,8 @@ import akka.event.{DiagnosticLoggingAdapter, LoggingAdapter}
|
|||
import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey, ripemd160, sha256}
|
||||
import fr.acinq.bitcoin.Script._
|
||||
import fr.acinq.bitcoin._
|
||||
import fr.acinq.eclair.Features.Wumbo
|
||||
import fr.acinq.eclair.Features.hasFeature
|
||||
import fr.acinq.eclair.blockchain.EclairWallet
|
||||
import fr.acinq.eclair.blockchain.fee.{FeeEstimator, FeeTargets}
|
||||
import fr.acinq.eclair.channel.Channel.REFRESH_CHANNEL_UPDATE_INTERVAL
|
||||
|
@ -80,6 +82,23 @@ object Helpers {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the number of confirmations needed to safely handle the funding transaction,
|
||||
* we make sure the cumulative block reward largely exceeds the channel size.
|
||||
*
|
||||
* @param nodeParams
|
||||
* @param fundingSatoshis funding amount of the channel
|
||||
* @return number of confirmations needed
|
||||
*/
|
||||
def minDepthForFunding(nodeParams: NodeParams, fundingSatoshis: Satoshi): Long = fundingSatoshis match {
|
||||
case funding if funding <= Channel.MAX_FUNDING => nodeParams.minDepthBlocks
|
||||
case funding if funding > Channel.MAX_FUNDING =>
|
||||
val blockReward = 6.25 // this is true as of ~May 2020, but will be too large after 2024
|
||||
val scalingFactor = 15
|
||||
val blocksToReachFunding = (((scalingFactor * funding.toBtc.toDouble) / blockReward).ceil + 1).toInt
|
||||
nodeParams.minDepthBlocks.max(blocksToReachFunding)
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by the fundee
|
||||
*/
|
||||
|
@ -87,7 +106,11 @@ object Helpers {
|
|||
// BOLT #2: if the chain_hash value, within the open_channel, message is set to a hash of a chain that is unknown to the receiver:
|
||||
// MUST reject the channel.
|
||||
if (nodeParams.chainHash != open.chainHash) throw InvalidChainHash(open.temporaryChannelId, local = nodeParams.chainHash, remote = open.chainHash)
|
||||
if (open.fundingSatoshis < nodeParams.minFundingSatoshis || open.fundingSatoshis >= Channel.MAX_FUNDING) throw InvalidFundingAmount(open.temporaryChannelId, open.fundingSatoshis, nodeParams.minFundingSatoshis, Channel.MAX_FUNDING)
|
||||
|
||||
if(open.fundingSatoshis < nodeParams.minFundingSatoshis || open.fundingSatoshis > nodeParams.maxFundingSatoshis) throw InvalidFundingAmount(open.temporaryChannelId, open.fundingSatoshis, nodeParams.minFundingSatoshis, nodeParams.maxFundingSatoshis)
|
||||
|
||||
// BOLT #2: Channel funding limits
|
||||
if (open.fundingSatoshis >= Channel.MAX_FUNDING && !hasFeature(nodeParams.features, Wumbo)) throw InvalidFundingAmount(open.temporaryChannelId, open.fundingSatoshis, nodeParams.minFundingSatoshis, Channel.MAX_FUNDING)
|
||||
|
||||
// BOLT #2: The receiving node MUST fail the channel if: push_msat is greater than funding_satoshis * 1000.
|
||||
if (open.pushMsat > open.fundingSatoshis) throw InvalidPushAmount(open.temporaryChannelId, open.pushMsat, open.fundingSatoshis.toMilliSatoshi)
|
||||
|
|
|
@ -24,6 +24,7 @@ import akka.util.Timeout
|
|||
import com.google.common.net.HostAndPort
|
||||
import fr.acinq.bitcoin.Crypto.PublicKey
|
||||
import fr.acinq.bitcoin.{ByteVector32, DeterministicWallet, Satoshi}
|
||||
import fr.acinq.eclair.Features.Wumbo
|
||||
import fr.acinq.eclair.Logs.LogCategory
|
||||
import fr.acinq.eclair.blockchain.EclairWallet
|
||||
import fr.acinq.eclair.channel._
|
||||
|
@ -308,14 +309,25 @@ class Peer(val nodeParams: NodeParams, remoteNodeId: PublicKey, authenticator: A
|
|||
stay
|
||||
|
||||
case Event(c: Peer.OpenChannel, d: ConnectedData) =>
|
||||
val (channel, localParams) = createNewChannel(nodeParams, funder = true, c.fundingSatoshis, origin_opt = Some(sender))
|
||||
c.timeout_opt.map(openTimeout => context.system.scheduler.scheduleOnce(openTimeout.duration, channel, Channel.TickChannelOpenTimeout)(context.dispatcher))
|
||||
val temporaryChannelId = randomBytes32
|
||||
val channelFeeratePerKw = nodeParams.onChainFeeConf.feeEstimator.getFeeratePerKw(target = nodeParams.onChainFeeConf.feeTargets.commitmentBlockTarget)
|
||||
val fundingTxFeeratePerKw = c.fundingTxFeeratePerKw_opt.getOrElse(nodeParams.onChainFeeConf.feeEstimator.getFeeratePerKw(target = nodeParams.onChainFeeConf.feeTargets.fundingBlockTarget))
|
||||
log.info(s"requesting a new channel with fundingSatoshis=${c.fundingSatoshis}, pushMsat=${c.pushMsat} and fundingFeeratePerByte=${c.fundingTxFeeratePerKw_opt} temporaryChannelId=$temporaryChannelId localParams=$localParams")
|
||||
channel ! INPUT_INIT_FUNDER(temporaryChannelId, c.fundingSatoshis, c.pushMsat, channelFeeratePerKw, fundingTxFeeratePerKw, localParams, d.transport, d.remoteInit, c.channelFlags.getOrElse(nodeParams.channelFlags), ChannelVersion.STANDARD)
|
||||
stay using d.copy(channels = d.channels + (TemporaryChannelId(temporaryChannelId) -> channel))
|
||||
if (c.fundingSatoshis >= Channel.MAX_FUNDING && !Features.hasFeature(nodeParams.features, Wumbo)) {
|
||||
sender ! Status.Failure(new RuntimeException(s"fundingSatoshis=${c.fundingSatoshis} is too big, you must enable large channels support in 'eclair.features' to use funding above ${Channel.MAX_FUNDING} (see eclair.conf)"))
|
||||
stay
|
||||
} else if (c.fundingSatoshis >= Channel.MAX_FUNDING && !Features.hasFeature(d.remoteInit.features, Wumbo)) {
|
||||
sender ! Status.Failure(new RuntimeException(s"fundingSatoshis=${c.fundingSatoshis} is too big, the remote peer doesn't support wumbo"))
|
||||
stay
|
||||
} else if (c.fundingSatoshis > nodeParams.maxFundingSatoshis) {
|
||||
sender ! Status.Failure(new RuntimeException(s"fundingSatoshis=${c.fundingSatoshis} is too big for the current settings, increase 'eclair.max-funding-satoshis' (see eclair.conf)"))
|
||||
stay
|
||||
} else {
|
||||
val (channel, localParams) = createNewChannel(nodeParams, funder = true, c.fundingSatoshis, origin_opt = Some(sender))
|
||||
c.timeout_opt.map(openTimeout => context.system.scheduler.scheduleOnce(openTimeout.duration, channel, Channel.TickChannelOpenTimeout)(context.dispatcher))
|
||||
val temporaryChannelId = randomBytes32
|
||||
val channelFeeratePerKw = nodeParams.onChainFeeConf.feeEstimator.getFeeratePerKw(target = nodeParams.onChainFeeConf.feeTargets.commitmentBlockTarget)
|
||||
val fundingTxFeeratePerKw = c.fundingTxFeeratePerKw_opt.getOrElse(nodeParams.onChainFeeConf.feeEstimator.getFeeratePerKw(target = nodeParams.onChainFeeConf.feeTargets.fundingBlockTarget))
|
||||
log.info(s"requesting a new channel with fundingSatoshis=${c.fundingSatoshis}, pushMsat=${c.pushMsat} and fundingFeeratePerByte=${c.fundingTxFeeratePerKw_opt} temporaryChannelId=$temporaryChannelId localParams=$localParams")
|
||||
channel ! INPUT_INIT_FUNDER(temporaryChannelId, c.fundingSatoshis, c.pushMsat, channelFeeratePerKw, fundingTxFeeratePerKw, localParams, d.transport, d.remoteInit, c.channelFlags.getOrElse(nodeParams.channelFlags), ChannelVersion.STANDARD)
|
||||
stay using d.copy(channels = d.channels + (TemporaryChannelId(temporaryChannelId) -> channel))
|
||||
}
|
||||
|
||||
case Event(msg: wire.OpenChannel, d: ConnectedData) =>
|
||||
d.transport ! TransportHandler.ReadAck(msg)
|
||||
|
@ -678,7 +690,6 @@ object Peer {
|
|||
case class Disconnect(nodeId: PublicKey)
|
||||
case object ResumeAnnouncements
|
||||
case class OpenChannel(remoteNodeId: PublicKey, fundingSatoshis: Satoshi, pushMsat: MilliSatoshi, fundingTxFeeratePerKw_opt: Option[Long], channelFlags: Option[Byte], timeout_opt: Option[Timeout]) {
|
||||
require(fundingSatoshis < Channel.MAX_FUNDING, s"fundingSatoshis must be less than ${Channel.MAX_FUNDING}")
|
||||
require(pushMsat <= fundingSatoshis, s"pushMsat must be less or equal to fundingSatoshis")
|
||||
require(fundingSatoshis >= 0.sat, s"fundingSatoshis must be positive")
|
||||
require(pushMsat >= 0.msat, s"pushMsat must be positive")
|
||||
|
|
|
@ -16,9 +16,9 @@
|
|||
|
||||
package fr.acinq.eclair.router
|
||||
|
||||
import fr.acinq.bitcoin.Btc
|
||||
import fr.acinq.bitcoin.Crypto.PublicKey
|
||||
import fr.acinq.eclair._
|
||||
import fr.acinq.eclair.channel.Channel
|
||||
import fr.acinq.eclair.router.Graph.GraphStructure.{DirectedGraph, GraphEdge}
|
||||
import fr.acinq.eclair.router.Router._
|
||||
import fr.acinq.eclair.wire.ChannelUpdate
|
||||
|
@ -351,8 +351,8 @@ object Graph {
|
|||
val BLOCK_TIME_TWO_MONTHS = 8640
|
||||
|
||||
// Low/High bound for channel capacity
|
||||
val CAPACITY_CHANNEL_LOW = (1000 sat).toMilliSatoshi
|
||||
val CAPACITY_CHANNEL_HIGH = Channel.MAX_FUNDING.toMilliSatoshi
|
||||
val CAPACITY_CHANNEL_LOW = Btc(0.001).toMilliSatoshi
|
||||
val CAPACITY_CHANNEL_HIGH = Btc(1).toMilliSatoshi
|
||||
|
||||
// Low/High bound for CLTV channel value
|
||||
val CLTV_LOW = 9
|
||||
|
|
|
@ -99,6 +99,8 @@ class FeaturesSpec extends FunSuite {
|
|||
assert(areSupported(ByteVector.fromLong(1L << PaymentSecret.optional)))
|
||||
assert(areSupported(ByteVector.fromLong(1L << BasicMultiPartPayment.mandatory)))
|
||||
assert(areSupported(ByteVector.fromLong(1L << BasicMultiPartPayment.optional)))
|
||||
assert(areSupported(ByteVector.fromLong(1L << Wumbo.mandatory)))
|
||||
assert(areSupported(ByteVector.fromLong(1L << Wumbo.optional)))
|
||||
|
||||
val testCases = Map(
|
||||
bin" 00000000000000001011" -> true,
|
||||
|
@ -108,11 +110,12 @@ class FeaturesSpec extends FunSuite {
|
|||
bin" 00011000001000000000" -> true,
|
||||
bin" 00101000000000000000" -> true,
|
||||
bin" 00000000010001000000" -> true,
|
||||
// unknown optional feature bits
|
||||
bin" 01000000000000000000" -> true,
|
||||
bin" 10000000000000000000" -> true,
|
||||
// unknown optional feature bits
|
||||
bin" 001000000000000000000000" -> true,
|
||||
bin" 100000000000000000000000" -> true,
|
||||
// those are useful for nonreg testing of the areSupported method (which needs to be updated with every new supported mandatory bit)
|
||||
bin" 000001000000000000000000" -> false,
|
||||
bin" 000100000000000000000000" -> false,
|
||||
bin" 010000000000000000000000" -> false,
|
||||
bin" 0001000000000000000000000000" -> false,
|
||||
|
|
|
@ -20,7 +20,7 @@ import java.sql.{Connection, DriverManager}
|
|||
import java.util.concurrent.atomic.AtomicLong
|
||||
|
||||
import fr.acinq.bitcoin.Crypto.PrivateKey
|
||||
import fr.acinq.bitcoin.{Block, ByteVector32, Script}
|
||||
import fr.acinq.bitcoin.{Block, Btc, ByteVector32, Script}
|
||||
import fr.acinq.eclair.NodeParams.BITCOIND
|
||||
import fr.acinq.eclair.blockchain.fee.{FeeEstimator, FeeTargets, FeeratesPerKw, OnChainFeeConf}
|
||||
import fr.acinq.eclair.crypto.LocalKeyManager
|
||||
|
@ -107,6 +107,7 @@ object TestConstants {
|
|||
paymentRequestExpiry = 1 hour,
|
||||
multiPartPaymentExpiry = 30 seconds,
|
||||
minFundingSatoshis = 1000 sat,
|
||||
maxFundingSatoshis = 16777215 sat,
|
||||
routerConf = RouterConf(
|
||||
randomizeRouteSelection = false,
|
||||
channelExcludeDuration = 60 seconds,
|
||||
|
@ -186,6 +187,7 @@ object TestConstants {
|
|||
paymentRequestExpiry = 1 hour,
|
||||
multiPartPaymentExpiry = 30 seconds,
|
||||
minFundingSatoshis = 1000 sat,
|
||||
maxFundingSatoshis = 16777215 sat,
|
||||
routerConf = RouterConf(
|
||||
randomizeRouteSelection = false,
|
||||
channelExcludeDuration = 60 seconds,
|
||||
|
|
|
@ -16,7 +16,8 @@
|
|||
|
||||
package fr.acinq.eclair.channel
|
||||
|
||||
import fr.acinq.bitcoin.Transaction
|
||||
import fr.acinq.bitcoin.{Btc, Transaction}
|
||||
import fr.acinq.eclair.TestConstants.Alice.nodeParams
|
||||
import fr.acinq.eclair.TestUtils.NoLoggingDiagnostics
|
||||
import fr.acinq.eclair.channel.Helpers.Closing
|
||||
import fr.acinq.eclair.channel.Helpers.Closing._
|
||||
|
@ -27,6 +28,16 @@ import scala.concurrent.duration._
|
|||
|
||||
class HelpersSpec extends FunSuite {
|
||||
|
||||
test("compute the funding tx min depth according to funding amount") {
|
||||
assert(Helpers.minDepthForFunding(nodeParams, Btc(1)) == 4)
|
||||
assert(Helpers.minDepthForFunding(nodeParams.copy(minDepthBlocks = 6), Btc(1)) == 6) // 4 conf would be enough but we use min-depth=6
|
||||
assert(Helpers.minDepthForFunding(nodeParams, Btc(6.25)) == 16) // we use scaling_factor=15 and a fixed block reward of 6.25BTC
|
||||
assert(Helpers.minDepthForFunding(nodeParams, Btc(12.50)) == 31)
|
||||
assert(Helpers.minDepthForFunding(nodeParams, Btc(12.60)) == 32)
|
||||
assert(Helpers.minDepthForFunding(nodeParams, Btc(30)) == 73)
|
||||
assert(Helpers.minDepthForFunding(nodeParams, Btc(50)) == 121)
|
||||
}
|
||||
|
||||
test("compute refresh delay") {
|
||||
import org.scalatest.Matchers._
|
||||
implicit val log = NoLoggingDiagnostics
|
||||
|
|
|
@ -17,7 +17,7 @@
|
|||
package fr.acinq.eclair.channel.states.a
|
||||
|
||||
import akka.testkit.{TestFSMRef, TestProbe}
|
||||
import fr.acinq.bitcoin.{Block, ByteVector32, Satoshi}
|
||||
import fr.acinq.bitcoin.{Block, Btc, ByteVector32, Satoshi}
|
||||
import fr.acinq.eclair.TestConstants.{Alice, Bob}
|
||||
import fr.acinq.eclair.blockchain.{MakeFundingTxResponse, TestWallet}
|
||||
import fr.acinq.eclair.channel.Channel.TickChannelOpenTimeout
|
||||
|
@ -26,7 +26,7 @@ import fr.acinq.eclair.channel.{WAIT_FOR_FUNDING_INTERNAL, _}
|
|||
import fr.acinq.eclair.wire.{AcceptChannel, ChannelTlv, Error, Init, OpenChannel, TlvStream}
|
||||
import fr.acinq.eclair.{CltvExpiryDelta, LongToBtcAmount, TestConstants, TestkitBaseClass}
|
||||
import org.scalatest.{Outcome, Tag}
|
||||
import scodec.bits.ByteVector
|
||||
import scodec.bits.{ByteVector, HexStringSyntax}
|
||||
|
||||
import scala.concurrent.duration._
|
||||
import scala.concurrent.{Future, Promise}
|
||||
|
@ -43,18 +43,28 @@ class WaitForAcceptChannelStateSpec extends TestkitBaseClass with StateTestsHelp
|
|||
val noopWallet = new TestWallet {
|
||||
override def makeFundingTx(pubkeyScript: ByteVector, amount: Satoshi, feeRatePerKw: Long): Future[MakeFundingTxResponse] = Promise[MakeFundingTxResponse].future // will never be completed
|
||||
}
|
||||
val setup = if (test.tags.contains("mainnet")) {
|
||||
init(TestConstants.Alice.nodeParams.copy(chainHash = Block.LivenetGenesisBlock.hash), TestConstants.Bob.nodeParams.copy(chainHash = Block.LivenetGenesisBlock.hash), wallet = noopWallet)
|
||||
} else {
|
||||
init(wallet = noopWallet)
|
||||
}
|
||||
|
||||
import com.softwaremill.quicklens._
|
||||
val aliceNodeParams = TestConstants.Alice.nodeParams
|
||||
.modify(_.chainHash).setToIf(test.tags.contains("mainnet"))(Block.LivenetGenesisBlock.hash)
|
||||
.modify(_.features).setToIf(test.tags.contains("wumbo"))(hex"80000")
|
||||
.modify(_.maxFundingSatoshis).setToIf(test.tags.contains("high-max-funding-size"))(Btc(100))
|
||||
|
||||
val bobNodeParams = TestConstants.Bob.nodeParams
|
||||
.modify(_.chainHash).setToIf(test.tags.contains("mainnet"))(Block.LivenetGenesisBlock.hash)
|
||||
.modify(_.features).setToIf(test.tags.contains("wumbo"))(hex"80000")
|
||||
.modify(_.maxFundingSatoshis).setToIf(test.tags.contains("high-max-funding-size"))(Btc(100))
|
||||
|
||||
val setup = init(aliceNodeParams, bobNodeParams, wallet = noopWallet)
|
||||
|
||||
import setup._
|
||||
val channelVersion = ChannelVersion.STANDARD
|
||||
val (aliceParams, bobParams) = (Alice.channelParams, Bob.channelParams)
|
||||
val aliceInit = Init(aliceParams.features)
|
||||
val bobInit = Init(bobParams.features)
|
||||
within(30 seconds) {
|
||||
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, TestConstants.fundingSatoshis, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, aliceParams, alice2bob.ref, bobInit, ChannelFlags.Empty, channelVersion)
|
||||
val fundingAmount = if(test.tags.contains("wumbo")) Btc(5).toSatoshi else TestConstants.fundingSatoshis
|
||||
alice ! INPUT_INIT_FUNDER(ByteVector32.Zeroes, fundingAmount, TestConstants.pushMsat, TestConstants.feeratePerKw, TestConstants.feeratePerKw, aliceParams, alice2bob.ref, bobInit, ChannelFlags.Empty, channelVersion)
|
||||
bob ! INPUT_INIT_FUNDEE(ByteVector32.Zeroes, bobParams, bob2alice.ref, aliceInit)
|
||||
alice2bob.expectMsgType[OpenChannel]
|
||||
alice2bob.forward(bob)
|
||||
|
@ -147,6 +157,14 @@ class WaitForAcceptChannelStateSpec extends TestkitBaseClass with StateTestsHelp
|
|||
awaitCond(alice.stateName == CLOSED)
|
||||
}
|
||||
|
||||
test("recv AcceptChannel (wumbo size channel)", Tag("wumbo"), Tag("high-max-funding-size")) { f =>
|
||||
import f._
|
||||
val accept = bob2alice.expectMsgType[AcceptChannel]
|
||||
assert(accept.minimumDepth == 13) // with wumbo tag we use fundingSatoshis=5BTC
|
||||
bob2alice.forward(alice, accept)
|
||||
awaitCond(alice.stateName == WAIT_FOR_FUNDING_INTERNAL)
|
||||
}
|
||||
|
||||
test("recv Error") { f =>
|
||||
import f._
|
||||
alice ! Error(ByteVector32.Zeroes, "oops")
|
||||
|
|
|
@ -17,14 +17,15 @@
|
|||
package fr.acinq.eclair.channel.states.a
|
||||
|
||||
import akka.testkit.{TestFSMRef, TestProbe}
|
||||
import fr.acinq.bitcoin.{Block, ByteVector32}
|
||||
import fr.acinq.bitcoin.{Block, Btc, ByteVector32}
|
||||
import fr.acinq.eclair.TestConstants.{Alice, Bob}
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.channel.states.StateTestsHelperMethods
|
||||
import fr.acinq.eclair.wire.{ChannelTlv, Error, Init, OpenChannel, TlvStream}
|
||||
import fr.acinq.eclair.wire.{AcceptChannel, Error, Init, OpenChannel}
|
||||
import fr.acinq.eclair.{CltvExpiryDelta, LongToBtcAmount, TestConstants, TestkitBaseClass, ToMilliSatoshiConversion}
|
||||
import org.scalatest.Outcome
|
||||
import scodec.bits.ByteVector
|
||||
import org.scalatest.{Outcome, Tag}
|
||||
import scodec.bits.{ByteVector, HexStringSyntax}
|
||||
|
||||
import scala.concurrent.duration._
|
||||
|
||||
|
@ -37,7 +38,13 @@ class WaitForOpenChannelStateSpec extends TestkitBaseClass with StateTestsHelper
|
|||
case class FixtureParam(bob: TestFSMRef[State, Data, Channel], alice2bob: TestProbe, bob2alice: TestProbe, bob2blockchain: TestProbe)
|
||||
|
||||
override def withFixture(test: OneArgTest): Outcome = {
|
||||
val setup = init()
|
||||
import com.softwaremill.quicklens._
|
||||
val bobNodeParams = Bob.nodeParams
|
||||
.modify(_.features).setToIf(test.tags.contains("wumbo"))(hex"80000")
|
||||
.modify(_.maxFundingSatoshis).setToIf(test.tags.contains("max-funding-satoshis"))(Btc(1))
|
||||
|
||||
val setup = init(nodeParamsB = bobNodeParams)
|
||||
|
||||
import setup._
|
||||
val channelVersion = ChannelVersion.STANDARD
|
||||
val (aliceParams, bobParams) = (Alice.channelParams, Bob.channelParams)
|
||||
|
@ -77,20 +84,29 @@ class WaitForOpenChannelStateSpec extends TestkitBaseClass with StateTestsHelper
|
|||
val lowFunding = 100.sat
|
||||
bob ! open.copy(fundingSatoshis = lowFunding)
|
||||
val error = bob2alice.expectMsgType[Error]
|
||||
assert(error === Error(open.temporaryChannelId, InvalidFundingAmount(open.temporaryChannelId, lowFunding, Bob.nodeParams.minFundingSatoshis, Channel.MAX_FUNDING).getMessage))
|
||||
assert(error === Error(open.temporaryChannelId, InvalidFundingAmount(open.temporaryChannelId, lowFunding, Bob.nodeParams.minFundingSatoshis, Bob.nodeParams.maxFundingSatoshis).getMessage))
|
||||
awaitCond(bob.stateName == CLOSED)
|
||||
}
|
||||
|
||||
test("recv OpenChannel (funding too high)") { f =>
|
||||
test("recv OpenChannel (funding over channel limit)") { f =>
|
||||
import f._
|
||||
val open = alice2bob.expectMsgType[OpenChannel]
|
||||
val highFundingMsat = 100000000.sat
|
||||
bob ! open.copy(fundingSatoshis = highFundingMsat)
|
||||
val error = bob2alice.expectMsgType[Error]
|
||||
assert(error.toAscii === Error(open.temporaryChannelId, InvalidFundingAmount(open.temporaryChannelId, highFundingMsat, Bob.nodeParams.minFundingSatoshis, Channel.MAX_FUNDING).getMessage).toAscii)
|
||||
assert(error.toAscii === Error(open.temporaryChannelId, InvalidFundingAmount(open.temporaryChannelId, highFundingMsat, Bob.nodeParams.minFundingSatoshis, Bob.nodeParams.maxFundingSatoshis).getMessage).toAscii)
|
||||
awaitCond(bob.stateName == CLOSED)
|
||||
}
|
||||
|
||||
test("recv OpenChannel (fundingSatoshis > max-funding-satoshis)", Tag("wumbo")) { f =>
|
||||
import f._
|
||||
val open = alice2bob.expectMsgType[OpenChannel]
|
||||
val highFundingSat = Bob.nodeParams.maxFundingSatoshis + Btc(1)
|
||||
bob ! open.copy(fundingSatoshis = highFundingSat)
|
||||
val error = bob2alice.expectMsgType[Error]
|
||||
assert(error.toAscii === Error(open.temporaryChannelId, InvalidFundingAmount(open.temporaryChannelId, highFundingSat, Bob.nodeParams.minFundingSatoshis, Bob.nodeParams.maxFundingSatoshis).getMessage).toAscii)
|
||||
}
|
||||
|
||||
test("recv OpenChannel (invalid max accepted htlcs)") { f =>
|
||||
import f._
|
||||
val open = alice2bob.expectMsgType[OpenChannel]
|
||||
|
@ -180,6 +196,15 @@ class WaitForOpenChannelStateSpec extends TestkitBaseClass with StateTestsHelper
|
|||
awaitCond(bob.stateName == CLOSED)
|
||||
}
|
||||
|
||||
test("recv OpenChannel (wumbo size)", Tag("wumbo"), Tag("max-funding-satoshis")) { f =>
|
||||
import f._
|
||||
val open = alice2bob.expectMsgType[OpenChannel]
|
||||
val highFundingSat = Btc(1).toSatoshi
|
||||
bob ! open.copy(fundingSatoshis = highFundingSat)
|
||||
bob2alice.expectMsgType[AcceptChannel]
|
||||
awaitCond(bob.stateName == WAIT_FOR_FUNDING_CREATED)
|
||||
}
|
||||
|
||||
test("recv Error") { f =>
|
||||
import f._
|
||||
bob ! Error(ByteVector32.Zeroes, "oops")
|
||||
|
|
|
@ -24,17 +24,18 @@ import akka.testkit.{TestKit, TestProbe}
|
|||
import com.google.common.net.HostAndPort
|
||||
import com.typesafe.config.{Config, ConfigFactory}
|
||||
import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey}
|
||||
import fr.acinq.bitcoin.{Base58, Base58Check, Bech32, Block, ByteVector32, Crypto, OP_0, OP_CHECKSIG, OP_DUP, OP_EQUAL, OP_EQUALVERIFY, OP_HASH160, OP_PUSHDATA, Satoshi, Script, ScriptFlags, Transaction}
|
||||
import fr.acinq.bitcoin.{Base58, Base58Check, Bech32, Block, Btc, ByteVector32, Crypto, OP_0, OP_CHECKSIG, OP_DUP, OP_EQUAL, OP_EQUALVERIFY, OP_HASH160, OP_PUSHDATA, Satoshi, Script, ScriptFlags, Transaction}
|
||||
import fr.acinq.eclair.blockchain.bitcoind.BitcoindService
|
||||
import fr.acinq.eclair.blockchain.bitcoind.rpc.ExtendedBitcoinClient
|
||||
import fr.acinq.eclair.blockchain.{Watch, WatchConfirmed}
|
||||
import fr.acinq.eclair.channel.Channel.{BroadcastChannelUpdate, PeriodicRefresh}
|
||||
import fr.acinq.eclair.channel.ChannelCommandResponse.ChannelOpened
|
||||
import fr.acinq.eclair.channel.Register.{Forward, ForwardShortId}
|
||||
import fr.acinq.eclair.channel._
|
||||
import fr.acinq.eclair.crypto.Sphinx.DecryptedFailurePacket
|
||||
import fr.acinq.eclair.db._
|
||||
import fr.acinq.eclair.io.Peer
|
||||
import fr.acinq.eclair.io.Peer.{Disconnect, PeerRoutingMessage}
|
||||
import fr.acinq.eclair.io.Peer.{Disconnect, GetPeerInfo, PeerInfo, PeerRoutingMessage}
|
||||
import fr.acinq.eclair.payment.PaymentRequest.ExtraHop
|
||||
import fr.acinq.eclair.payment._
|
||||
import fr.acinq.eclair.payment.receive.MultiPartHandler.ReceivePayment
|
||||
|
@ -49,7 +50,7 @@ import fr.acinq.eclair.router.{Announcements, AnnouncementsBatchValidationSpec,
|
|||
import fr.acinq.eclair.transactions.Transactions
|
||||
import fr.acinq.eclair.transactions.Transactions.{HtlcSuccessTx, HtlcTimeoutTx}
|
||||
import fr.acinq.eclair.wire._
|
||||
import fr.acinq.eclair.{CltvExpiryDelta, Kit, LongToBtcAmount, MilliSatoshi, Setup, ShortChannelId, randomBytes32}
|
||||
import fr.acinq.eclair.{CltvExpiryDelta, Kit, LongToBtcAmount, MilliSatoshi, Setup, ShortChannelId, ToMilliSatoshiConversion, randomBytes32}
|
||||
import grizzled.slf4j.Logging
|
||||
import org.json4s.DefaultFormats
|
||||
import org.json4s.JsonAST.{JString, JValue}
|
||||
|
@ -61,6 +62,7 @@ import scala.compat.Platform
|
|||
import scala.concurrent.Await
|
||||
import scala.concurrent.ExecutionContext.Implicits.global
|
||||
import scala.concurrent.duration._
|
||||
import scala.util.{Failure, Success, Try}
|
||||
|
||||
/**
|
||||
* Created by PM on 15/03/2017.
|
||||
|
@ -149,10 +151,10 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
|||
import collection.JavaConversions._
|
||||
instantiateEclairNode("A", ConfigFactory.parseMap(Map("eclair.node-alias" -> "A", "eclair.expiry-delta-blocks" -> 130, "eclair.server.port" -> 29730, "eclair.api.port" -> 28080, "eclair.features" -> "028a8a", "eclair.channel-flags" -> 0)).withFallback(commonConfig)) // A's channels are private
|
||||
instantiateEclairNode("B", ConfigFactory.parseMap(Map("eclair.node-alias" -> "B", "eclair.expiry-delta-blocks" -> 131, "eclair.server.port" -> 29731, "eclair.api.port" -> 28081, "eclair.features" -> "028a8a", "eclair.trampoline-payments-enable" -> true)).withFallback(commonConfig))
|
||||
instantiateEclairNode("C", ConfigFactory.parseMap(Map("eclair.node-alias" -> "C", "eclair.expiry-delta-blocks" -> 132, "eclair.server.port" -> 29732, "eclair.api.port" -> 28082, "eclair.features" -> "028a8a", "eclair.trampoline-payments-enable" -> true, "eclair.max-payment-attempts" -> 15)).withFallback(commonConfig))
|
||||
instantiateEclairNode("C", ConfigFactory.parseMap(Map("eclair.node-alias" -> "C", "eclair.expiry-delta-blocks" -> 132, "eclair.server.port" -> 29732, "eclair.api.port" -> 28082, "eclair.features" -> "0a8a8a", "eclair.max-funding-satoshis" -> 500000000, "eclair.trampoline-payments-enable" -> true, "eclair.max-payment-attempts" -> 15)).withFallback(commonConfig))
|
||||
instantiateEclairNode("D", ConfigFactory.parseMap(Map("eclair.node-alias" -> "D", "eclair.expiry-delta-blocks" -> 133, "eclair.server.port" -> 29733, "eclair.api.port" -> 28083, "eclair.features" -> "028a8a", "eclair.trampoline-payments-enable" -> true)).withFallback(commonConfig))
|
||||
instantiateEclairNode("E", ConfigFactory.parseMap(Map("eclair.node-alias" -> "E", "eclair.expiry-delta-blocks" -> 134, "eclair.server.port" -> 29734, "eclair.api.port" -> 28084)).withFallback(commonConfig))
|
||||
instantiateEclairNode("F1", ConfigFactory.parseMap(Map("eclair.node-alias" -> "F1", "eclair.expiry-delta-blocks" -> 135, "eclair.server.port" -> 29735, "eclair.api.port" -> 28085)).withFallback(commonConfig))
|
||||
instantiateEclairNode("F1", ConfigFactory.parseMap(Map("eclair.node-alias" -> "F1", "eclair.expiry-delta-blocks" -> 135, "eclair.server.port" -> 29735, "eclair.api.port" -> 28085, "eclair.features" -> "0a8a8a", "eclair.max-funding-satoshis" -> 500000000)).withFallback(commonConfig))
|
||||
instantiateEclairNode("F2", ConfigFactory.parseMap(Map("eclair.node-alias" -> "F2", "eclair.expiry-delta-blocks" -> 136, "eclair.server.port" -> 29736, "eclair.api.port" -> 28086)).withFallback(commonConfig))
|
||||
instantiateEclairNode("F3", ConfigFactory.parseMap(Map("eclair.node-alias" -> "F3", "eclair.expiry-delta-blocks" -> 137, "eclair.server.port" -> 29737, "eclair.api.port" -> 28087, "eclair.features" -> "028a8a", "eclair.trampoline-payments-enable" -> true)).withFallback(commonConfig))
|
||||
instantiateEclairNode("F4", ConfigFactory.parseMap(Map("eclair.node-alias" -> "F4", "eclair.expiry-delta-blocks" -> 138, "eclair.server.port" -> 29738, "eclair.api.port" -> 28088)).withFallback(commonConfig))
|
||||
|
@ -260,6 +262,86 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
|||
awaitAnnouncements(nodes.filterKeys(key => !List("A", "B").contains(key)), 10, 12, 24)
|
||||
}
|
||||
|
||||
test("open a wumbo channel and wait for longer than the default min_depth") {
|
||||
// we open a 5BTC channel and check that we scale `min_depth` up to 13 confirmations
|
||||
val funder = nodes("C")
|
||||
val fundee = nodes("F1")
|
||||
val sender = TestProbe()
|
||||
sender.send(funder.switchboard, Peer.OpenChannel(
|
||||
remoteNodeId = fundee.nodeParams.nodeId,
|
||||
fundingSatoshis = 5.btc,
|
||||
pushMsat = 0.msat,
|
||||
fundingTxFeeratePerKw_opt = None,
|
||||
channelFlags = None,
|
||||
timeout_opt = None))
|
||||
val tempChannelId = sender.expectMsgType[ChannelOpened](10 seconds).channelId
|
||||
|
||||
// mine the funding tx
|
||||
generateBlocks(bitcoincli, 2)
|
||||
|
||||
// get the channelId
|
||||
sender.send(fundee.register, 'channels)
|
||||
val Some((_, fundeeChannel)) = sender.expectMsgType[Map[ByteVector32, ActorRef]].find(_._1 == tempChannelId)
|
||||
sender.send(fundeeChannel, CMD_GETSTATEDATA)
|
||||
val channelId = sender.expectMsgType[HasCommitments].channelId
|
||||
|
||||
awaitCond({
|
||||
sender.send(funder.register, Forward(channelId, CMD_GETSTATE))
|
||||
sender.expectMsgType[State] == WAIT_FOR_FUNDING_LOCKED
|
||||
})
|
||||
|
||||
generateBlocks(bitcoincli, 6)
|
||||
|
||||
// after 8 blocks the fundee is still waiting for more confirmations
|
||||
sender.send(fundee.register, Forward(channelId, CMD_GETSTATE))
|
||||
assert(sender.expectMsgType[State] == WAIT_FOR_FUNDING_CONFIRMED)
|
||||
|
||||
// after 8 blocks the funder is still waiting for funding_locked from the fundee
|
||||
sender.send(funder.register, Forward(channelId, CMD_GETSTATE))
|
||||
assert(sender.expectMsgType[State] == WAIT_FOR_FUNDING_LOCKED)
|
||||
|
||||
// simulate a disconnection
|
||||
sender.send(funder.switchboard, Peer.Disconnect(fundee.nodeParams.nodeId))
|
||||
assert(sender.expectMsgType[String] == "disconnecting")
|
||||
|
||||
awaitCond({
|
||||
sender.send(fundee.register, Forward(channelId, CMD_GETSTATE))
|
||||
val fundeeState = sender.expectMsgType[State]
|
||||
sender.send(funder.register, Forward(channelId, CMD_GETSTATE))
|
||||
val funderState = sender.expectMsgType[State]
|
||||
fundeeState == OFFLINE && funderState == OFFLINE
|
||||
})
|
||||
|
||||
// reconnect and check the fundee is waiting for more conf, funder is waiting for fundee to send funding_locked
|
||||
awaitCond({
|
||||
// reconnection
|
||||
sender.send(fundee.switchboard, Peer.Connect(
|
||||
nodeId = funder.nodeParams.nodeId,
|
||||
address_opt = Some(HostAndPort.fromParts(funder.nodeParams.publicAddresses.head.socketAddress.getHostString, funder.nodeParams.publicAddresses.head.socketAddress.getPort))
|
||||
))
|
||||
sender.expectMsgAnyOf(10 seconds, "connected", "already connected", "reconnection in progress")
|
||||
|
||||
sender.send(fundee.register, Forward(channelId, CMD_GETSTATE))
|
||||
val fundeeState = sender.expectMsgType[State](max = 30 seconds)
|
||||
sender.send(funder.register, Forward(channelId, CMD_GETSTATE))
|
||||
val funderState = sender.expectMsgType[State](max = 30 seconds)
|
||||
fundeeState == WAIT_FOR_FUNDING_CONFIRMED && funderState == WAIT_FOR_FUNDING_LOCKED
|
||||
}, max = 30 seconds, interval = 10 seconds)
|
||||
|
||||
// 5 extra blocks make it 13, just the amount of confirmations needed
|
||||
generateBlocks(bitcoincli, 5)
|
||||
|
||||
awaitCond({
|
||||
sender.send(fundee.register, Forward(channelId, CMD_GETSTATE))
|
||||
val fundeeState = sender.expectMsgType[State]
|
||||
sender.send(funder.register, Forward(channelId, CMD_GETSTATE))
|
||||
val funderState = sender.expectMsgType[State]
|
||||
fundeeState == NORMAL && funderState == NORMAL
|
||||
})
|
||||
|
||||
awaitAnnouncements(nodes.filterKeys(_ == "A"), 10, 13, 28)
|
||||
}
|
||||
|
||||
test("send an HTLC A->D") {
|
||||
val sender = TestProbe()
|
||||
val amountMsat = 4200000.msat
|
||||
|
@ -826,7 +908,7 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
|||
generateBlocks(bitcoincli, 2)
|
||||
// and we wait for C'channel to close
|
||||
awaitCond(stateListener.expectMsgType[ChannelStateChanged].currentState == CLOSED, max = 30 seconds)
|
||||
awaitAnnouncements(nodes.filterKeys(_ == "A"), 9, 11, 24)
|
||||
awaitAnnouncements(nodes.filterKeys(_ == "A"), 10, 12, 26)
|
||||
}
|
||||
|
||||
def getBlockCount: Long = {
|
||||
|
@ -907,7 +989,7 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
|||
generateBlocks(bitcoincli, 2, Some(address))
|
||||
// and we wait for C'channel to close
|
||||
awaitCond(stateListener.expectMsgType[ChannelStateChanged].currentState == CLOSED, max = 30 seconds)
|
||||
awaitAnnouncements(nodes.filterKeys(_ == "A"), 8, 10, 22)
|
||||
awaitAnnouncements(nodes.filterKeys(_ == "A"), 9, 11, 24)
|
||||
}
|
||||
|
||||
test("propagate a failure upstream when a downstream htlc times out (local commit)") {
|
||||
|
@ -965,7 +1047,7 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
|||
generateBlocks(bitcoincli, 2, Some(address))
|
||||
// and we wait for C'channel to close
|
||||
awaitCond(stateListener.expectMsgType[ChannelStateChanged].currentState == CLOSED, max = 30 seconds)
|
||||
awaitAnnouncements(nodes.filterKeys(_ == "A"), 7, 9, 20)
|
||||
awaitAnnouncements(nodes.filterKeys(_ == "A"), 8, 10, 22)
|
||||
}
|
||||
|
||||
test("propagate a failure upstream when a downstream htlc times out (remote commit)") {
|
||||
|
@ -1027,7 +1109,7 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
|||
generateBlocks(bitcoincli, 2, Some(address))
|
||||
// and we wait for C'channel to close
|
||||
awaitCond(stateListener.expectMsgType[ChannelStateChanged].currentState == CLOSED, max = 30 seconds)
|
||||
awaitAnnouncements(nodes.filterKeys(_ == "A"), 6, 8, 18)
|
||||
awaitAnnouncements(nodes.filterKeys(_ == "A"), 7, 9, 20)
|
||||
}
|
||||
|
||||
test("punish a node that has published a revoked commit tx") {
|
||||
|
@ -1150,7 +1232,7 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
|||
// and we wait for C'channel to close
|
||||
awaitCond(stateListener.expectMsgType[ChannelStateChanged].currentState == CLOSED, max = 30 seconds)
|
||||
// this will remove the channel
|
||||
awaitAnnouncements(nodes.filterKeys(_ == "A"), 5, 7, 16)
|
||||
awaitAnnouncements(nodes.filterKeys(_ == "A"), 6, 8, 18)
|
||||
}
|
||||
|
||||
test("generate and validate lots of channels") {
|
||||
|
@ -1177,7 +1259,7 @@ class IntegrationSpec extends TestKit(ActorSystem("test")) with BitcoindService
|
|||
announcements.foreach(ann => nodes("A").router ! PeerRoutingMessage(sender.ref, remoteNodeId, ann))
|
||||
awaitCond({
|
||||
sender.send(nodes("D").router, 'channels)
|
||||
sender.expectMsgType[Iterable[ChannelAnnouncement]](5 seconds).size == channels.size + 7 // 7 remaining channels because D->F{1-5} have disappeared
|
||||
sender.expectMsgType[Iterable[ChannelAnnouncement]](5 seconds).size == channels.size + 8 // 8 remaining channels because D->F{1-5} have disappeared
|
||||
}, max = 120 seconds, interval = 1 second)
|
||||
}
|
||||
|
||||
|
|
|
@ -19,15 +19,16 @@ package fr.acinq.eclair.io
|
|||
import java.net.{Inet4Address, InetAddress, InetSocketAddress, ServerSocket}
|
||||
|
||||
import akka.actor.FSM.{CurrentState, SubscribeTransitionCallBack, Transition}
|
||||
import akka.actor.Status.Failure
|
||||
import akka.actor.{ActorRef, PoisonPill}
|
||||
import akka.testkit.{TestFSMRef, TestProbe}
|
||||
import fr.acinq.bitcoin.Block
|
||||
import fr.acinq.bitcoin.{Block, Btc, Satoshi}
|
||||
import fr.acinq.bitcoin.Crypto.PublicKey
|
||||
import fr.acinq.eclair.TestConstants._
|
||||
import fr.acinq.eclair._
|
||||
import fr.acinq.eclair.blockchain.{EclairWallet, TestWallet}
|
||||
import fr.acinq.eclair.channel.states.StateTestsHelperMethods
|
||||
import fr.acinq.eclair.channel.{ChannelCreated, HasCommitments}
|
||||
import fr.acinq.eclair.channel.{Channel, ChannelCreated, HasCommitments}
|
||||
import fr.acinq.eclair.crypto.TransportHandler
|
||||
import fr.acinq.eclair.io.Peer._
|
||||
import fr.acinq.eclair.router.RoutingSyncSpec.makeFakeRoutingInfo
|
||||
|
@ -66,6 +67,8 @@ class PeerSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
val aliceParams = TestConstants.Alice.nodeParams
|
||||
.modify(_.syncWhitelist).setToIf(test.tags.contains("sync-whitelist-bob"))(Set(remoteNodeId))
|
||||
.modify(_.syncWhitelist).setToIf(test.tags.contains("sync-whitelist-random"))(Set(randomKey.publicKey))
|
||||
.modify(_.features).setToIf(test.tags.contains("wumbo"))(hex"80000")
|
||||
.modify(_.maxFundingSatoshis).setToIf(test.tags.contains("high-max-funding-satoshis"))(Btc(0.9))
|
||||
|
||||
if (test.tags.contains("with_node_announcements")) {
|
||||
val bobAnnouncement = NodeAnnouncement(randomBytes64, ByteVector.empty, 1, Bob.nodeParams.nodeId, Color(100.toByte, 200.toByte, 300.toByte), "node-alias", fakeIPAddress :: Nil)
|
||||
|
@ -245,7 +248,7 @@ class PeerSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
(bin" 1000101010001010", bin" 0000101010001010"), // option_data_loss_protect, initial_routing_sync, gossip_queries, var_onion_optin, gossip_queries_ex, payment_secret
|
||||
(bin" 0100101010001010", bin" 0000101010001010"), // option_data_loss_protect, initial_routing_sync, gossip_queries, var_onion_optin, gossip_queries_ex, payment_secret
|
||||
(bin"000000101000101010001010", bin" 0000101010001010"), // option_data_loss_protect, initial_routing_sync, gossip_queries, var_onion_optin, gossip_queries_ex, payment_secret, basic_mpp
|
||||
(bin"000010101000101010001010", bin"000010000000101010001010") // option_data_loss_protect, initial_routing_sync, gossip_queries, var_onion_optin, gossip_queries_ex, payment_secret, basic_mpp and 19
|
||||
(bin"000010101000101010001010", bin"000010000000101010001010") // option_data_loss_protect, initial_routing_sync, gossip_queries, var_onion_optin, gossip_queries_ex, payment_secret, basic_mpp and large_channel_support (optional)
|
||||
)
|
||||
|
||||
for ((configuredFeatures, sentFeatures) <- testCases) {
|
||||
|
@ -299,6 +302,49 @@ class PeerSpec extends TestkitBaseClass with StateTestsHelperMethods {
|
|||
probe.expectMsg("disconnecting")
|
||||
}
|
||||
|
||||
test("don't spawn a wumbo channel if wumbo feature isn't enabled") { f =>
|
||||
import f._
|
||||
|
||||
val probe = TestProbe()
|
||||
val fundingAmountBig = Channel.MAX_FUNDING + 10000.sat
|
||||
system.eventStream.subscribe(probe.ref, classOf[ChannelCreated])
|
||||
connect(remoteNodeId, authenticator, watcher, router, relayer, connection, transport, peer)
|
||||
|
||||
assert(peer.stateData.channels.isEmpty)
|
||||
probe.send(peer, Peer.OpenChannel(remoteNodeId, fundingAmountBig, 0 msat, None, None, None))
|
||||
|
||||
assert(probe.expectMsgType[Failure].cause.getMessage == s"fundingSatoshis=$fundingAmountBig is too big, you must enable large channels support in 'eclair.features' to use funding above ${Channel.MAX_FUNDING} (see eclair.conf)")
|
||||
}
|
||||
|
||||
test("don't spawn a wumbo channel if remote doesn't support wumbo", Tag("wumbo")) { f =>
|
||||
import f._
|
||||
|
||||
val probe = TestProbe()
|
||||
val fundingAmountBig = Channel.MAX_FUNDING + 10000.sat
|
||||
system.eventStream.subscribe(probe.ref, classOf[ChannelCreated])
|
||||
connect(remoteNodeId, authenticator, watcher, router, relayer, connection, transport, peer) // Bob doesn't support wumbo, Alice does
|
||||
|
||||
assert(peer.stateData.channels.isEmpty)
|
||||
probe.send(peer, Peer.OpenChannel(remoteNodeId, fundingAmountBig, 0 msat, None, None, None))
|
||||
|
||||
assert(probe.expectMsgType[Failure].cause.getMessage == s"fundingSatoshis=$fundingAmountBig is too big, the remote peer doesn't support wumbo")
|
||||
}
|
||||
|
||||
test("don't spawn a channel if fundingSatoshis is greater than maxFundingSatoshis", Tag("high-max-funding-satoshis"), Tag("wumbo")) { f =>
|
||||
import f._
|
||||
|
||||
val probe = TestProbe()
|
||||
val fundingAmountBig = Btc(1).toSatoshi
|
||||
system.eventStream.subscribe(probe.ref, classOf[ChannelCreated])
|
||||
connect(remoteNodeId, authenticator, watcher, router, relayer, connection, transport, peer, remoteInit = wire.Init(hex"80000")) // Bob supports wumbo
|
||||
|
||||
assert(peer.stateData.channels.isEmpty)
|
||||
probe.send(peer, Peer.OpenChannel(remoteNodeId, fundingAmountBig, 0 msat, None, None, None))
|
||||
|
||||
assert(probe.expectMsgType[Failure].cause.getMessage == s"fundingSatoshis=$fundingAmountBig is too big for the current settings, increase 'eclair.max-funding-satoshis' (see eclair.conf)")
|
||||
}
|
||||
|
||||
|
||||
test("use correct fee rates when spawning a channel") { f =>
|
||||
import f._
|
||||
|
||||
|
|
|
@ -386,9 +386,9 @@ class PaymentRequestSpec extends FunSuite {
|
|||
Features(bin" 00010100001000000000") -> Result(allowMultiPart = true, requirePaymentSecret = true, areSupported = true),
|
||||
Features(bin" 00011000001000000000") -> Result(allowMultiPart = true, requirePaymentSecret = false, areSupported = true),
|
||||
Features(bin" 00101000001000000000") -> Result(allowMultiPart = true, requirePaymentSecret = false, areSupported = true),
|
||||
Features(bin" 01000000000000000000") -> Result(allowMultiPart = false, requirePaymentSecret = false, areSupported = false),
|
||||
Features(bin" 01000000000000000000") -> Result(allowMultiPart = false, requirePaymentSecret = false, areSupported = true),
|
||||
Features(bin" 0000010000000000000000000") -> Result(allowMultiPart = false, requirePaymentSecret = false, areSupported = true),
|
||||
Features(bin" 0000011000000000000000000") -> Result(allowMultiPart = false, requirePaymentSecret = false, areSupported = false),
|
||||
Features(bin" 0000011000000000000000000") -> Result(allowMultiPart = false, requirePaymentSecret = false, areSupported = true),
|
||||
Features(bin" 0000110000000000000000000") -> Result(allowMultiPart = false, requirePaymentSecret = false, areSupported = false),
|
||||
// those are useful for nonreg testing of the areSupported method (which needs to be updated with every new supported mandatory bit)
|
||||
Features(bin" 0000100000000000000000000") -> Result(allowMultiPart = false, requirePaymentSecret = false, areSupported = false),
|
||||
|
|
|
@ -17,7 +17,7 @@
|
|||
package fr.acinq.eclair.router
|
||||
|
||||
import fr.acinq.bitcoin.Crypto.PublicKey
|
||||
import fr.acinq.bitcoin.{Block, ByteVector32, ByteVector64, Satoshi}
|
||||
import fr.acinq.bitcoin.{Block, Btc, ByteVector32, ByteVector64, Satoshi}
|
||||
import fr.acinq.eclair.payment.PaymentRequest.ExtraHop
|
||||
import fr.acinq.eclair.router.Graph.GraphStructure.DirectedGraph.graphEdgeToHop
|
||||
import fr.acinq.eclair.router.Graph.GraphStructure.{DirectedGraph, GraphEdge}
|
||||
|
@ -430,13 +430,13 @@ class RouteCalculationSpec extends FunSuite with ParallelTestExecution {
|
|||
val extraHop4 = ExtraHop(d, ShortChannelId(4), 50.sat.toMilliSatoshi, 0, CltvExpiryDelta(42))
|
||||
val extraHops = extraHop1 :: extraHop2 :: extraHop3 :: extraHop4 :: Nil
|
||||
|
||||
val amount = 900 sat // below RoutingHeuristics.CAPACITY_CHANNEL_LOW
|
||||
val amount = 90000 sat // below RoutingHeuristics.CAPACITY_CHANNEL_LOW
|
||||
val assistedChannels = Router.toAssistedChannels(extraHops, e, amount.toMilliSatoshi)
|
||||
|
||||
assert(assistedChannels(extraHop4.shortChannelId) === AssistedChannel(extraHop4, e, 1050.sat.toMilliSatoshi))
|
||||
assert(assistedChannels(extraHop3.shortChannelId) === AssistedChannel(extraHop3, d, 1200.sat.toMilliSatoshi))
|
||||
assert(assistedChannels(extraHop2.shortChannelId) === AssistedChannel(extraHop2, c, 1400.sat.toMilliSatoshi))
|
||||
assert(assistedChannels(extraHop1.shortChannelId) === AssistedChannel(extraHop1, b, 1426.sat.toMilliSatoshi))
|
||||
assert(assistedChannels(extraHop4.shortChannelId) === AssistedChannel(extraHop4, e, 100050.sat.toMilliSatoshi))
|
||||
assert(assistedChannels(extraHop3.shortChannelId) === AssistedChannel(extraHop3, d, 100200.sat.toMilliSatoshi))
|
||||
assert(assistedChannels(extraHop2.shortChannelId) === AssistedChannel(extraHop2, c, 100400.sat.toMilliSatoshi))
|
||||
assert(assistedChannels(extraHop1.shortChannelId) === AssistedChannel(extraHop1, b, 101416.sat.toMilliSatoshi))
|
||||
}
|
||||
|
||||
test("blacklist routes") {
|
||||
|
|
|
@ -90,8 +90,6 @@ class OpenChannelController(val handlers: Handlers, val stage: Stage) extends Lo
|
|||
fundingSatError.setText("Capacity must be greater than 0")
|
||||
case Success(capacitySat) if capacitySat < 50000.sat =>
|
||||
fundingSatError.setText("Capacity is low and the channel may not be able to open")
|
||||
case Success(capacitySat) if capacitySat >= Channel.MAX_FUNDING =>
|
||||
fundingSatError.setText(s"Capacity must be less than ${CoinUtils.formatAmountInUnit(Channel.MAX_FUNDING, FxApp.getUnit, withUnit = true)}")
|
||||
case Success(_) => fundingSatError.setText("")
|
||||
case _ => fundingSatError.setText("Capacity is not valid")
|
||||
}
|
||||
|
@ -111,8 +109,6 @@ class OpenChannelController(val handlers: Handlers, val stage: Stage) extends Lo
|
|||
Try(if (Strings.isNullOrEmpty(feerateField.getText())) None else Some(feerateField.getText().toLong))) match {
|
||||
case (Success(capacitySat), _, _) if capacitySat <= 0.sat =>
|
||||
fundingSatError.setText("Capacity must be greater than 0")
|
||||
case (Success(capacitySat), _, _) if capacitySat >= Channel.MAX_FUNDING =>
|
||||
fundingSatError.setText(s"Capacity must be less than ${CoinUtils.formatAmountInUnit(Channel.MAX_FUNDING, FxApp.getUnit, withUnit = true)}")
|
||||
case (Success(capacitySat), Success(pushMsat), _) if pushMsat > capacitySat.toMilliSatoshi.toLong =>
|
||||
pushMsatError.setText("Push must be less or equal to capacity")
|
||||
case (Success(_), Success(pushMsat), _) if pushMsat < 0 =>
|
||||
|
|
Loading…
Add table
Reference in a new issue