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

Move channel parameters to their own conf section (#2149)

For consistency with existing sections: `router`, `relay`, `peer-connection`, etc.
This commit is contained in:
Pierre-Marie Padiou 2022-01-27 15:16:26 +01:00 committed by GitHub
parent 57c2cc5df9
commit ffecd62cc1
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
36 changed files with 324 additions and 295 deletions

View file

@ -16,8 +16,6 @@ eclair {
password = "" // password for basic auth, must be non empty if json-rpc api is enabled
}
watch-spent-window = 1 minute // at startup watches will be put back within that window to reduce herd effect; must be > 0s
bitcoind {
host = "localhost"
rpcport = 8332
@ -74,45 +72,49 @@ eclair {
channel-flags {
announce-channel = true
}
dust-limit-satoshis = 546
max-remote-dust-limit-satoshis = 600
htlc-minimum-msat = 1
// The following parameters apply to each HTLC direction (incoming or outgoing), which means that the total HTLC limits will be twice what is set here
max-htlc-value-in-flight-msat = 5000000000 // 50 mBTC
max-accepted-htlcs = 30
reserve-to-funding-ratio = 0.01 // recommended by BOLT #2
max-reserve-to-funding-ratio = 0.05 // channel reserve can't be more than 5% of the funding amount (recommended: 1%)
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'
to-remote-delay-blocks = 720 // number of blocks that the other node's to-self outputs must be delayed (720 ~ 5 days)
max-to-local-delay-blocks = 2016 // maximum number of blocks that we are ready to accept for our own delayed outputs (2016 ~ 2 weeks)
mindepth-blocks = 3
expiry-delta-blocks = 144
// When we receive the preimage for an HTLC and want to fulfill it but the upstream peer stops responding, we want to
// avoid letting its HTLC-timeout transaction become enforceable on-chain (otherwise there is a race condition between
// our HTLC-success and their HTLC-timeout).
// We will close the channel when the HTLC-timeout will happen in less than this number.
// NB: this number effectively reduces the expiry-delta-blocks, so you may want to take that into account and increase
// expiry-delta-blocks.
fulfill-safety-before-timeout-blocks = 24
min-final-expiry-delta-blocks = 30 // Bolt 11 invoice's min_final_cltv_expiry; must be strictly greater than fulfill-safety-before-timeout-blocks
max-block-processing-delay = 30 seconds // we add a random delay before processing blocks, capped at this value, to prevent herd effect
max-tx-publish-retry-delay = 60 seconds // we add a random delay before retrying failed transaction publication
// The default strategy, when we encounter an unhandled exception or internal error, is to locally force-close the
// channel. Not only is there a delay before the channel balance gets refunded, but if the exception was due to some
// misconfiguration or bug in eclair that affects all channels, we risk force-closing all channels.
// This is why an alternative behavior is to simply log an error and stop the node. Note that if you don't closely
// monitor your node, there is a risk that your peers take advantage of the downtime to try and cheat by publishing a
// revoked commitment. Additionally, while there is no known way of triggering an internal error in eclair from the
// outside, there may very well be a bug that allows just that, which could be used as a way to remotely stop the node
// (with the default behavior, it would "only" cause a local force-close of the channel).
unhandled-exception-strategy = "local-close" // local-close or stop
revocation-timeout = 20 seconds // after sending a commit_sig, we will wait for at most that duration before disconnecting
}
dust-limit-satoshis = 546
max-remote-dust-limit-satoshis = 600
htlc-minimum-msat = 1
// The following parameters apply to each HTLC direction (incoming or outgoing), which means that the total HTLC limits will be twice what is set here
max-htlc-value-in-flight-msat = 5000000000 // 50 mBTC
max-accepted-htlcs = 30
reserve-to-funding-ratio = 0.01 // recommended by BOLT #2
max-reserve-to-funding-ratio = 0.05 // channel reserve can't be more than 5% of the funding amount (recommended: 1%)
balance-check-interval = 1 hour
to-remote-delay-blocks = 720 // number of blocks that the other node's to-self outputs must be delayed (720 ~ 5 days)
max-to-local-delay-blocks = 2016 // maximum number of blocks that we are ready to accept for our own delayed outputs (2016 ~ 2 weeks)
mindepth-blocks = 3
expiry-delta-blocks = 144
// When we receive the preimage for an HTLC and want to fulfill it but the upstream peer stops responding, we want to
// avoid letting its HTLC-timeout transaction become enforceable on-chain (otherwise there is a race condition between
// our HTLC-success and their HTLC-timeout).
// We will close the channel when the HTLC-timeout will happen in less than this number.
// NB: this number effectively reduces the expiry-delta-blocks, so you may want to take that into account and increase
// expiry-delta-blocks.
fulfill-safety-before-timeout-blocks = 24
min-final-expiry-delta-blocks = 30 // Bolt 11 invoice's min_final_cltv_expiry; must be strictly greater than fulfill-safety-before-timeout-blocks
max-block-processing-delay = 30 seconds // we add a random delay before processing blocks, capped at this value, to prevent herd effect
max-tx-publish-retry-delay = 60 seconds // we add a random delay before retrying failed transaction publication
// The default strategy, when we encounter an unhandled exception or internal error, is to locally force-close the
// channel. Not only is there a delay before the channel balance gets refunded, but if the exception was due to some
// misconfiguration or bug in eclair that affects all channels, we risk force-closing all channels.
// This is why an alternative behavior is to simply log an error and stop the node. Note that if you don't closely
// monitor your node, there is a risk that your peers take advantage of the downtime to try and cheat by publishing a
// revoked commitment. Additionally, while there is no known way of triggering an internal error in eclair from the
// outside, there may very well be a bug that allows just that, which could be used as a way to remotely stop the node
// (with the default behavior, it would "only" cause a local force-close of the channel).
unhandled-exception-strategy = "local-close" // local-close or stop
relay {
fees {
// Fees for public channels
@ -197,8 +199,6 @@ eclair {
update-fee-min-diff-ratio = 0.1
}
revocation-timeout = 20 seconds // after sending a commit_sig, we will wait for at most that duration before disconnecting
peer-connection {
auth-timeout = 15 seconds // will disconnect if connection authentication doesn't happen within that timeframe
init-timeout = 15 seconds // will disconnect if initialization doesn't happen within that timeframe
@ -213,13 +213,13 @@ 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
router {
watch-spent-window = 1 minute // at startup watches will be put back within that window to reduce herd effect; must be > 0s
channel-exclude-duration = 60 seconds // when a temporary channel failure is returned, we exclude the channel from our payment routes for this duration
broadcast-interval = 60 seconds // see BOLT #7
init-timeout = 5 minutes

View file

@ -22,7 +22,7 @@ import fr.acinq.bitcoin.{Block, ByteVector32, Crypto, Satoshi}
import fr.acinq.eclair.Setup.Seeds
import fr.acinq.eclair.blockchain.fee._
import fr.acinq.eclair.channel.{Channel, ChannelFlags}
import fr.acinq.eclair.channel.Channel.UnhandledExceptionStrategy
import fr.acinq.eclair.channel.Channel.{ChannelConf, UnhandledExceptionStrategy}
import fr.acinq.eclair.crypto.Noise.KeyPair
import fr.acinq.eclair.crypto.keymanager.{ChannelKeyManager, NodeKeyManager}
import fr.acinq.eclair.db._
@ -62,36 +62,16 @@ case class NodeParams(nodeKeyManager: NodeKeyManager,
private val overrideFeatures: Map[PublicKey, Features],
syncWhitelist: Set[PublicKey],
pluginParams: Seq[PluginParams],
dustLimit: Satoshi,
maxRemoteDustLimit: Satoshi,
channelConf: ChannelConf,
onChainFeeConf: OnChainFeeConf,
maxHtlcValueInFlightMsat: UInt64,
maxAcceptedHtlcs: Int,
expiryDelta: CltvExpiryDelta,
fulfillSafetyBeforeTimeout: CltvExpiryDelta,
minFinalExpiryDelta: CltvExpiryDelta,
maxBlockProcessingDelay: FiniteDuration,
maxTxPublishRetryDelay: FiniteDuration,
htlcMinimum: MilliSatoshi,
toRemoteDelay: CltvExpiryDelta,
maxToLocalDelay: CltvExpiryDelta,
minDepthBlocks: Int,
relayParams: RelayParams,
reserveToFundingRatio: Double,
maxReserveToFundingRatio: Double,
unhandledExceptionStrategy: UnhandledExceptionStrategy,
db: Databases,
revocationTimeout: FiniteDuration,
autoReconnect: Boolean,
initialRandomReconnectDelay: FiniteDuration,
maxReconnectInterval: FiniteDuration,
chainHash: ByteVector32,
channelFlags: ChannelFlags,
watchSpentWindow: FiniteDuration,
paymentRequestExpiry: FiniteDuration,
multiPartPaymentExpiry: FiniteDuration,
minFundingSatoshis: Satoshi,
maxFundingSatoshis: Satoshi,
peerConnectionConf: PeerConnection.Conf,
routerConf: RouterConf,
socksProxy_opt: Option[Socks5ProxyParams],
@ -222,6 +202,26 @@ object NodeParams extends Logging {
"router.path-finding.hop-cost-millionths" -> "router.path-finding.default.hop-cost.fee-proportional-millionths",
// v0.6.3
"channel-flags" -> "channel.channel-flags",
"dust-limit-satoshis" -> "channel.dust-limit-satoshis",
"max-remote-dust-limit-satoshis" -> "channel.max-remote-dust-limit-satoshis",
"htlc-minimum-msat" -> "channel.htlc-minimum-msat",
"max-htlc-value-in-flight-msat" -> "channel.max-htlc-value-in-flight-msat",
"max-accepted-htlcs" -> "channel.max-accepted-htlcs",
"reserve-to-funding-ratio" -> "channel.reserve-to-funding-ratio",
"max-reserve-to-funding-ratio" -> "channel.max-reserve-to-funding-ratio",
"min-funding-satoshis" -> "channel.min-funding-satoshis",
"max-funding-satoshis" -> "channel.max-funding-satoshis",
"to-remote-delay-blocks" -> "channel.to-remote-delay-blocks",
"max-to-local-delay-blocks" -> "channel.max-to-local-delay-blocks",
"mindepth-blocks" -> "channel.mindepth-blocks",
"expiry-delta-blocks" -> "channel.expiry-delta-blocks",
"fulfill-safety-before-timeout-blocks" -> "channel.fulfill-safety-before-timeout-blocks",
"min-final-expiry-delta-blocks" -> "channel.min-final-expiry-delta-blocks",
"max-block-processing-delay" -> "channel.max-block-processing-delay",
"max-tx-publish-retry-delay" -> "channel.max-tx-publish-retry-delay",
"unhandled-exception-strategy" -> "channel.unhandled-exception-strategy",
"revocation-timeout" -> "channel.revocation-timeout",
"watch-spent-window" -> "router.watch-spent-window",
)
deprecatedKeyPaths.foreach {
case (old, new_) => require(!config.hasPath(old), s"configuration key '$old' has been replaced by '$new_'")
@ -239,29 +239,29 @@ object NodeParams extends Logging {
val color = ByteVector.fromValidHex(config.getString("node-color"))
require(color.size == 3, "color should be a 3-bytes hex buffer")
val watchSpentWindow = FiniteDuration(config.getDuration("watch-spent-window").getSeconds, TimeUnit.SECONDS)
require(watchSpentWindow > 0.seconds, "watch-spent-window must be strictly greater than 0")
val watchSpentWindow = FiniteDuration(config.getDuration("router.watch-spent-window").getSeconds, TimeUnit.SECONDS)
require(watchSpentWindow > 0.seconds, "router.watch-spent-window must be strictly greater than 0")
val dustLimitSatoshis = Satoshi(config.getLong("dust-limit-satoshis"))
val dustLimitSatoshis = Satoshi(config.getLong("channel.dust-limit-satoshis"))
if (chainHash == Block.LivenetGenesisBlock.hash) {
require(dustLimitSatoshis >= Channel.MIN_DUST_LIMIT, s"dust limit must be greater than ${Channel.MIN_DUST_LIMIT}")
}
val htlcMinimum = MilliSatoshi(config.getInt("htlc-minimum-msat"))
require(htlcMinimum > 0.msat, "htlc-minimum-msat must be strictly greater than 0")
val htlcMinimum = MilliSatoshi(config.getInt("channel.htlc-minimum-msat"))
require(htlcMinimum > 0.msat, "channel.htlc-minimum-msat must be strictly greater than 0")
val maxAcceptedHtlcs = config.getInt("max-accepted-htlcs")
require(maxAcceptedHtlcs <= Channel.MAX_ACCEPTED_HTLCS, s"max-accepted-htlcs must be lower than ${Channel.MAX_ACCEPTED_HTLCS}")
val maxAcceptedHtlcs = config.getInt("channel.max-accepted-htlcs")
require(maxAcceptedHtlcs <= Channel.MAX_ACCEPTED_HTLCS, s"channel.max-accepted-htlcs must be lower than ${Channel.MAX_ACCEPTED_HTLCS}")
val maxToLocalCLTV = CltvExpiryDelta(config.getInt("max-to-local-delay-blocks"))
val offeredCLTV = CltvExpiryDelta(config.getInt("to-remote-delay-blocks"))
val maxToLocalCLTV = CltvExpiryDelta(config.getInt("channel.max-to-local-delay-blocks"))
val offeredCLTV = CltvExpiryDelta(config.getInt("channel.to-remote-delay-blocks"))
require(maxToLocalCLTV <= Channel.MAX_TO_SELF_DELAY && offeredCLTV <= Channel.MAX_TO_SELF_DELAY, s"CLTV delay values too high, max is ${Channel.MAX_TO_SELF_DELAY}")
val expiryDelta = CltvExpiryDelta(config.getInt("expiry-delta-blocks"))
val fulfillSafetyBeforeTimeout = CltvExpiryDelta(config.getInt("fulfill-safety-before-timeout-blocks"))
require(fulfillSafetyBeforeTimeout * 2 < expiryDelta, "fulfill-safety-before-timeout-blocks must be smaller than expiry-delta-blocks / 2 because it effectively reduces that delta; if you want to increase this value, you may want to increase expiry-delta-blocks as well")
val minFinalExpiryDelta = CltvExpiryDelta(config.getInt("min-final-expiry-delta-blocks"))
require(minFinalExpiryDelta > fulfillSafetyBeforeTimeout, "min-final-expiry-delta-blocks must be strictly greater than fulfill-safety-before-timeout-blocks; otherwise it may lead to undesired channel closure")
val expiryDelta = CltvExpiryDelta(config.getInt("channel.expiry-delta-blocks"))
val fulfillSafetyBeforeTimeout = CltvExpiryDelta(config.getInt("channel.fulfill-safety-before-timeout-blocks"))
require(fulfillSafetyBeforeTimeout * 2 < expiryDelta, "channel.fulfill-safety-before-timeout-blocks must be smaller than channel.expiry-delta-blocks / 2 because it effectively reduces that delta; if you want to increase this value, you may want to increase expiry-delta-blocks as well")
val minFinalExpiryDelta = CltvExpiryDelta(config.getInt("channel.min-final-expiry-delta-blocks"))
require(minFinalExpiryDelta > fulfillSafetyBeforeTimeout, "channel.min-final-expiry-delta-blocks must be strictly greater than channel.fulfill-safety-before-timeout-blocks; otherwise it may lead to undesired channel closure")
val nodeAlias = config.getString("node-alias")
require(nodeAlias.getBytes("UTF-8").length <= 32, "invalid alias, too long (max allowed 32 bytes)")
@ -369,7 +369,7 @@ object NodeParams extends Logging {
PathFindingExperimentConf(experiments.toMap)
}
val unhandledExceptionStrategy = config.getString("unhandled-exception-strategy") match {
val unhandledExceptionStrategy = config.getString("channel.unhandled-exception-strategy") match {
case "local-close" => UnhandledExceptionStrategy.LocalClose
case "stop" => UnhandledExceptionStrategy.Stop
}
@ -398,8 +398,28 @@ object NodeParams extends Logging {
pluginParams = pluginParams,
overrideFeatures = overrideFeatures,
syncWhitelist = syncWhitelist,
dustLimit = dustLimitSatoshis,
maxRemoteDustLimit = Satoshi(config.getLong("max-remote-dust-limit-satoshis")),
channelConf = ChannelConf(
channelFlags = channelFlags,
dustLimit = dustLimitSatoshis,
maxRemoteDustLimit = Satoshi(config.getLong("channel.max-remote-dust-limit-satoshis")),
htlcMinimum = htlcMinimum,
maxHtlcValueInFlightMsat = UInt64(config.getLong("channel.max-htlc-value-in-flight-msat")),
maxAcceptedHtlcs = maxAcceptedHtlcs,
reserveToFundingRatio = config.getDouble("channel.reserve-to-funding-ratio"),
maxReserveToFundingRatio = config.getDouble("channel.max-reserve-to-funding-ratio"),
minFundingSatoshis = Satoshi(config.getLong("channel.min-funding-satoshis")),
maxFundingSatoshis = Satoshi(config.getLong("channel.max-funding-satoshis")),
toRemoteDelay = offeredCLTV,
maxToLocalDelay = maxToLocalCLTV,
minDepthBlocks = config.getInt("channel.mindepth-blocks"),
expiryDelta = expiryDelta,
fulfillSafetyBeforeTimeout = fulfillSafetyBeforeTimeout,
minFinalExpiryDelta = minFinalExpiryDelta,
maxBlockProcessingDelay = FiniteDuration(config.getDuration("channel.max-block-processing-delay").getSeconds, TimeUnit.SECONDS),
maxTxPublishRetryDelay = FiniteDuration(config.getDuration("channel.max-tx-publish-retry-delay").getSeconds, TimeUnit.SECONDS),
unhandledExceptionStrategy = unhandledExceptionStrategy,
revocationTimeout = FiniteDuration(config.getDuration("channel.revocation-timeout").getSeconds, TimeUnit.SECONDS)
),
onChainFeeConf = OnChainFeeConf(
feeTargets = feeTargets,
feeEstimator = feeEstimator,
@ -428,37 +448,18 @@ object NodeParams extends Logging {
nodeId -> tolerance
}.toMap
),
maxHtlcValueInFlightMsat = UInt64(config.getLong("max-htlc-value-in-flight-msat")),
maxAcceptedHtlcs = maxAcceptedHtlcs,
expiryDelta = expiryDelta,
fulfillSafetyBeforeTimeout = fulfillSafetyBeforeTimeout,
minFinalExpiryDelta = minFinalExpiryDelta,
maxBlockProcessingDelay = FiniteDuration(config.getDuration("max-block-processing-delay").getSeconds, TimeUnit.SECONDS),
maxTxPublishRetryDelay = FiniteDuration(config.getDuration("max-tx-publish-retry-delay").getSeconds, TimeUnit.SECONDS),
htlcMinimum = htlcMinimum,
toRemoteDelay = CltvExpiryDelta(config.getInt("to-remote-delay-blocks")),
maxToLocalDelay = CltvExpiryDelta(config.getInt("max-to-local-delay-blocks")),
minDepthBlocks = config.getInt("mindepth-blocks"),
relayParams = RelayParams(
publicChannelFees = getRelayFees(config.getConfig("relay.fees.public-channels")),
privateChannelFees = getRelayFees(config.getConfig("relay.fees.private-channels")),
minTrampolineFees = getRelayFees(config.getConfig("relay.fees.min-trampoline")),
),
reserveToFundingRatio = config.getDouble("reserve-to-funding-ratio"),
maxReserveToFundingRatio = config.getDouble("max-reserve-to-funding-ratio"),
unhandledExceptionStrategy = unhandledExceptionStrategy,
db = database,
revocationTimeout = FiniteDuration(config.getDuration("revocation-timeout").getSeconds, TimeUnit.SECONDS),
autoReconnect = config.getBoolean("auto-reconnect"),
initialRandomReconnectDelay = FiniteDuration(config.getDuration("initial-random-reconnect-delay").getSeconds, TimeUnit.SECONDS),
maxReconnectInterval = FiniteDuration(config.getDuration("max-reconnect-interval").getSeconds, TimeUnit.SECONDS),
chainHash = chainHash,
channelFlags = channelFlags,
watchSpentWindow = watchSpentWindow,
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")),
peerConnectionConf = PeerConnection.Conf(
authTimeout = FiniteDuration(config.getDuration("peer-connection.auth-timeout").getSeconds, TimeUnit.SECONDS),
initTimeout = FiniteDuration(config.getDuration("peer-connection.init-timeout").getSeconds, TimeUnit.SECONDS),
@ -470,6 +471,7 @@ object NodeParams extends Logging {
maxOnionMessagesPerSecond = config.getInt("onion-messages.max-per-peer-per-second")
),
routerConf = RouterConf(
watchSpentWindow = watchSpentWindow,
channelExcludeDuration = FiniteDuration(config.getDuration("router.channel-exclude-duration").getSeconds, TimeUnit.SECONDS),
routerBroadcastInterval = FiniteDuration(config.getDuration("router.broadcast-interval").getSeconds, TimeUnit.SECONDS),
requestNodeAnnouncements = config.getBoolean("router.sync.request-node-announcements"),

View file

@ -63,6 +63,27 @@ import scala.util.{Failure, Random, Success, Try}
object Channel {
case class ChannelConf(channelFlags: ChannelFlags,
dustLimit: Satoshi,
maxRemoteDustLimit: Satoshi,
htlcMinimum: MilliSatoshi,
maxHtlcValueInFlightMsat: UInt64,
maxAcceptedHtlcs: Int,
reserveToFundingRatio: Double,
maxReserveToFundingRatio: Double,
minFundingSatoshis: Satoshi,
maxFundingSatoshis: Satoshi,
toRemoteDelay: CltvExpiryDelta,
maxToLocalDelay: CltvExpiryDelta,
minDepthBlocks: Int,
expiryDelta: CltvExpiryDelta,
fulfillSafetyBeforeTimeout: CltvExpiryDelta,
minFinalExpiryDelta: CltvExpiryDelta,
maxBlockProcessingDelay: FiniteDuration,
maxTxPublishRetryDelay: FiniteDuration,
unhandledExceptionStrategy: UnhandledExceptionStrategy,
revocationTimeout: FiniteDuration)
trait TxPublisherFactory {
def spawnTxPublisher(context: ActorContext, remoteNodeId: PublicKey): typed.ActorRef[TxPublisher.Command]
}
@ -164,7 +185,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, remo
// this will be used to detect htlc timeouts
context.system.eventStream.subscribe(self, classOf[CurrentBlockHeight])
// the constant delay by which we delay processing of blocks (it will be smoothened among all channels)
private val blockProcessingDelay = Random.nextLong(nodeParams.maxBlockProcessingDelay.toMillis + 1).millis
private val blockProcessingDelay = Random.nextLong(nodeParams.channelConf.maxBlockProcessingDelay.toMillis + 1).millis
// this will be used to make sure the current commitment fee is up-to-date
context.system.eventStream.subscribe(self, classOf[CurrentFeerates])
@ -303,9 +324,9 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, remo
val fees = getRelayFees(nodeParams, remoteNodeId, data.commitments)
if (fees.feeBase != normal.channelUpdate.feeBaseMsat ||
fees.feeProportionalMillionths != normal.channelUpdate.feeProportionalMillionths ||
nodeParams.expiryDelta != normal.channelUpdate.cltvExpiryDelta) {
nodeParams.channelConf.expiryDelta != normal.channelUpdate.cltvExpiryDelta) {
log.info("refreshing channel_update due to configuration changes")
self ! CMD_UPDATE_RELAY_FEE(ActorRef.noSender, fees.feeBase, fees.feeProportionalMillionths, Some(nodeParams.expiryDelta))
self ! CMD_UPDATE_RELAY_FEE(ActorRef.noSender, fees.feeBase, fees.feeProportionalMillionths, Some(nodeParams.channelConf.expiryDelta))
}
// we need to periodically re-send channel updates, otherwise channel will be considered stale and get pruned by network
// we take into account the date of the last update so that we don't send superfluous updates when we restart the app
@ -347,7 +368,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, remo
context.system.eventStream.publish(ChannelCreated(self, peer, remoteNodeId, isFunder = false, open.temporaryChannelId, open.feeratePerKw, None))
val fundingPubkey = keyManager.fundingPublicKey(localParams.fundingKeyPath).publicKey
val channelKeyPath = keyManager.keyPath(localParams, channelConfig)
val minimumDepth = Helpers.minDepthForFunding(nodeParams, open.fundingSatoshis)
val minimumDepth = Helpers.minDepthForFunding(nodeParams.channelConf, open.fundingSatoshis)
// In order to allow TLV extensions and keep backwards-compatibility, we include an empty upfront_shutdown_script if this feature is not used.
// See https://github.com/lightningnetwork/lightning-rfc/pull/714.
val localShutdownScript = if (Features.canUseFeature(localParams.initFeatures, remoteInit.features, Features.UpfrontShutdownScript)) localParams.defaultFinalScriptPubKey else ByteVector.empty
@ -521,7 +542,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, remo
// 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}")
watchFundingTx(commitments)
val fundingMinDepth = Helpers.minDepthForFunding(nodeParams, fundingAmount)
val fundingMinDepth = Helpers.minDepthForFunding(nodeParams.channelConf, fundingAmount)
blockchain ! WatchFundingConfirmed(self, commitInput.outPoint.txid, fundingMinDepth)
goto(WAIT_FOR_FUNDING_CONFIRMED) using DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments, None, nodeParams.currentBlockHeight, None, Right(fundingSigned)) storing() sending fundingSigned
}
@ -561,7 +582,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, remo
context.system.eventStream.publish(ChannelSignatureReceived(self, commitments))
log.info(s"publishing funding tx for channelId=$channelId fundingTxid=${commitInput.outPoint.txid}")
watchFundingTx(commitments)
blockchain ! WatchFundingConfirmed(self, commitInput.outPoint.txid, nodeParams.minDepthBlocks)
blockchain ! WatchFundingConfirmed(self, commitInput.outPoint.txid, nodeParams.channelConf.minDepthBlocks)
log.info(s"committing txid=${fundingTx.txid}")
// we will publish the funding tx only after the channel state has been written to disk because we want to
@ -617,7 +638,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, remo
Try(Transaction.correctlySpends(commitments.fullySignedLocalCommitTx(keyManager).tx, Seq(fundingTx), ScriptFlags.STANDARD_SCRIPT_VERIFY_FLAGS)) match {
case Success(_) =>
log.info(s"channelId=${commitments.channelId} was confirmed at blockHeight=$blockHeight txIndex=$txIndex")
blockchain ! WatchFundingLost(self, commitments.commitInput.outPoint.txid, nodeParams.minDepthBlocks)
blockchain ! WatchFundingLost(self, commitments.commitInput.outPoint.txid, nodeParams.channelConf.minDepthBlocks)
if (!d.commitments.localParams.isFunder) context.system.eventStream.publish(TransactionPublished(commitments.channelId, remoteNodeId, fundingTx, 0 sat, "funding"))
context.system.eventStream.publish(TransactionConfirmed(commitments.channelId, remoteNodeId, fundingTx))
val channelKeyPath = keyManager.keyPath(d.commitments.localParams, commitments.channelConfig)
@ -670,7 +691,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, remo
context.system.eventStream.publish(ShortChannelIdAssigned(self, commitments.channelId, shortChannelId, None))
// we create a channel_update early so that we can use it to send payments through this channel, but it won't be propagated to other nodes since the channel is not yet announced
val fees = getRelayFees(nodeParams, remoteNodeId, commitments)
val initialChannelUpdate = Announcements.makeChannelUpdate(nodeParams.chainHash, nodeParams.privateKey, remoteNodeId, shortChannelId, nodeParams.expiryDelta, d.commitments.remoteParams.htlcMinimum, fees.feeBase, fees.feeProportionalMillionths, commitments.capacity.toMilliSatoshi, enable = Helpers.aboveReserve(d.commitments))
val initialChannelUpdate = Announcements.makeChannelUpdate(nodeParams.chainHash, nodeParams.privateKey, remoteNodeId, shortChannelId, nodeParams.channelConf.expiryDelta, d.commitments.remoteParams.htlcMinimum, fees.feeBase, fees.feeProportionalMillionths, commitments.capacity.toMilliSatoshi, enable = Helpers.aboveReserve(d.commitments))
// we need to periodically re-send channel updates, otherwise channel will be considered stale and get pruned by network
context.system.scheduler.scheduleWithFixedDelay(initialDelay = REFRESH_CHANNEL_UPDATE_INTERVAL, delay = REFRESH_CHANNEL_UPDATE_INTERVAL, receiver = self, message = BroadcastChannelUpdate(PeriodicRefresh))
goto(NORMAL) using DATA_NORMAL(commitments.copy(remoteNextCommitInfo = Right(nextPerCommitmentPoint)), shortChannelId, buried = false, None, initialChannelUpdate, None, None, None) storing()
@ -817,7 +838,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, remo
}
context.system.eventStream.publish(ChannelSignatureSent(self, commitments1))
// we expect a quick response from our peer
startSingleTimer(RevocationTimeout.toString, RevocationTimeout(commitments1.remoteCommit.index, peer), nodeParams.revocationTimeout)
startSingleTimer(RevocationTimeout.toString, RevocationTimeout(commitments1.remoteCommit.index, peer), nodeParams.channelConf.revocationTimeout)
handleCommandSuccess(c, d.copy(commitments = commitments1)).storing().sending(commit).acking(commitments1.localChanges.signed)
case Left(cause) => handleCommandError(cause, c)
}
@ -1174,7 +1195,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, remo
}
context.system.eventStream.publish(ChannelSignatureSent(self, commitments1))
// we expect a quick response from our peer
startSingleTimer(RevocationTimeout.toString, RevocationTimeout(commitments1.remoteCommit.index, peer), nodeParams.revocationTimeout)
startSingleTimer(RevocationTimeout.toString, RevocationTimeout(commitments1.remoteCommit.index, peer), nodeParams.channelConf.revocationTimeout)
handleCommandSuccess(c, d.copy(commitments = commitments1)).storing().sending(commit).acking(commitments1.localChanges.signed)
case Left(cause) => handleCommandError(cause, c)
}
@ -1467,7 +1488,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, remo
case Event(WatchOutputSpentTriggered(tx), d: DATA_CLOSING) =>
// one of the outputs of the local/remote/revoked commit was spent
// we just put a watch to be notified when it is confirmed
blockchain ! WatchTxConfirmed(self, tx.txid, nodeParams.minDepthBlocks)
blockchain ! WatchTxConfirmed(self, tx.txid, nodeParams.channelConf.minDepthBlocks)
// when a remote or local commitment tx containing outgoing htlcs is published on the network,
// we watch it in order to extract payment preimage if funds are pulled by the counterparty
// we can then use these preimages to fulfill origin htlcs
@ -1502,7 +1523,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, remo
val (localCommitPublished1, claimHtlcTx_opt) = Closing.claimLocalCommitHtlcTxOutput(localCommitPublished, keyManager, d.commitments, tx, nodeParams.onChainFeeConf.feeEstimator, nodeParams.onChainFeeConf.feeTargets)
claimHtlcTx_opt.foreach(claimHtlcTx => {
txPublisher ! PublishFinalTx(claimHtlcTx, claimHtlcTx.fee, None)
blockchain ! WatchTxConfirmed(self, claimHtlcTx.tx.txid, nodeParams.minDepthBlocks)
blockchain ! WatchTxConfirmed(self, claimHtlcTx.tx.txid, nodeParams.channelConf.minDepthBlocks)
})
Closing.updateLocalCommitPublished(localCommitPublished1, tx)
}),
@ -1662,10 +1683,10 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, remo
when(SYNCING)(handleExceptions {
case Event(_: ChannelReestablish, d: DATA_WAIT_FOR_FUNDING_CONFIRMED) =>
val minDepth = if (d.commitments.localParams.isFunder) {
nodeParams.minDepthBlocks
nodeParams.channelConf.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)
Helpers.minDepthForFunding(nodeParams.channelConf, 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 ! WatchFundingConfirmed(self, d.commitments.commitInput.outPoint.txid, minDepth)
@ -1704,7 +1725,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, remo
commitments1.remoteNextCommitInfo match {
case Left(_) =>
// we expect them to (re-)send the revocation immediately
startSingleTimer(RevocationTimeout.toString, RevocationTimeout(commitments1.remoteCommit.index, peer), nodeParams.revocationTimeout)
startSingleTimer(RevocationTimeout.toString, RevocationTimeout(commitments1.remoteCommit.index, peer), nodeParams.channelConf.revocationTimeout)
case _ => ()
}
@ -2145,7 +2166,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, remo
val knownSpendingTxs = Set(commitments.localCommit.commitTxAndRemoteSig.commitTx.tx.txid, commitments.remoteCommit.txid) ++ commitments.remoteNextCommitInfo.left.toSeq.map(_.nextRemoteCommit.txid).toSet ++ additionalKnownSpendingTxs
blockchain ! WatchFundingSpent(self, commitments.commitInput.outPoint.txid, commitments.commitInput.outPoint.index.toInt, knownSpendingTxs)
// TODO: implement this? (not needed if we use a reasonable min_depth)
//blockchain ! WatchLost(self, commitments.commitInput.outPoint.txid, nodeParams.minDepthBlocks, BITCOIN_FUNDING_LOST)
//blockchain ! WatchLost(self, commitments.commitInput.outPoint.txid, nodeParams.channelConf.minDepthBlocks, BITCOIN_FUNDING_LOST)
}
/**
@ -2281,7 +2302,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, remo
private def handleNewBlock(c: CurrentBlockHeight, d: HasCommitments) = {
val timedOutOutgoing = d.commitments.timedOutOutgoingHtlcs(c.blockHeight)
val almostTimedOutIncoming = d.commitments.almostTimedOutIncomingHtlcs(c.blockHeight, nodeParams.fulfillSafetyBeforeTimeout)
val almostTimedOutIncoming = d.commitments.almostTimedOutIncomingHtlcs(c.blockHeight, nodeParams.channelConf.fulfillSafetyBeforeTimeout)
if (timedOutOutgoing.nonEmpty) {
// Downstream timed out.
handleLocalError(HtlcsTimedoutDownstream(d.channelId, timedOutOutgoing), d, Some(c))
@ -2338,7 +2359,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, remo
spendLocalCurrent(dd) sending error
case _ =>
// unhandled exception: we apply the configured strategy
nodeParams.unhandledExceptionStrategy match {
nodeParams.channelConf.unhandledExceptionStrategy match {
case UnhandledExceptionStrategy.LocalClose =>
spendLocalCurrent(dd) sending error
case UnhandledExceptionStrategy.Stop =>
@ -2399,7 +2420,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, remo
// the funder pays the fee
val fee = if (isFunder) closingTx.fee else 0.sat
txPublisher ! PublishFinalTx(closingTx, fee, None)
blockchain ! WatchTxConfirmed(self, closingTx.tx.txid, nodeParams.minDepthBlocks)
blockchain ! WatchTxConfirmed(self, closingTx.tx.txid, nodeParams.channelConf.minDepthBlocks)
}
private def spendLocalCurrent(d: HasCommitments) = {
@ -2438,7 +2459,7 @@ class Channel(val nodeParams: NodeParams, val wallet: OnChainChannelFunder, remo
*/
private def watchConfirmedIfNeeded(txs: Iterable[Transaction], irrevocablySpent: Map[OutPoint, Transaction]): Unit = {
val (skip, process) = txs.partition(Closing.inputsAlreadySpent(_, irrevocablySpent))
process.foreach(tx => blockchain ! WatchTxConfirmed(self, tx.txid, nodeParams.minDepthBlocks))
process.foreach(tx => blockchain ! WatchTxConfirmed(self, tx.txid, nodeParams.channelConf.minDepthBlocks))
skip.foreach(tx => log.info(s"no need to watch txid=${tx.txid}, it has already been confirmed"))
}

View file

@ -23,7 +23,7 @@ import fr.acinq.bitcoin._
import fr.acinq.eclair._
import fr.acinq.eclair.blockchain.OnChainAddressGenerator
import fr.acinq.eclair.blockchain.fee.{FeeEstimator, FeeTargets, FeeratePerKw}
import fr.acinq.eclair.channel.Channel.REFRESH_CHANNEL_UPDATE_INTERVAL
import fr.acinq.eclair.channel.Channel.{ChannelConf, REFRESH_CHANNEL_UPDATE_INTERVAL}
import fr.acinq.eclair.crypto.Generators
import fr.acinq.eclair.crypto.keymanager.ChannelKeyManager
import fr.acinq.eclair.db.ChannelsDb
@ -70,13 +70,13 @@ object Helpers {
* @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
def minDepthForFunding(channelConf: ChannelConf, fundingSatoshis: Satoshi): Long = fundingSatoshis match {
case funding if funding <= Channel.MAX_FUNDING => channelConf.minDepthBlocks
case 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)
channelConf.minDepthBlocks.max(blocksToReachFunding)
}
def extractShutdownScript(channelId: ByteVector32, localFeatures: Features, remoteFeatures: Features, upfrontShutdownScript_opt: Option[ByteVector]): Either[ChannelException, Option[ByteVector]] = {
@ -104,16 +104,16 @@ object Helpers {
// MUST reject the channel.
if (nodeParams.chainHash != open.chainHash) return Left(InvalidChainHash(open.temporaryChannelId, local = nodeParams.chainHash, remote = open.chainHash))
if (open.fundingSatoshis < nodeParams.minFundingSatoshis || open.fundingSatoshis > nodeParams.maxFundingSatoshis) return Left(InvalidFundingAmount(open.temporaryChannelId, open.fundingSatoshis, nodeParams.minFundingSatoshis, nodeParams.maxFundingSatoshis))
if (open.fundingSatoshis < nodeParams.channelConf.minFundingSatoshis || open.fundingSatoshis > nodeParams.channelConf.maxFundingSatoshis) return Left(InvalidFundingAmount(open.temporaryChannelId, open.fundingSatoshis, nodeParams.channelConf.minFundingSatoshis, nodeParams.channelConf.maxFundingSatoshis))
// BOLT #2: Channel funding limits
if (open.fundingSatoshis >= Channel.MAX_FUNDING && !localFeatures.hasFeature(Features.Wumbo)) return Left(InvalidFundingAmount(open.temporaryChannelId, open.fundingSatoshis, nodeParams.minFundingSatoshis, Channel.MAX_FUNDING))
if (open.fundingSatoshis >= Channel.MAX_FUNDING && !localFeatures.hasFeature(Features.Wumbo)) return Left(InvalidFundingAmount(open.temporaryChannelId, open.fundingSatoshis, nodeParams.channelConf.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) return Left(InvalidPushAmount(open.temporaryChannelId, open.pushMsat, open.fundingSatoshis.toMilliSatoshi))
// BOLT #2: The receiving node MUST fail the channel if: to_self_delay is unreasonably large.
if (open.toSelfDelay > Channel.MAX_TO_SELF_DELAY || open.toSelfDelay > nodeParams.maxToLocalDelay) return Left(ToSelfDelayTooHigh(open.temporaryChannelId, open.toSelfDelay, nodeParams.maxToLocalDelay))
if (open.toSelfDelay > Channel.MAX_TO_SELF_DELAY || open.toSelfDelay > nodeParams.channelConf.maxToLocalDelay) return Left(ToSelfDelayTooHigh(open.temporaryChannelId, open.toSelfDelay, nodeParams.channelConf.maxToLocalDelay))
// BOLT #2: The receiving node MUST fail the channel if: max_accepted_htlcs is greater than 483.
if (open.maxAcceptedHtlcs > Channel.MAX_ACCEPTED_HTLCS) return Left(InvalidMaxAcceptedHtlcs(open.temporaryChannelId, open.maxAcceptedHtlcs, Channel.MAX_ACCEPTED_HTLCS))
@ -121,7 +121,7 @@ object Helpers {
// BOLT #2: The receiving node MUST fail the channel if: it considers feerate_per_kw too small for timely processing.
if (isFeeTooSmall(open.feeratePerKw)) return Left(FeerateTooSmall(open.temporaryChannelId, open.feeratePerKw))
if (open.dustLimitSatoshis > nodeParams.maxRemoteDustLimit) return Left(DustLimitTooLarge(open.temporaryChannelId, open.dustLimitSatoshis, nodeParams.maxRemoteDustLimit))
if (open.dustLimitSatoshis > nodeParams.channelConf.maxRemoteDustLimit) return Left(DustLimitTooLarge(open.temporaryChannelId, open.dustLimitSatoshis, nodeParams.channelConf.maxRemoteDustLimit))
// BOLT #2: The receiving node MUST fail the channel if: dust_limit_satoshis is greater than channel_reserve_satoshis.
if (open.dustLimitSatoshis > open.channelReserveSatoshis) return Left(DustLimitTooLarge(open.temporaryChannelId, open.dustLimitSatoshis, open.channelReserveSatoshis))
@ -145,7 +145,7 @@ object Helpers {
// now, but it will be done later when we receive `funding_created`
val reserveToFundingRatio = open.channelReserveSatoshis.toLong.toDouble / Math.max(open.fundingSatoshis.toLong, 1)
if (reserveToFundingRatio > nodeParams.maxReserveToFundingRatio) return Left(ChannelReserveTooHigh(open.temporaryChannelId, open.channelReserveSatoshis, reserveToFundingRatio, nodeParams.maxReserveToFundingRatio))
if (reserveToFundingRatio > nodeParams.channelConf.maxReserveToFundingRatio) return Left(ChannelReserveTooHigh(open.temporaryChannelId, open.channelReserveSatoshis, reserveToFundingRatio, nodeParams.channelConf.maxReserveToFundingRatio))
val channelFeatures = ChannelFeatures(channelType, localFeatures, remoteFeatures)
extractShutdownScript(open.temporaryChannelId, localFeatures, remoteFeatures, open.upfrontShutdownScript_opt).map(script_opt => (channelFeatures, script_opt))
@ -175,14 +175,14 @@ object Helpers {
if (accept.dustLimitSatoshis < Channel.MIN_DUST_LIMIT) return Left(DustLimitTooSmall(accept.temporaryChannelId, accept.dustLimitSatoshis, Channel.MIN_DUST_LIMIT))
}
if (accept.dustLimitSatoshis > nodeParams.maxRemoteDustLimit) return Left(DustLimitTooLarge(open.temporaryChannelId, accept.dustLimitSatoshis, nodeParams.maxRemoteDustLimit))
if (accept.dustLimitSatoshis > nodeParams.channelConf.maxRemoteDustLimit) return Left(DustLimitTooLarge(open.temporaryChannelId, accept.dustLimitSatoshis, nodeParams.channelConf.maxRemoteDustLimit))
// BOLT #2: The receiving node MUST fail the channel if: dust_limit_satoshis is greater than channel_reserve_satoshis.
if (accept.dustLimitSatoshis > accept.channelReserveSatoshis) return Left(DustLimitTooLarge(accept.temporaryChannelId, accept.dustLimitSatoshis, accept.channelReserveSatoshis))
// if minimum_depth is unreasonably large:
// MAY reject the channel.
if (accept.toSelfDelay > Channel.MAX_TO_SELF_DELAY || accept.toSelfDelay > nodeParams.maxToLocalDelay) return Left(ToSelfDelayTooHigh(accept.temporaryChannelId, accept.toSelfDelay, nodeParams.maxToLocalDelay))
if (accept.toSelfDelay > Channel.MAX_TO_SELF_DELAY || accept.toSelfDelay > nodeParams.channelConf.maxToLocalDelay) return Left(ToSelfDelayTooHigh(accept.temporaryChannelId, accept.toSelfDelay, nodeParams.channelConf.maxToLocalDelay))
// if channel_reserve_satoshis is less than dust_limit_satoshis within the open_channel message:
// MUST reject the channel.
@ -193,7 +193,7 @@ object Helpers {
if (open.channelReserveSatoshis < accept.dustLimitSatoshis) return Left(DustLimitAboveOurChannelReserve(accept.temporaryChannelId, accept.dustLimitSatoshis, open.channelReserveSatoshis))
val reserveToFundingRatio = accept.channelReserveSatoshis.toLong.toDouble / Math.max(open.fundingSatoshis.toLong, 1)
if (reserveToFundingRatio > nodeParams.maxReserveToFundingRatio) return Left(ChannelReserveTooHigh(open.temporaryChannelId, accept.channelReserveSatoshis, reserveToFundingRatio, nodeParams.maxReserveToFundingRatio))
if (reserveToFundingRatio > nodeParams.channelConf.maxReserveToFundingRatio) return Left(ChannelReserveTooHigh(open.temporaryChannelId, accept.channelReserveSatoshis, reserveToFundingRatio, nodeParams.channelConf.maxReserveToFundingRatio))
val channelFeatures = ChannelFeatures(channelType, localFeatures, remoteFeatures)
extractShutdownScript(accept.temporaryChannelId, localFeatures, remoteFeatures, accept.upfrontShutdownScript_opt).map(script_opt => (channelFeatures, script_opt))

View file

@ -102,7 +102,7 @@ private class FinalTxPublisher(nodeParams: NodeParams,
case ParentTxOk => publish()
case ParentTxMissing =>
log.debug("parent tx is missing, retrying after delay...")
timers.startSingleTimer(CheckParentTx, (1 + Random.nextLong(nodeParams.maxTxPublishRetryDelay.toMillis)).millis)
timers.startSingleTimer(CheckParentTx, (1 + Random.nextLong(nodeParams.channelConf.maxTxPublishRetryDelay.toMillis)).millis)
Behaviors.same
case UnknownFailure(reason) =>
log.error("could not check parent tx: ", reason)

View file

@ -136,7 +136,7 @@ private class MempoolTxMonitor(nodeParams: NodeParams,
context.system.eventStream ! EventStream.Subscribe(messageAdapter)
Behaviors.receiveMessagePartial {
case WrappedCurrentBlockHeight(currentBlockHeight) =>
timers.startSingleTimer(CheckTxConfirmationsKey, CheckTxConfirmations(currentBlockHeight), (1 + Random.nextLong(nodeParams.maxTxPublishRetryDelay.toMillis)).millis)
timers.startSingleTimer(CheckTxConfirmationsKey, CheckTxConfirmations(currentBlockHeight), (1 + Random.nextLong(nodeParams.channelConf.maxTxPublishRetryDelay.toMillis)).millis)
Behaviors.same
case CheckTxConfirmations(currentBlockHeight) =>
context.pipeToSelf(bitcoinClient.getTxConfirmations(cmd.tx.txid)) {
@ -149,7 +149,7 @@ private class MempoolTxMonitor(nodeParams: NodeParams,
if (confirmations == 0) {
cmd.replyTo ! TxInMempool(cmd.tx.txid, currentBlockHeight)
Behaviors.same
} else if (confirmations < nodeParams.minDepthBlocks) {
} else if (confirmations < nodeParams.channelConf.minDepthBlocks) {
log.info("txid={} has {} confirmations, waiting to reach min depth", cmd.tx.txid, confirmations)
cmd.replyTo ! TxRecentlyConfirmed(cmd.tx.txid, confirmations)
Behaviors.same

View file

@ -181,7 +181,7 @@ private class ReplaceableTxPrePublisher(nodeParams: NodeParams,
// - their commit is not confirmed: if it is, there is no need to publish our htlc transactions
// - if this is an htlc-success transaction, we have the preimage
context.pipeToSelf(bitcoinClient.getTxConfirmations(cmd.commitments.remoteCommit.txid)) {
case Success(Some(depth)) if depth >= nodeParams.minDepthBlocks => RemoteCommitTxConfirmed
case Success(Some(depth)) if depth >= nodeParams.channelConf.minDepthBlocks => RemoteCommitTxConfirmed
case Success(_) => ParentTxOk
case Failure(reason) => UnknownFailure(reason)
}
@ -243,7 +243,7 @@ private class ReplaceableTxPrePublisher(nodeParams: NodeParams,
// - our commit is not confirmed: if it is, there is no need to publish our claim-htlc transactions
// - if this is a claim-htlc-success transaction, we have the preimage
context.pipeToSelf(bitcoinClient.getTxConfirmations(cmd.commitments.localCommit.commitTxAndRemoteSig.commitTx.tx.txid)) {
case Success(Some(depth)) if depth >= nodeParams.minDepthBlocks => LocalCommitTxConfirmed
case Success(Some(depth)) if depth >= nodeParams.channelConf.minDepthBlocks => LocalCommitTxConfirmed
case Success(_) => ParentTxOk
case Failure(reason) => UnknownFailure(reason)
}

View file

@ -220,7 +220,7 @@ private class ReplaceableTxPublisher(nodeParams: NodeParams,
None
}
// We avoid a herd effect whenever we fee bump transactions.
targetFeerate_opt.foreach(targetFeerate => timers.startSingleTimer(BumpFeeKey, BumpFee(targetFeerate), (1 + Random.nextLong(nodeParams.maxTxPublishRetryDelay.toMillis)).millis))
targetFeerate_opt.foreach(targetFeerate => timers.startSingleTimer(BumpFeeKey, BumpFee(targetFeerate), (1 + Random.nextLong(nodeParams.channelConf.maxTxPublishRetryDelay.toMillis)).millis))
Behaviors.same
case BumpFee(targetFeerate) => fundReplacement(targetFeerate, tx)
case UpdateConfirmationTarget(target) =>

View file

@ -259,7 +259,7 @@ private class TxPublisher(nodeParams: NodeParams, factory: TxPublisher.ChildFact
// Our transaction has been evicted from the mempool because it depended on an unconfirmed input that has
// been replaced. We should be able to retry right now with new wallet inputs (no need to wait for a new
// block).
timers.startSingleTimer(cmd, (1 + Random.nextLong(nodeParams.maxTxPublishRetryDelay.toMillis)).millis)
timers.startSingleTimer(cmd, (1 + Random.nextLong(nodeParams.channelConf.maxTxPublishRetryDelay.toMillis)).millis)
run(pending2, retryNextBlock, channelContext)
case TxRejectedReason.CouldNotFund =>
// We don't have enough funds at the moment to afford our target feerate, but it may change once pending
@ -293,7 +293,7 @@ private class TxPublisher(nodeParams: NodeParams, factory: TxPublisher.ChildFact
case WrappedCurrentBlockHeight(currentBlockHeight) =>
if (retryNextBlock.nonEmpty) {
log.info("{} transactions are still pending at block {}, retrying {} transactions that previously failed", pending.size, currentBlockHeight, retryNextBlock.length)
retryNextBlock.foreach(cmd => timers.startSingleTimer(cmd, (1 + Random.nextLong(nodeParams.maxTxPublishRetryDelay.toMillis)).millis))
retryNextBlock.foreach(cmd => timers.startSingleTimer(cmd, (1 + Random.nextLong(nodeParams.channelConf.maxTxPublishRetryDelay.toMillis)).millis))
}
run(pending, Seq.empty, channelContext)

View file

@ -85,7 +85,7 @@ private class TxTimeLocksMonitor(nodeParams: NodeParams,
case WrappedCurrentBlockHeight(currentBlockHeight) =>
if (cltvTimeout <= currentBlockHeight) {
context.system.eventStream ! EventStream.Unsubscribe(messageAdapter)
timers.startSingleTimer(CheckRelativeTimeLock, (1 + Random.nextLong(nodeParams.maxTxPublishRetryDelay.toMillis)).millis)
timers.startSingleTimer(CheckRelativeTimeLock, (1 + Random.nextLong(nodeParams.channelConf.maxTxPublishRetryDelay.toMillis)).millis)
Behaviors.same
} else {
Behaviors.same

View file

@ -141,7 +141,7 @@ class Peer(val nodeParams: NodeParams, remoteNodeId: PublicKey, wallet: OnChainA
} else if (c.fundingSatoshis >= Channel.MAX_FUNDING && !d.remoteFeatures.hasFeature(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) {
} else if (c.fundingSatoshis > nodeParams.channelConf.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 {
@ -154,7 +154,7 @@ class Peer(val nodeParams: NodeParams, remoteNodeId: PublicKey, wallet: OnChainA
val channelFeeratePerKw = nodeParams.onChainFeeConf.getCommitmentFeerate(remoteNodeId, channelType, c.fundingSatoshis, None)
val fundingTxFeeratePerKw = c.fundingTxFeeratePerKw_opt.getOrElse(nodeParams.onChainFeeConf.feeEstimator.getFeeratePerKw(target = nodeParams.onChainFeeConf.feeTargets.fundingBlockTarget))
log.info(s"requesting a new channel with type=$channelType 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.peerConnection, d.remoteInit, c.channelFlags.getOrElse(nodeParams.channelFlags), channelConfig, channelType)
channel ! INPUT_INIT_FUNDER(temporaryChannelId, c.fundingSatoshis, c.pushMsat, channelFeeratePerKw, fundingTxFeeratePerKw, localParams, d.peerConnection, d.remoteInit, c.channelFlags.getOrElse(nodeParams.channelConf.channelFlags), channelConfig, channelType)
stay() using d.copy(channels = d.channels + (TemporaryChannelId(temporaryChannelId) -> channel))
}
@ -505,12 +505,12 @@ object Peer {
LocalParams(
nodeParams.nodeId,
nodeParams.channelKeyManager.newFundingKeyPath(isFunder), // we make sure that funder and fundee key path end differently
dustLimit = nodeParams.dustLimit,
maxHtlcValueInFlightMsat = nodeParams.maxHtlcValueInFlightMsat,
channelReserve = (fundingAmount * nodeParams.reserveToFundingRatio).max(nodeParams.dustLimit), // BOLT #2: make sure that our reserve is above our dust limit
htlcMinimum = nodeParams.htlcMinimum,
toSelfDelay = nodeParams.toRemoteDelay, // we choose their delay
maxAcceptedHtlcs = nodeParams.maxAcceptedHtlcs,
dustLimit = nodeParams.channelConf.dustLimit,
maxHtlcValueInFlightMsat = nodeParams.channelConf.maxHtlcValueInFlightMsat,
channelReserve = (fundingAmount * nodeParams.channelConf.reserveToFundingRatio).max(nodeParams.channelConf.dustLimit), // BOLT #2: make sure that our reserve is above our dust limit
htlcMinimum = nodeParams.channelConf.htlcMinimum,
toSelfDelay = nodeParams.channelConf.toRemoteDelay, // we choose their delay
maxAcceptedHtlcs = nodeParams.channelConf.maxAcceptedHtlcs,
isFunder = isFunder,
defaultFinalScriptPubKey = defaultFinalScriptPubkey,
walletStaticPaymentBasepoint = walletStaticPaymentBasepoint,

View file

@ -98,7 +98,7 @@ class MultiPartHandler(nodeParams: NodeParams, register: ActorRef, db: IncomingP
Map(Features.PaymentSecret -> FeatureSupport.Mandatory, Features.VariableLengthOnion -> FeatureSupport.Mandatory)
}
// Insert a fake invoice and then restart the incoming payment handler
val paymentRequest = PaymentRequest(nodeParams.chainHash, amount, paymentHash, nodeParams.privateKey, desc, nodeParams.minFinalExpiryDelta, paymentSecret = p.payload.paymentSecret, features = PaymentRequestFeatures(features))
val paymentRequest = PaymentRequest(nodeParams.chainHash, amount, paymentHash, nodeParams.privateKey, desc, nodeParams.channelConf.minFinalExpiryDelta, paymentSecret = p.payload.paymentSecret, features = PaymentRequestFeatures(features))
log.debug("generated fake payment request={} from amount={} (KeySend)", PaymentRequest.write(paymentRequest), amount)
db.addIncomingPayment(paymentRequest, paymentPreimage, paymentType = PaymentType.KeySend)
ctx.self ! p
@ -241,7 +241,7 @@ object MultiPartHandler {
paymentHash,
nodeParams.privateKey,
description,
nodeParams.minFinalExpiryDelta,
nodeParams.channelConf.minFinalExpiryDelta,
fallbackAddress_opt,
expirySeconds = Some(expirySeconds),
extraHops = extraHops,
@ -305,7 +305,7 @@ object MultiPartHandler {
}
private def validatePaymentCltv(nodeParams: NodeParams, payment: IncomingPaymentPacket.FinalPacket, record: IncomingPayment)(implicit log: LoggingAdapter): Boolean = {
val minExpiry = record.paymentRequest.minFinalCltvExpiryDelta.getOrElse(nodeParams.minFinalExpiryDelta).toCltvExpiry(nodeParams.currentBlockHeight)
val minExpiry = record.paymentRequest.minFinalCltvExpiryDelta.getOrElse(nodeParams.channelConf.minFinalExpiryDelta).toCltvExpiry(nodeParams.currentBlockHeight)
if (payment.add.cltvExpiry < minExpiry) {
log.warning("received payment with expiry too small for amount={} totalAmount={}", payment.add.amountMsat, payment.payload.totalAmount)
false

View file

@ -106,7 +106,7 @@ object NodeRelay {
val fee = nodeFee(nodeParams.relayParams.minTrampolineFees, payloadOut.amountToForward)
if (upstream.amountIn - payloadOut.amountToForward < fee) {
Some(TrampolineFeeInsufficient)
} else if (upstream.expiryIn - payloadOut.outgoingCltv < nodeParams.expiryDelta) {
} else if (upstream.expiryIn - payloadOut.outgoingCltv < nodeParams.channelConf.expiryDelta) {
Some(TrampolineExpiryTooSoon)
} else if (payloadOut.outgoingCltv <= CltvExpiry(nodeParams.currentBlockHeight)) {
Some(TrampolineExpiryTooSoon)

View file

@ -133,7 +133,7 @@ class PaymentInitiator(nodeParams: NodeParams, outgoingPaymentFactory: PaymentIn
case pf: PaymentFailed => pending.get(pf.id).foreach {
case pp: PendingTrampolinePayment =>
val trampolineRoute = Seq(
NodeHop(nodeParams.nodeId, pp.r.trampolineNodeId, nodeParams.expiryDelta, 0 msat),
NodeHop(nodeParams.nodeId, pp.r.trampolineNodeId, nodeParams.channelConf.expiryDelta, 0 msat),
NodeHop(pp.r.trampolineNodeId, pp.r.recipientNodeId, pp.r.trampolineAttempts.last._2, pp.r.trampolineAttempts.last._1)
)
val decryptedFailures = pf.failures.collect { case RemoteFailure(_, _, Sphinx.DecryptedFailurePacket(_, f)) => f }
@ -194,7 +194,7 @@ class PaymentInitiator(nodeParams: NodeParams, outgoingPaymentFactory: PaymentIn
private def buildTrampolinePayment(r: SendRequestedPayment, trampolineNodeId: PublicKey, trampolineFees: MilliSatoshi, trampolineExpiryDelta: CltvExpiryDelta): Try[(MilliSatoshi, CltvExpiry, OnionRoutingPacket)] = {
val trampolineRoute = Seq(
NodeHop(nodeParams.nodeId, trampolineNodeId, nodeParams.expiryDelta, 0 msat),
NodeHop(nodeParams.nodeId, trampolineNodeId, nodeParams.channelConf.expiryDelta, 0 msat),
NodeHop(trampolineNodeId, r.recipientNodeId, trampolineExpiryDelta, trampolineFees) // for now we only use a single trampoline hop
)
val finalPayload = if (r.paymentRequest.features.allowMultiPart) {

View file

@ -87,6 +87,7 @@ object EclairInternalsSerializer {
).as[PathFindingExperimentConf]
val routerConfCodec: Codec[RouterConf] = (
("watchSpentWindow" | finiteDurationCodec) ::
("channelExcludeDuration" | finiteDurationCodec) ::
("routerBroadcastInterval" | finiteDurationCodec) ::
("requestNodeAnnouncements" | bool(8)) ::

View file

@ -88,7 +88,7 @@ class Router(val nodeParams: NodeParams, watcher: typed.ActorRef[ZmqWatcher.Comm
val txid = pc.fundingTxid
val TxCoordinates(_, _, outputIndex) = ShortChannelId.coordinates(pc.ann.shortChannelId)
// avoid herd effect at startup because watch-spent are intensive in terms of rpc calls to bitcoind
context.system.scheduler.scheduleOnce(Random.nextLong(nodeParams.watchSpentWindow.toSeconds).seconds) {
context.system.scheduler.scheduleOnce(Random.nextLong(nodeParams.routerConf.watchSpentWindow.toSeconds).seconds) {
watcher ! WatchExternalChannelSpent(self, txid, outputIndex, pc.ann.shortChannelId)
}
}
@ -297,7 +297,8 @@ object Router {
)
}
case class RouterConf(channelExcludeDuration: FiniteDuration,
case class RouterConf(watchSpentWindow: FiniteDuration,
channelExcludeDuration: FiniteDuration,
routerBroadcastInterval: FiniteDuration,
requestNodeAnnouncements: Boolean,
encodingType: EncodingType,

View file

@ -261,7 +261,7 @@ class StartupSpec extends AnyFunSuite {
}
test("NodeParams should fail if htlc-minimum-msat is set to 0") {
val noHtlcMinimumConf = ConfigFactory.parseString("htlc-minimum-msat = 0")
val noHtlcMinimumConf = ConfigFactory.parseString("channel.htlc-minimum-msat = 0")
assert(Try(makeNodeParamsWithDefaults(noHtlcMinimumConf.withFallback(defaultConf))).isFailure)
}

View file

@ -20,7 +20,7 @@ import fr.acinq.bitcoin.{Block, ByteVector32, Satoshi, SatoshiLong, Script}
import fr.acinq.eclair.FeatureSupport.{Mandatory, Optional}
import fr.acinq.eclair.Features._
import fr.acinq.eclair.blockchain.fee._
import fr.acinq.eclair.channel.Channel.UnhandledExceptionStrategy
import fr.acinq.eclair.channel.Channel.{ChannelConf, UnhandledExceptionStrategy}
import fr.acinq.eclair.channel.{ChannelFlags, LocalParams}
import fr.acinq.eclair.crypto.keymanager.{LocalChannelKeyManager, LocalNodeKeyManager}
import fr.acinq.eclair.io.MessageRelay.RelayAll
@ -99,8 +99,28 @@ object TestConstants {
pluginParams = List(pluginParams),
overrideFeatures = Map.empty,
syncWhitelist = Set.empty,
dustLimit = 1100 sat,
maxRemoteDustLimit = 1500 sat,
channelConf = ChannelConf(
dustLimit = 1100 sat,
maxRemoteDustLimit = 1500 sat,
maxHtlcValueInFlightMsat = UInt64(500000000),
maxAcceptedHtlcs = 100,
expiryDelta = CltvExpiryDelta(144),
fulfillSafetyBeforeTimeout = CltvExpiryDelta(6),
minFinalExpiryDelta = CltvExpiryDelta(18),
maxBlockProcessingDelay = 10 millis,
maxTxPublishRetryDelay = 10 millis,
htlcMinimum = 0 msat,
minDepthBlocks = 3,
toRemoteDelay = CltvExpiryDelta(144),
maxToLocalDelay = CltvExpiryDelta(1000),
reserveToFundingRatio = 0.01, // note: not used (overridden below)
maxReserveToFundingRatio = 0.05,
unhandledExceptionStrategy = UnhandledExceptionStrategy.LocalClose,
revocationTimeout = 20 seconds,
channelFlags = ChannelFlags.Public,
minFundingSatoshis = 1000 sat,
maxFundingSatoshis = 16777215 sat,
),
onChainFeeConf = OnChainFeeConf(
feeTargets = FeeTargets(6, 2, 36, 12, 18, 0),
feeEstimator = new TestFeeEstimator,
@ -109,17 +129,6 @@ object TestConstants {
defaultFeerateTolerance = FeerateTolerance(0.5, 8.0, anchorOutputsFeeratePerKw, DustTolerance(25_000 sat, closeOnUpdateFeeOverflow = true)),
perNodeFeerateTolerance = Map.empty
),
maxHtlcValueInFlightMsat = UInt64(500000000),
maxAcceptedHtlcs = 100,
expiryDelta = CltvExpiryDelta(144),
fulfillSafetyBeforeTimeout = CltvExpiryDelta(6),
minFinalExpiryDelta = CltvExpiryDelta(18),
maxBlockProcessingDelay = 10 millis,
maxTxPublishRetryDelay = 10 millis,
htlcMinimum = 0 msat,
minDepthBlocks = 3,
toRemoteDelay = CltvExpiryDelta(144),
maxToLocalDelay = CltvExpiryDelta(1000),
relayParams = RelayParams(
publicChannelFees = RelayFees(
feeBase = 546000 msat,
@ -130,21 +139,13 @@ object TestConstants {
minTrampolineFees = RelayFees(
feeBase = 548000 msat,
feeProportionalMillionths = 30)),
reserveToFundingRatio = 0.01, // note: not used (overridden below)
maxReserveToFundingRatio = 0.05,
unhandledExceptionStrategy = UnhandledExceptionStrategy.LocalClose,
db = TestDatabases.inMemoryDb(),
revocationTimeout = 20 seconds,
autoReconnect = false,
initialRandomReconnectDelay = 5 seconds,
maxReconnectInterval = 1 hour,
chainHash = Block.RegtestGenesisBlock.hash,
channelFlags = ChannelFlags.Public,
watchSpentWindow = 1 second,
paymentRequestExpiry = 1 hour,
multiPartPaymentExpiry = 30 seconds,
minFundingSatoshis = 1000 sat,
maxFundingSatoshis = 16777215 sat,
peerConnectionConf = PeerConnection.Conf(
authTimeout = 10 seconds,
initTimeout = 10 seconds,
@ -156,6 +157,7 @@ object TestConstants {
maxOnionMessagesPerSecond = 10
),
routerConf = RouterConf(
watchSpentWindow = 1 second,
channelExcludeDuration = 60 seconds,
routerBroadcastInterval = 5 seconds,
requestNodeAnnouncements = true,
@ -232,8 +234,28 @@ object TestConstants {
pluginParams = Nil,
overrideFeatures = Map.empty,
syncWhitelist = Set.empty,
dustLimit = 1000 sat,
maxRemoteDustLimit = 1500 sat,
channelConf = ChannelConf(
dustLimit = 1000 sat,
maxRemoteDustLimit = 1500 sat,
maxHtlcValueInFlightMsat = UInt64.MaxValue, // Bob has no limit on the combined max value of in-flight htlcs
maxAcceptedHtlcs = 30,
expiryDelta = CltvExpiryDelta(144),
fulfillSafetyBeforeTimeout = CltvExpiryDelta(6),
minFinalExpiryDelta = CltvExpiryDelta(18),
maxBlockProcessingDelay = 10 millis,
maxTxPublishRetryDelay = 10 millis,
htlcMinimum = 1000 msat,
minDepthBlocks = 3,
toRemoteDelay = CltvExpiryDelta(144),
maxToLocalDelay = CltvExpiryDelta(1000),
reserveToFundingRatio = 0.01, // note: not used (overridden below)
maxReserveToFundingRatio = 0.05,
unhandledExceptionStrategy = UnhandledExceptionStrategy.LocalClose,
revocationTimeout = 20 seconds,
channelFlags = ChannelFlags.Public,
minFundingSatoshis = 1000 sat,
maxFundingSatoshis = 16777215 sat,
),
onChainFeeConf = OnChainFeeConf(
feeTargets = FeeTargets(6, 2, 36, 12, 18, 0),
feeEstimator = new TestFeeEstimator,
@ -242,17 +264,6 @@ object TestConstants {
defaultFeerateTolerance = FeerateTolerance(0.75, 1.5, anchorOutputsFeeratePerKw, DustTolerance(30_000 sat, closeOnUpdateFeeOverflow = true)),
perNodeFeerateTolerance = Map.empty
),
maxHtlcValueInFlightMsat = UInt64.MaxValue, // Bob has no limit on the combined max value of in-flight htlcs
maxAcceptedHtlcs = 30,
expiryDelta = CltvExpiryDelta(144),
fulfillSafetyBeforeTimeout = CltvExpiryDelta(6),
minFinalExpiryDelta = CltvExpiryDelta(18),
maxBlockProcessingDelay = 10 millis,
maxTxPublishRetryDelay = 10 millis,
htlcMinimum = 1000 msat,
minDepthBlocks = 3,
toRemoteDelay = CltvExpiryDelta(144),
maxToLocalDelay = CltvExpiryDelta(1000),
relayParams = RelayParams(
publicChannelFees = RelayFees(
feeBase = 546000 msat,
@ -263,21 +274,13 @@ object TestConstants {
minTrampolineFees = RelayFees(
feeBase = 548000 msat,
feeProportionalMillionths = 30)),
reserveToFundingRatio = 0.01, // note: not used (overridden below)
maxReserveToFundingRatio = 0.05,
unhandledExceptionStrategy = UnhandledExceptionStrategy.LocalClose,
db = TestDatabases.inMemoryDb(),
revocationTimeout = 20 seconds,
autoReconnect = false,
initialRandomReconnectDelay = 5 seconds,
maxReconnectInterval = 1 hour,
chainHash = Block.RegtestGenesisBlock.hash,
channelFlags = ChannelFlags.Public,
watchSpentWindow = 1 second,
paymentRequestExpiry = 1 hour,
multiPartPaymentExpiry = 30 seconds,
minFundingSatoshis = 1000 sat,
maxFundingSatoshis = 16777215 sat,
peerConnectionConf = PeerConnection.Conf(
authTimeout = 10 seconds,
initTimeout = 10 seconds,
@ -289,6 +292,7 @@ object TestConstants {
maxOnionMessagesPerSecond = 10
),
routerConf = RouterConf(
watchSpentWindow = 1 second,
channelExcludeDuration = 60 seconds,
routerBroadcastInterval = 5 seconds,
requestNodeAnnouncements = true,

View file

@ -39,13 +39,13 @@ class HelpersSpec extends TestKitBaseClass with AnyFunSuiteLike with ChannelStat
implicit val log: akka.event.LoggingAdapter = akka.event.NoLogging
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)
assert(Helpers.minDepthForFunding(nodeParams.channelConf, Btc(1)) == 4)
assert(Helpers.minDepthForFunding(nodeParams.channelConf.copy(minDepthBlocks = 6), Btc(1)) == 6) // 4 conf would be enough but we use min-depth=6
assert(Helpers.minDepthForFunding(nodeParams.channelConf, Btc(6.25)) == 16) // we use scaling_factor=15 and a fixed block reward of 6.25BTC
assert(Helpers.minDepthForFunding(nodeParams.channelConf, Btc(12.50)) == 31)
assert(Helpers.minDepthForFunding(nodeParams.channelConf, Btc(12.60)) == 32)
assert(Helpers.minDepthForFunding(nodeParams.channelConf, Btc(30)) == 73)
assert(Helpers.minDepthForFunding(nodeParams.channelConf, Btc(50)) == 121)
}
test("compute refresh delay") {
@ -176,7 +176,7 @@ class HelpersSpec extends TestKitBaseClass with AnyFunSuiteLike with ChannelStat
def findTimedOutHtlcs(f: Fixture, withoutHtlcId: Boolean): Unit = {
import f._
val dustLimit = alice.underlyingActor.nodeParams.dustLimit
val dustLimit = alice.underlyingActor.nodeParams.channelConf.dustLimit
val commitmentFormat = alice.stateData.asInstanceOf[DATA_CLOSING].commitments.commitmentFormat
val localCommit = alice.stateData.asInstanceOf[DATA_CLOSING].commitments.localCommit
val remoteCommit = bob.stateData.asInstanceOf[DATA_CLOSING].commitments.remoteCommit

View file

@ -210,10 +210,10 @@ class RestoreSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with Chan
Alice.nodeParams
.modify(_.relayParams.privateChannelFees.feeProportionalMillionths).setTo(2345),
Alice.nodeParams
.modify(_.expiryDelta).setTo(CltvExpiryDelta(147)),
.modify(_.channelConf.expiryDelta).setTo(CltvExpiryDelta(147)),
Alice.nodeParams
.modify(_.relayParams.privateChannelFees.feeProportionalMillionths).setTo(2345)
.modify(_.expiryDelta).setTo(CltvExpiryDelta(147)),
.modify(_.channelConf.expiryDelta).setTo(CltvExpiryDelta(147)),
) foreach { newConfig =>
val newAlice: TestFSMRef[ChannelState, ChannelData, Channel] = TestFSMRef(new Channel(newConfig, wallet, Bob.nodeParams.nodeId, alice2blockchain.ref, relayerA.ref, FakeTxPublisherFactory(alice2blockchain)), alicePeer.ref)
@ -223,7 +223,7 @@ class RestoreSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with Chan
assert(!Announcements.areSameIgnoreFlags(u1.channelUpdate, oldStateData.channelUpdate))
assert(u1.channelUpdate.feeBaseMsat === newConfig.relayParams.privateChannelFees.feeBase)
assert(u1.channelUpdate.feeProportionalMillionths === newConfig.relayParams.privateChannelFees.feeProportionalMillionths)
assert(u1.channelUpdate.cltvExpiryDelta === newConfig.expiryDelta)
assert(u1.channelUpdate.cltvExpiryDelta === newConfig.channelConf.expiryDelta)
newAlice ! INPUT_RECONNECTED(alice2bob.ref, aliceInit, bobInit)
bob ! INPUT_RECONNECTED(bob2alice.ref, bobInit, aliceInit)

View file

@ -86,13 +86,13 @@ class MempoolTxMonitorSpec extends TestKitBaseClass with AnyFunSuiteLike with Bi
probe.expectMsg(TxInMempool(tx.txid, currentBlockHeight(probe)))
probe.expectNoMessage(100 millis)
assert(TestConstants.Alice.nodeParams.minDepthBlocks > 1)
assert(TestConstants.Alice.nodeParams.channelConf.minDepthBlocks > 1)
generateBlocks(1)
system.eventStream.publish(CurrentBlockHeight(currentBlockHeight(probe)))
probe.expectMsg(TxRecentlyConfirmed(tx.txid, 1))
probe.expectNoMessage(100 millis) // we wait for more than one confirmation to protect against reorgs
generateBlocks(TestConstants.Alice.nodeParams.minDepthBlocks - 1)
generateBlocks(TestConstants.Alice.nodeParams.channelConf.minDepthBlocks - 1)
system.eventStream.publish(CurrentBlockHeight(currentBlockHeight(probe)))
probe.expectMsg(TxDeeplyBuried(tx))
}
@ -109,7 +109,7 @@ class MempoolTxMonitorSpec extends TestKitBaseClass with AnyFunSuiteLike with Bi
monitor ! Publish(probe.ref, tx2, tx2.txIn.head.outPoint, "test-tx", 10 sat)
waitTxInMempool(bitcoinClient, tx2.txid, probe)
generateBlocks(TestConstants.Alice.nodeParams.minDepthBlocks)
generateBlocks(TestConstants.Alice.nodeParams.channelConf.minDepthBlocks)
system.eventStream.publish(CurrentBlockHeight(currentBlockHeight(probe)))
probe.expectMsg(TxDeeplyBuried(tx2))
}
@ -257,7 +257,7 @@ class MempoolTxMonitorSpec extends TestKitBaseClass with AnyFunSuiteLike with Bi
assert(txPublished.miningFee === 15.sat)
assert(txPublished.desc === "test-tx")
generateBlocks(TestConstants.Alice.nodeParams.minDepthBlocks)
generateBlocks(TestConstants.Alice.nodeParams.channelConf.minDepthBlocks)
system.eventStream.publish(CurrentBlockHeight(currentBlockHeight(probe)))
eventListener.expectMsg(TransactionConfirmed(txPublished.channelId, txPublished.remoteNodeId, tx))
}

View file

@ -121,15 +121,15 @@ trait ChannelStateTestsHelperMethods extends TestKitBase {
system.eventStream.subscribe(channelUpdateListener.ref, classOf[LocalChannelDown])
val router = TestProbe()
val finalNodeParamsA = nodeParamsA
.modify(_.dustLimit).setToIf(tags.contains(ChannelStateTestsTags.HighDustLimitDifferenceAliceBob))(5000 sat)
.modify(_.dustLimit).setToIf(tags.contains(ChannelStateTestsTags.HighDustLimitDifferenceBobAlice))(1000 sat)
.modify(_.maxRemoteDustLimit).setToIf(tags.contains(ChannelStateTestsTags.HighDustLimitDifferenceAliceBob))(10000 sat)
.modify(_.maxRemoteDustLimit).setToIf(tags.contains(ChannelStateTestsTags.HighDustLimitDifferenceBobAlice))(10000 sat)
.modify(_.channelConf.dustLimit).setToIf(tags.contains(ChannelStateTestsTags.HighDustLimitDifferenceAliceBob))(5000 sat)
.modify(_.channelConf.dustLimit).setToIf(tags.contains(ChannelStateTestsTags.HighDustLimitDifferenceBobAlice))(1000 sat)
.modify(_.channelConf.maxRemoteDustLimit).setToIf(tags.contains(ChannelStateTestsTags.HighDustLimitDifferenceAliceBob))(10000 sat)
.modify(_.channelConf.maxRemoteDustLimit).setToIf(tags.contains(ChannelStateTestsTags.HighDustLimitDifferenceBobAlice))(10000 sat)
val finalNodeParamsB = nodeParamsB
.modify(_.dustLimit).setToIf(tags.contains(ChannelStateTestsTags.HighDustLimitDifferenceAliceBob))(1000 sat)
.modify(_.dustLimit).setToIf(tags.contains(ChannelStateTestsTags.HighDustLimitDifferenceBobAlice))(5000 sat)
.modify(_.maxRemoteDustLimit).setToIf(tags.contains(ChannelStateTestsTags.HighDustLimitDifferenceAliceBob))(10000 sat)
.modify(_.maxRemoteDustLimit).setToIf(tags.contains(ChannelStateTestsTags.HighDustLimitDifferenceBobAlice))(10000 sat)
.modify(_.channelConf.dustLimit).setToIf(tags.contains(ChannelStateTestsTags.HighDustLimitDifferenceAliceBob))(1000 sat)
.modify(_.channelConf.dustLimit).setToIf(tags.contains(ChannelStateTestsTags.HighDustLimitDifferenceBobAlice))(5000 sat)
.modify(_.channelConf.maxRemoteDustLimit).setToIf(tags.contains(ChannelStateTestsTags.HighDustLimitDifferenceAliceBob))(10000 sat)
.modify(_.channelConf.maxRemoteDustLimit).setToIf(tags.contains(ChannelStateTestsTags.HighDustLimitDifferenceBobAlice))(10000 sat)
val alice: TestFSMRef[ChannelState, ChannelData, Channel] = TestFSMRef(new Channel(finalNodeParamsA, wallet, finalNodeParamsB.nodeId, alice2blockchain.ref, relayerA.ref, FakeTxPublisherFactory(alice2blockchain), origin_opt = Some(aliceOrigin.ref)), alicePeer.ref)
val bob: TestFSMRef[ChannelState, ChannelData, Channel] = TestFSMRef(new Channel(finalNodeParamsB, wallet, finalNodeParamsA.nodeId, bob2blockchain.ref, relayerB.ref, FakeTxPublisherFactory(bob2blockchain)), bobPeer.ref)
SetupFixture(alice, bob, aliceOrigin, alice2bob, bob2alice, alice2blockchain, bob2blockchain, router, relayerA, relayerB, channelUpdateListener, wallet, alicePeer, bobPeer)

View file

@ -44,12 +44,12 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
import com.softwaremill.quicklens._
val aliceNodeParams = Alice.nodeParams
.modify(_.chainHash).setToIf(test.tags.contains("mainnet"))(Block.LivenetGenesisBlock.hash)
.modify(_.maxFundingSatoshis).setToIf(test.tags.contains("high-max-funding-size"))(Btc(100))
.modify(_.maxRemoteDustLimit).setToIf(test.tags.contains("high-remote-dust-limit"))(15000 sat)
.modify(_.channelConf.maxFundingSatoshis).setToIf(test.tags.contains("high-max-funding-size"))(Btc(100))
.modify(_.channelConf.maxRemoteDustLimit).setToIf(test.tags.contains("high-remote-dust-limit"))(15000 sat)
val bobNodeParams = Bob.nodeParams
.modify(_.chainHash).setToIf(test.tags.contains("mainnet"))(Block.LivenetGenesisBlock.hash)
.modify(_.maxFundingSatoshis).setToIf(test.tags.contains("high-max-funding-size"))(Btc(100))
.modify(_.channelConf.maxFundingSatoshis).setToIf(test.tags.contains("high-max-funding-size"))(Btc(100))
val setup = init(aliceNodeParams, bobNodeParams, wallet = new NoOpOnChainWallet())
@ -209,7 +209,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
val highDustLimitSatoshis = 2000.sat
alice ! accept.copy(dustLimitSatoshis = highDustLimitSatoshis)
val error = alice2bob.expectMsgType[Error]
assert(error === Error(accept.temporaryChannelId, DustLimitTooLarge(accept.temporaryChannelId, highDustLimitSatoshis, Alice.nodeParams.maxRemoteDustLimit).getMessage))
assert(error === Error(accept.temporaryChannelId, DustLimitTooLarge(accept.temporaryChannelId, highDustLimitSatoshis, Alice.nodeParams.channelConf.maxRemoteDustLimit).getMessage))
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
}
@ -220,7 +220,7 @@ class WaitForAcceptChannelStateSpec extends TestKitBaseClass with FixtureAnyFunS
val delayTooHigh = CltvExpiryDelta(10000)
alice ! accept.copy(toSelfDelay = delayTooHigh)
val error = alice2bob.expectMsgType[Error]
assert(error === Error(accept.temporaryChannelId, ToSelfDelayTooHigh(accept.temporaryChannelId, delayTooHigh, Alice.nodeParams.maxToLocalDelay).getMessage))
assert(error === Error(accept.temporaryChannelId, ToSelfDelayTooHigh(accept.temporaryChannelId, delayTooHigh, Alice.nodeParams.channelConf.maxToLocalDelay).getMessage))
awaitCond(alice.stateName == CLOSED)
aliceOrigin.expectMsgType[Status.Failure]
}

View file

@ -43,7 +43,7 @@ class WaitForOpenChannelStateSpec extends TestKitBaseClass with FixtureAnyFunSui
import com.softwaremill.quicklens._
val bobNodeParams = Bob.nodeParams
.modify(_.maxFundingSatoshis).setToIf(test.tags.contains("max-funding-satoshis"))(Btc(1))
.modify(_.channelConf.maxFundingSatoshis).setToIf(test.tags.contains("max-funding-satoshis"))(Btc(1))
val setup = init(nodeParamsB = bobNodeParams)
@ -118,7 +118,7 @@ class WaitForOpenChannelStateSpec extends TestKitBaseClass with FixtureAnyFunSui
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, Bob.nodeParams.maxFundingSatoshis).getMessage))
assert(error === Error(open.temporaryChannelId, InvalidFundingAmount(open.temporaryChannelId, lowFunding, Bob.nodeParams.channelConf.minFundingSatoshis, Bob.nodeParams.channelConf.maxFundingSatoshis).getMessage))
awaitCond(bob.stateName == CLOSED)
}
@ -128,17 +128,17 @@ class WaitForOpenChannelStateSpec extends TestKitBaseClass with FixtureAnyFunSui
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, Bob.nodeParams.maxFundingSatoshis).getMessage).toAscii)
assert(error.toAscii === Error(open.temporaryChannelId, InvalidFundingAmount(open.temporaryChannelId, highFundingMsat, Bob.nodeParams.channelConf.minFundingSatoshis, Bob.nodeParams.channelConf.maxFundingSatoshis).getMessage).toAscii)
awaitCond(bob.stateName == CLOSED)
}
test("recv OpenChannel (fundingSatoshis > max-funding-satoshis)", Tag(ChannelStateTestsTags.Wumbo)) { f =>
import f._
val open = alice2bob.expectMsgType[OpenChannel]
val highFundingSat = Bob.nodeParams.maxFundingSatoshis + Btc(1)
val highFundingSat = Bob.nodeParams.channelConf.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)
assert(error.toAscii === Error(open.temporaryChannelId, InvalidFundingAmount(open.temporaryChannelId, highFundingSat, Bob.nodeParams.channelConf.minFundingSatoshis, Bob.nodeParams.channelConf.maxFundingSatoshis).getMessage).toAscii)
}
test("recv OpenChannel (invalid max accepted htlcs)") { f =>
@ -167,7 +167,7 @@ class WaitForOpenChannelStateSpec extends TestKitBaseClass with FixtureAnyFunSui
val delayTooHigh = CltvExpiryDelta(10000)
bob ! open.copy(toSelfDelay = delayTooHigh)
val error = bob2alice.expectMsgType[Error]
assert(error === Error(open.temporaryChannelId, ToSelfDelayTooHigh(open.temporaryChannelId, delayTooHigh, Alice.nodeParams.maxToLocalDelay).getMessage))
assert(error === Error(open.temporaryChannelId, ToSelfDelayTooHigh(open.temporaryChannelId, delayTooHigh, Alice.nodeParams.channelConf.maxToLocalDelay).getMessage))
awaitCond(bob.stateName == CLOSED)
}
@ -223,7 +223,7 @@ class WaitForOpenChannelStateSpec extends TestKitBaseClass with FixtureAnyFunSui
val dustLimitTooHigh = 2000.sat
bob ! open.copy(dustLimitSatoshis = dustLimitTooHigh)
val error = bob2alice.expectMsgType[Error]
assert(error === Error(open.temporaryChannelId, DustLimitTooLarge(open.temporaryChannelId, dustLimitTooHigh, Bob.nodeParams.maxRemoteDustLimit).getMessage))
assert(error === Error(open.temporaryChannelId, DustLimitTooLarge(open.temporaryChannelId, dustLimitTooHigh, Bob.nodeParams.channelConf.maxRemoteDustLimit).getMessage))
awaitCond(bob.stateName == CLOSED)
}

View file

@ -43,9 +43,9 @@ class WaitForFundingCreatedStateSpec extends TestKitBaseClass with FixtureAnyFun
override def withFixture(test: OneArgTest): Outcome = {
import com.softwaremill.quicklens._
val aliceNodeParams = Alice.nodeParams
.modify(_.maxFundingSatoshis).setToIf(test.tags.contains(ChannelStateTestsTags.Wumbo))(Btc(100))
.modify(_.channelConf.maxFundingSatoshis).setToIf(test.tags.contains(ChannelStateTestsTags.Wumbo))(Btc(100))
val bobNodeParams = Bob.nodeParams
.modify(_.maxFundingSatoshis).setToIf(test.tags.contains(ChannelStateTestsTags.Wumbo))(Btc(100))
.modify(_.channelConf.maxFundingSatoshis).setToIf(test.tags.contains(ChannelStateTestsTags.Wumbo))(Btc(100))
val (fundingSatoshis, pushMsat) = if (test.tags.contains("funder_below_reserve")) {
(1000100 sat, (1000000 sat).toMilliSatoshi) // toLocal = 100 satoshis
@ -85,7 +85,7 @@ class WaitForFundingCreatedStateSpec extends TestKitBaseClass with FixtureAnyFun
bob2blockchain.expectMsgType[TxPublisher.SetChannelId]
bob2blockchain.expectMsgType[WatchFundingSpent]
val watchConfirmed = bob2blockchain.expectMsgType[WatchFundingConfirmed]
assert(watchConfirmed.minDepth === Alice.nodeParams.minDepthBlocks)
assert(watchConfirmed.minDepth === Alice.nodeParams.channelConf.minDepthBlocks)
}
test("recv FundingCreated (wumbo)", Tag(ChannelStateTestsTags.Wumbo)) { f =>
@ -98,7 +98,7 @@ class WaitForFundingCreatedStateSpec extends TestKitBaseClass with FixtureAnyFun
bob2blockchain.expectMsgType[WatchFundingSpent]
val watchConfirmed = bob2blockchain.expectMsgType[WatchFundingConfirmed]
// when we are fundee, we use a higher min depth for wumbo channels
assert(watchConfirmed.minDepth > Bob.nodeParams.minDepthBlocks)
assert(watchConfirmed.minDepth > Bob.nodeParams.channelConf.minDepthBlocks)
}
test("recv FundingCreated (funder can't pay fees)", Tag("funder_below_reserve")) { f =>

View file

@ -44,9 +44,9 @@ class WaitForFundingSignedStateSpec extends TestKitBaseClass with FixtureAnyFunS
override def withFixture(test: OneArgTest): Outcome = {
import com.softwaremill.quicklens._
val aliceNodeParams = Alice.nodeParams
.modify(_.maxFundingSatoshis).setToIf(test.tags.contains(ChannelStateTestsTags.Wumbo))(Btc(100))
.modify(_.channelConf.maxFundingSatoshis).setToIf(test.tags.contains(ChannelStateTestsTags.Wumbo))(Btc(100))
val bobNodeParams = Bob.nodeParams
.modify(_.maxFundingSatoshis).setToIf(test.tags.contains(ChannelStateTestsTags.Wumbo))(Btc(100))
.modify(_.channelConf.maxFundingSatoshis).setToIf(test.tags.contains(ChannelStateTestsTags.Wumbo))(Btc(100))
val (fundingSatoshis, pushMsat) = if (test.tags.contains(ChannelStateTestsTags.Wumbo)) {
(Btc(5).toSatoshi, TestConstants.pushMsat)
@ -90,7 +90,7 @@ class WaitForFundingSignedStateSpec extends TestKitBaseClass with FixtureAnyFunS
assert(txPublished.tx.txid === fundingTxId)
assert(txPublished.miningFee > 0.sat)
val watchConfirmed = alice2blockchain.expectMsgType[WatchFundingConfirmed]
assert(watchConfirmed.minDepth === Alice.nodeParams.minDepthBlocks)
assert(watchConfirmed.minDepth === Alice.nodeParams.channelConf.minDepthBlocks)
aliceOrigin.expectMsgType[ChannelOpenResponse.ChannelOpened]
}
@ -102,7 +102,7 @@ class WaitForFundingSignedStateSpec extends TestKitBaseClass with FixtureAnyFunS
alice2blockchain.expectMsgType[WatchFundingSpent]
val watchConfirmed = alice2blockchain.expectMsgType[WatchFundingConfirmed]
// when we are funder, we keep our regular min depth even for wumbo channels
assert(watchConfirmed.minDepth === Alice.nodeParams.minDepthBlocks)
assert(watchConfirmed.minDepth === Alice.nodeParams.channelConf.minDepthBlocks)
aliceOrigin.expectMsgType[ChannelOpenResponse.ChannelOpened]
}

View file

@ -448,8 +448,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
val sender = TestProbe()
val initialState = alice.stateData.asInstanceOf[DATA_NORMAL]
assert(alice.underlyingActor.nodeParams.onChainFeeConf.feerateToleranceFor(bob.underlyingActor.nodeParams.nodeId).dustTolerance.maxExposure === 25_000.sat)
assert(alice.underlyingActor.nodeParams.dustLimit === 1100.sat)
assert(bob.underlyingActor.nodeParams.dustLimit === 1000.sat)
assert(alice.underlyingActor.nodeParams.channelConf.dustLimit === 1100.sat)
assert(bob.underlyingActor.nodeParams.channelConf.dustLimit === 1000.sat)
// Alice sends HTLCs to Bob that add 21 000 sat to the dust exposure.
// She signs them but Bob doesn't answer yet.
@ -472,8 +472,8 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
val sender = TestProbe()
val initialState = bob.stateData.asInstanceOf[DATA_NORMAL]
assert(bob.underlyingActor.nodeParams.onChainFeeConf.feerateToleranceFor(alice.underlyingActor.nodeParams.nodeId).dustTolerance.maxExposure === 30_000.sat)
assert(alice.underlyingActor.nodeParams.dustLimit === 1100.sat)
assert(bob.underlyingActor.nodeParams.dustLimit === 1000.sat)
assert(alice.underlyingActor.nodeParams.channelConf.dustLimit === 1100.sat)
assert(bob.underlyingActor.nodeParams.channelConf.dustLimit === 1000.sat)
// Bob sends HTLCs to Alice that add 21 000 sat to the dust exposure.
// He signs them but Alice doesn't answer yet.
@ -753,7 +753,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
import f._
val sender = TestProbe()
// for the test to be really useful we have constraint on parameters
assert(Alice.nodeParams.dustLimit > Bob.nodeParams.dustLimit)
assert(Alice.nodeParams.channelConf.dustLimit > Bob.nodeParams.channelConf.dustLimit)
// and a low feerate to avoid messing with dust exposure limits
val currentFeerate = FeeratePerKw(2500 sat)
alice.feeEstimator.setFeerate(FeeratesPerKw.single(currentFeerate))
@ -761,10 +761,10 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
updateFee(currentFeerate, alice, bob, alice2bob, bob2alice)
// we're gonna exchange two htlcs in each direction, the goal is to have bob's commitment have 4 htlcs, and alice's
// commitment only have 3. We will then check that alice indeed persisted 4 htlcs, and bob only 3.
val aliceMinReceive = Alice.nodeParams.dustLimit + weight2fee(currentFeerate, DefaultCommitmentFormat.htlcSuccessWeight)
val aliceMinOffer = Alice.nodeParams.dustLimit + weight2fee(currentFeerate, DefaultCommitmentFormat.htlcTimeoutWeight)
val bobMinReceive = Bob.nodeParams.dustLimit + weight2fee(currentFeerate, DefaultCommitmentFormat.htlcSuccessWeight)
val bobMinOffer = Bob.nodeParams.dustLimit + weight2fee(currentFeerate, DefaultCommitmentFormat.htlcTimeoutWeight)
val aliceMinReceive = Alice.nodeParams.channelConf.dustLimit + weight2fee(currentFeerate, DefaultCommitmentFormat.htlcSuccessWeight)
val aliceMinOffer = Alice.nodeParams.channelConf.dustLimit + weight2fee(currentFeerate, DefaultCommitmentFormat.htlcTimeoutWeight)
val bobMinReceive = Bob.nodeParams.channelConf.dustLimit + weight2fee(currentFeerate, DefaultCommitmentFormat.htlcSuccessWeight)
val bobMinOffer = Bob.nodeParams.channelConf.dustLimit + weight2fee(currentFeerate, DefaultCommitmentFormat.htlcTimeoutWeight)
val a2b_1 = bobMinReceive + 10.sat // will be in alice and bob tx
val a2b_2 = bobMinReceive + 20.sat // will be in alice and bob tx
val b2a_1 = aliceMinReceive + 10.sat // will be in alice and bob tx
@ -1363,16 +1363,16 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
test("recv RevokeAndAck (over max dust htlc exposure in local commit only with pending local changes)", Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs), Tag(ChannelStateTestsTags.HighDustLimitDifferenceAliceBob)) { f =>
import f._
val sender = TestProbe()
assert(alice.underlyingActor.nodeParams.dustLimit === 5000.sat)
assert(bob.underlyingActor.nodeParams.dustLimit === 1000.sat)
assert(alice.underlyingActor.nodeParams.channelConf.dustLimit === 5000.sat)
assert(bob.underlyingActor.nodeParams.channelConf.dustLimit === 1000.sat)
testRevokeAndAckDustOverflowSingleCommit(f)
}
test("recv RevokeAndAck (over max dust htlc exposure in remote commit only with pending local changes)", Tag(ChannelStateTestsTags.AnchorOutputsZeroFeeHtlcTxs), Tag(ChannelStateTestsTags.HighDustLimitDifferenceBobAlice)) { f =>
import f._
val sender = TestProbe()
assert(alice.underlyingActor.nodeParams.dustLimit === 1000.sat)
assert(bob.underlyingActor.nodeParams.dustLimit === 5000.sat)
assert(alice.underlyingActor.nodeParams.channelConf.dustLimit === 1000.sat)
assert(bob.underlyingActor.nodeParams.channelConf.dustLimit === 5000.sat)
testRevokeAndAckDustOverflowSingleCommit(f)
}
@ -2746,7 +2746,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
bob ! CMD_FULFILL_HTLC(htlc.id, r, commit = true)
bob2alice.expectMsgType[UpdateFulfillHtlc]
bob2alice.expectMsgType[CommitSig]
bob ! CurrentBlockHeight(htlc.cltvExpiry.blockHeight - Bob.nodeParams.fulfillSafetyBeforeTimeout.toInt)
bob ! CurrentBlockHeight(htlc.cltvExpiry.blockHeight - Bob.nodeParams.channelConf.fulfillSafetyBeforeTimeout.toInt)
val ChannelErrorOccurred(_, _, _, _, LocalError(err), isFatal) = listener.expectMsgType[ChannelErrorOccurred]
assert(isFatal)
@ -2779,7 +2779,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
bob ! CMD_FULFILL_HTLC(htlc.id, r, commit = false)
bob2alice.expectMsgType[UpdateFulfillHtlc]
bob2alice.expectNoMessage(500 millis)
bob ! CurrentBlockHeight(htlc.cltvExpiry.blockHeight - Bob.nodeParams.fulfillSafetyBeforeTimeout.toInt)
bob ! CurrentBlockHeight(htlc.cltvExpiry.blockHeight - Bob.nodeParams.channelConf.fulfillSafetyBeforeTimeout.toInt)
val ChannelErrorOccurred(_, _, _, _, LocalError(err), isFatal) = listener.expectMsgType[ChannelErrorOccurred]
assert(isFatal)
@ -2816,7 +2816,7 @@ class NormalStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
bob2alice.forward(alice)
alice2bob.expectMsgType[RevokeAndAck]
alice2bob.forward(bob)
bob ! CurrentBlockHeight(htlc.cltvExpiry.blockHeight - Bob.nodeParams.fulfillSafetyBeforeTimeout.toInt)
bob ! CurrentBlockHeight(htlc.cltvExpiry.blockHeight - Bob.nodeParams.channelConf.fulfillSafetyBeforeTimeout.toInt)
val ChannelErrorOccurred(_, _, _, _, LocalError(err), isFatal) = listener.expectMsgType[ChannelErrorOccurred]
assert(isFatal)

View file

@ -536,7 +536,7 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
// We simulate a pending fulfill on that HTLC but not relayed.
// When it is close to expiring upstream, we should close the channel.
bob.underlyingActor.nodeParams.db.pendingCommands.addSettlementCommand(initialState.channelId, CMD_FULFILL_HTLC(htlc.id, r, commit = true))
bob ! CurrentBlockHeight(htlc.cltvExpiry.blockHeight - bob.underlyingActor.nodeParams.fulfillSafetyBeforeTimeout.toInt)
bob ! CurrentBlockHeight(htlc.cltvExpiry.blockHeight - bob.underlyingActor.nodeParams.channelConf.fulfillSafetyBeforeTimeout.toInt)
val ChannelErrorOccurred(_, _, _, _, LocalError(err), isFatal) = listener.expectMsgType[ChannelErrorOccurred]
assert(isFatal)
@ -567,7 +567,7 @@ class OfflineStateSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with
// We simulate a pending failure on that HTLC.
// Even if we get close to expiring upstream we shouldn't close the channel, because we have nothing to lose.
bob ! CMD_FAIL_HTLC(htlc.id, Right(IncorrectOrUnknownPaymentDetails(0 msat, BlockHeight(0))))
bob ! CurrentBlockHeight(htlc.cltvExpiry.blockHeight - bob.underlyingActor.nodeParams.fulfillSafetyBeforeTimeout.toInt)
bob ! CurrentBlockHeight(htlc.cltvExpiry.blockHeight - bob.underlyingActor.nodeParams.channelConf.fulfillSafetyBeforeTimeout.toInt)
bob2blockchain.expectNoMessage(250 millis)
alice2blockchain.expectNoMessage(250 millis)

View file

@ -452,9 +452,9 @@ abstract class ChannelIntegrationSpec extends IntegrationSpec {
class StandardChannelIntegrationSpec extends ChannelIntegrationSpec {
test("start eclair nodes") {
instantiateEclairNode("A", ConfigFactory.parseMap(Map("eclair.node-alias" -> "A", "eclair.expiry-delta-blocks" -> 40, "eclair.fulfill-safety-before-timeout-blocks" -> 12, "eclair.server.port" -> 29740, "eclair.api.port" -> 28090).asJava).withFallback(withDefaultCommitment).withFallback(commonConfig))
instantiateEclairNode("C", ConfigFactory.parseMap(Map("eclair.node-alias" -> "C", "eclair.expiry-delta-blocks" -> 40, "eclair.fulfill-safety-before-timeout-blocks" -> 12, "eclair.server.port" -> 29741, "eclair.api.port" -> 28091).asJava).withFallback(withAnchorOutputs).withFallback(commonConfig))
instantiateEclairNode("F", ConfigFactory.parseMap(Map("eclair.node-alias" -> "F", "eclair.expiry-delta-blocks" -> 40, "eclair.fulfill-safety-before-timeout-blocks" -> 12, "eclair.server.port" -> 29742, "eclair.api.port" -> 28092).asJava).withFallback(withDefaultCommitment).withFallback(commonConfig))
instantiateEclairNode("A", ConfigFactory.parseMap(Map("eclair.node-alias" -> "A", "eclair.channel.expiry-delta-blocks" -> 40, "eclair.channel.fulfill-safety-before-timeout-blocks" -> 12, "eclair.server.port" -> 29740, "eclair.api.port" -> 28090).asJava).withFallback(withDefaultCommitment).withFallback(commonConfig))
instantiateEclairNode("C", ConfigFactory.parseMap(Map("eclair.node-alias" -> "C", "eclair.channel.expiry-delta-blocks" -> 40, "eclair.channel.fulfill-safety-before-timeout-blocks" -> 12, "eclair.server.port" -> 29741, "eclair.api.port" -> 28091).asJava).withFallback(withAnchorOutputs).withFallback(commonConfig))
instantiateEclairNode("F", ConfigFactory.parseMap(Map("eclair.node-alias" -> "F", "eclair.channel.expiry-delta-blocks" -> 40, "eclair.channel.fulfill-safety-before-timeout-blocks" -> 12, "eclair.server.port" -> 29742, "eclair.api.port" -> 28092).asJava).withFallback(withDefaultCommitment).withFallback(commonConfig))
}
test("connect nodes") {
@ -779,9 +779,9 @@ class AnchorOutputChannelIntegrationSpec extends AnchorChannelIntegrationSpec {
override val commitmentFormat = Transactions.UnsafeLegacyAnchorOutputsCommitmentFormat
test("start eclair nodes") {
instantiateEclairNode("A", ConfigFactory.parseMap(Map("eclair.node-alias" -> "A", "eclair.expiry-delta-blocks" -> 40, "eclair.fulfill-safety-before-timeout-blocks" -> 12, "eclair.server.port" -> 29750, "eclair.api.port" -> 28093).asJava).withFallback(withStaticRemoteKey).withFallback(commonConfig))
instantiateEclairNode("C", ConfigFactory.parseMap(Map("eclair.node-alias" -> "C", "eclair.expiry-delta-blocks" -> 40, "eclair.fulfill-safety-before-timeout-blocks" -> 12, "eclair.server.port" -> 29751, "eclair.api.port" -> 28094).asJava).withFallback(withAnchorOutputs).withFallback(commonConfig))
instantiateEclairNode("F", ConfigFactory.parseMap(Map("eclair.node-alias" -> "F", "eclair.expiry-delta-blocks" -> 40, "eclair.fulfill-safety-before-timeout-blocks" -> 12, "eclair.server.port" -> 29753, "eclair.api.port" -> 28095).asJava).withFallback(withAnchorOutputs).withFallback(commonConfig))
instantiateEclairNode("A", ConfigFactory.parseMap(Map("eclair.node-alias" -> "A", "eclair.channel.expiry-delta-blocks" -> 40, "eclair.channel.fulfill-safety-before-timeout-blocks" -> 12, "eclair.server.port" -> 29750, "eclair.api.port" -> 28093).asJava).withFallback(withStaticRemoteKey).withFallback(commonConfig))
instantiateEclairNode("C", ConfigFactory.parseMap(Map("eclair.node-alias" -> "C", "eclair.channel.expiry-delta-blocks" -> 40, "eclair.channel.fulfill-safety-before-timeout-blocks" -> 12, "eclair.server.port" -> 29751, "eclair.api.port" -> 28094).asJava).withFallback(withAnchorOutputs).withFallback(commonConfig))
instantiateEclairNode("F", ConfigFactory.parseMap(Map("eclair.node-alias" -> "F", "eclair.channel.expiry-delta-blocks" -> 40, "eclair.channel.fulfill-safety-before-timeout-blocks" -> 12, "eclair.server.port" -> 29753, "eclair.api.port" -> 28095).asJava).withFallback(withAnchorOutputs).withFallback(commonConfig))
}
test("connect nodes") {
@ -819,9 +819,9 @@ class AnchorOutputZeroFeeHtlcTxsChannelIntegrationSpec extends AnchorChannelInte
override val commitmentFormat = Transactions.ZeroFeeHtlcTxAnchorOutputsCommitmentFormat
test("start eclair nodes") {
instantiateEclairNode("A", ConfigFactory.parseMap(Map("eclair.node-alias" -> "A", "eclair.expiry-delta-blocks" -> 40, "eclair.fulfill-safety-before-timeout-blocks" -> 12, "eclair.server.port" -> 29760, "eclair.api.port" -> 28096).asJava).withFallback(withStaticRemoteKey).withFallback(commonConfig))
instantiateEclairNode("C", ConfigFactory.parseMap(Map("eclair.node-alias" -> "C", "eclair.expiry-delta-blocks" -> 40, "eclair.fulfill-safety-before-timeout-blocks" -> 12, "eclair.server.port" -> 29761, "eclair.api.port" -> 28097).asJava).withFallback(withAnchorOutputsZeroFeeHtlcTxs).withFallback(commonConfig))
instantiateEclairNode("F", ConfigFactory.parseMap(Map("eclair.node-alias" -> "F", "eclair.expiry-delta-blocks" -> 40, "eclair.fulfill-safety-before-timeout-blocks" -> 12, "eclair.server.port" -> 29763, "eclair.api.port" -> 28098).asJava).withFallback(withAnchorOutputsZeroFeeHtlcTxs).withFallback(commonConfig))
instantiateEclairNode("A", ConfigFactory.parseMap(Map("eclair.node-alias" -> "A", "eclair.channel.expiry-delta-blocks" -> 40, "eclair.channel.fulfill-safety-before-timeout-blocks" -> 12, "eclair.server.port" -> 29760, "eclair.api.port" -> 28096).asJava).withFallback(withStaticRemoteKey).withFallback(commonConfig))
instantiateEclairNode("C", ConfigFactory.parseMap(Map("eclair.node-alias" -> "C", "eclair.channel.expiry-delta-blocks" -> 40, "eclair.channel.fulfill-safety-before-timeout-blocks" -> 12, "eclair.server.port" -> 29761, "eclair.api.port" -> 28097).asJava).withFallback(withAnchorOutputsZeroFeeHtlcTxs).withFallback(commonConfig))
instantiateEclairNode("F", ConfigFactory.parseMap(Map("eclair.node-alias" -> "F", "eclair.channel.expiry-delta-blocks" -> 40, "eclair.channel.fulfill-safety-before-timeout-blocks" -> 12, "eclair.server.port" -> 29763, "eclair.api.port" -> 28098).asJava).withFallback(withAnchorOutputsZeroFeeHtlcTxs).withFallback(commonConfig))
}
test("connect nodes") {

View file

@ -81,14 +81,14 @@ abstract class IntegrationSpec extends TestKitBaseClass with BitcoindService wit
"eclair.bitcoind.zmqblock" -> s"tcp://127.0.0.1:$bitcoindZmqBlockPort",
"eclair.bitcoind.zmqtx" -> s"tcp://127.0.0.1:$bitcoindZmqTxPort",
"eclair.bitcoind.wallet" -> defaultWallet,
"eclair.mindepth-blocks" -> 2,
"eclair.max-htlc-value-in-flight-msat" -> 100000000000L,
"eclair.max-block-processing-delay" -> "2 seconds",
"eclair.channel.mindepth-blocks" -> 2,
"eclair.channel.max-htlc-value-in-flight-msat" -> 100000000000L,
"eclair.channel.max-block-processing-delay" -> "2 seconds",
"eclair.channel.to-remote-delay-blocks" -> 24,
"eclair.channel.max-funding-satoshis" -> 500000000,
"eclair.router.broadcast-interval" -> "2 seconds",
"eclair.auto-reconnect" -> false,
"eclair.to-remote-delay-blocks" -> 24,
"eclair.multi-part-payment-expiry" -> "20 seconds",
"eclair.max-funding-satoshis" -> 500000000).asJava).withFallback(ConfigFactory.load())
"eclair.multi-part-payment-expiry" -> "20 seconds").asJava).withFallback(ConfigFactory.load())
private val commonFeatures = ConfigFactory.parseMap(Map(
s"eclair.features.${DataLossProtect.rfcName}" -> "optional",

View file

@ -58,13 +58,13 @@ import scala.jdk.CollectionConverters._
class PaymentIntegrationSpec extends IntegrationSpec {
test("start eclair nodes") {
instantiateEclairNode("A", ConfigFactory.parseMap(Map("eclair.node-alias" -> "A", "eclair.expiry-delta-blocks" -> 130, "eclair.server.port" -> 29730, "eclair.api.port" -> 28080, "eclair.channel.channel-flags.announce-channel" -> false).asJava).withFallback(withDefaultCommitment).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.trampoline-payments-enable" -> true).asJava).withFallback(withDefaultCommitment).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.trampoline-payments-enable" -> true).asJava).withFallback(withAnchorOutputsZeroFeeHtlcTxs).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.trampoline-payments-enable" -> true).asJava).withFallback(withDefaultCommitment).withFallback(commonConfig))
instantiateEclairNode("E", ConfigFactory.parseMap(Map("eclair.node-alias" -> "E", "eclair.expiry-delta-blocks" -> 134, "eclair.server.port" -> 29734, "eclair.api.port" -> 28084).asJava).withFallback(withAnchorOutputsZeroFeeHtlcTxs).withFallback(commonConfig))
instantiateEclairNode("F", ConfigFactory.parseMap(Map("eclair.node-alias" -> "F", "eclair.expiry-delta-blocks" -> 135, "eclair.server.port" -> 29735, "eclair.api.port" -> 28085, "eclair.trampoline-payments-enable" -> true).asJava).withFallback(commonConfig))
instantiateEclairNode("G", ConfigFactory.parseMap(Map("eclair.node-alias" -> "G", "eclair.expiry-delta-blocks" -> 136, "eclair.server.port" -> 29736, "eclair.api.port" -> 28086, "eclair.relay.fees.public-channels.fee-base-msat" -> 1010, "eclair.relay.fees.public-channels.fee-proportional-millionths" -> 102, "eclair.trampoline-payments-enable" -> true).asJava).withFallback(commonConfig))
instantiateEclairNode("A", ConfigFactory.parseMap(Map("eclair.node-alias" -> "A", "eclair.channel.expiry-delta-blocks" -> 130, "eclair.server.port" -> 29730, "eclair.api.port" -> 28080, "eclair.channel.channel-flags.announce-channel" -> false).asJava).withFallback(withDefaultCommitment).withFallback(commonConfig)) // A's channels are private
instantiateEclairNode("B", ConfigFactory.parseMap(Map("eclair.node-alias" -> "B", "eclair.channel.expiry-delta-blocks" -> 131, "eclair.server.port" -> 29731, "eclair.api.port" -> 28081, "eclair.trampoline-payments-enable" -> true).asJava).withFallback(withDefaultCommitment).withFallback(commonConfig))
instantiateEclairNode("C", ConfigFactory.parseMap(Map("eclair.node-alias" -> "C", "eclair.channel.expiry-delta-blocks" -> 132, "eclair.server.port" -> 29732, "eclair.api.port" -> 28082, "eclair.trampoline-payments-enable" -> true).asJava).withFallback(withAnchorOutputsZeroFeeHtlcTxs).withFallback(commonConfig))
instantiateEclairNode("D", ConfigFactory.parseMap(Map("eclair.node-alias" -> "D", "eclair.channel.expiry-delta-blocks" -> 133, "eclair.server.port" -> 29733, "eclair.api.port" -> 28083, "eclair.trampoline-payments-enable" -> true).asJava).withFallback(withDefaultCommitment).withFallback(commonConfig))
instantiateEclairNode("E", ConfigFactory.parseMap(Map("eclair.node-alias" -> "E", "eclair.channel.expiry-delta-blocks" -> 134, "eclair.server.port" -> 29734, "eclair.api.port" -> 28084).asJava).withFallback(withAnchorOutputsZeroFeeHtlcTxs).withFallback(commonConfig))
instantiateEclairNode("F", ConfigFactory.parseMap(Map("eclair.node-alias" -> "F", "eclair.channel.expiry-delta-blocks" -> 135, "eclair.server.port" -> 29735, "eclair.api.port" -> 28085, "eclair.trampoline-payments-enable" -> true).asJava).withFallback(commonConfig))
instantiateEclairNode("G", ConfigFactory.parseMap(Map("eclair.node-alias" -> "G", "eclair.channel.expiry-delta-blocks" -> 136, "eclair.server.port" -> 29736, "eclair.api.port" -> 28086, "eclair.relay.fees.public-channels.fee-base-msat" -> 1010, "eclair.relay.fees.public-channels.fee-proportional-millionths" -> 102, "eclair.trampoline-payments-enable" -> true).asJava).withFallback(commonConfig))
}
test("connect nodes") {
@ -181,7 +181,7 @@ class PaymentIntegrationSpec extends IntegrationSpec {
nodes("B").register ! Register.ForwardShortId(sender.ref, shortIdBC, CMD_GETINFO(ActorRef.noSender))
val commitmentBC = sender.expectMsgType[RES_GETINFO].data.asInstanceOf[DATA_NORMAL].commitments
// we then forge a new channel_update for B-C...
val channelUpdateBC = Announcements.makeChannelUpdate(Block.RegtestGenesisBlock.hash, nodes("B").nodeParams.privateKey, nodes("C").nodeParams.nodeId, shortIdBC, nodes("B").nodeParams.expiryDelta + 1, nodes("C").nodeParams.htlcMinimum, nodes("B").nodeParams.relayParams.publicChannelFees.feeBase, nodes("B").nodeParams.relayParams.publicChannelFees.feeProportionalMillionths, 500000000 msat)
val channelUpdateBC = Announcements.makeChannelUpdate(Block.RegtestGenesisBlock.hash, nodes("B").nodeParams.privateKey, nodes("C").nodeParams.nodeId, shortIdBC, nodes("B").nodeParams.channelConf.expiryDelta + 1, nodes("C").nodeParams.channelConf.htlcMinimum, nodes("B").nodeParams.relayParams.publicChannelFees.feeBase, nodes("B").nodeParams.relayParams.publicChannelFees.feeProportionalMillionths, 500000000 msat)
// ...and notify B's relayer
nodes("B").system.eventStream.publish(LocalChannelUpdate(system.deadLetters, commitmentBC.channelId, shortIdBC, commitmentBC.remoteParams.nodeId, None, channelUpdateBC, commitmentBC))
// we retrieve a payment hash from D
@ -215,11 +215,11 @@ class PaymentIntegrationSpec extends IntegrationSpec {
logger.info(s"channelUpdateBC=$channelUpdateBC")
logger.info(s"channelUpdateBC_new=$channelUpdateBC_new")
assert(channelUpdateBC_new.timestamp > channelUpdateBC.timestamp)
assert(channelUpdateBC_new.cltvExpiryDelta == nodes("B").nodeParams.expiryDelta)
assert(channelUpdateBC_new.cltvExpiryDelta == nodes("B").nodeParams.channelConf.expiryDelta)
awaitCond({
sender.send(nodes("A").router, Router.GetChannelsMap)
val u = updateFor(nodes("B").nodeParams.nodeId, sender.expectMsgType[Map[ShortChannelId, PublicChannel]].apply(channelUpdateBC.shortChannelId)).get
u.cltvExpiryDelta == nodes("B").nodeParams.expiryDelta
u.cltvExpiryDelta == nodes("B").nodeParams.channelConf.expiryDelta
}, max = 30 seconds, interval = 1 second)
}

View file

@ -43,8 +43,8 @@ class PerformanceIntegrationSpec extends IntegrationSpec {
test("start eclair nodes") {
val commonPerfTestConfig = ConfigFactory.parseMap(Map(
"eclair.max-funding-satoshis" -> 100_000_000,
"eclair.max-accepted-htlcs" -> Channel.MAX_ACCEPTED_HTLCS,
"eclair.channel.max-funding-satoshis" -> 100_000_000,
"eclair.channel.max-accepted-htlcs" -> Channel.MAX_ACCEPTED_HTLCS,
"eclair.file-backup.enabled" -> false,
).asJava)

View file

@ -74,7 +74,7 @@ class PeerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike with Paralle
.modify(_.features).setToIf(test.tags.contains("wumbo"))(Features(Wumbo -> Optional))
.modify(_.features).setToIf(test.tags.contains("anchor_outputs"))(Features(StaticRemoteKey -> Optional, AnchorOutputs -> Optional))
.modify(_.features).setToIf(test.tags.contains("anchor_outputs_zero_fee_htlc_tx"))(Features(StaticRemoteKey -> Optional, AnchorOutputs -> Optional, AnchorOutputsZeroFeeHtlcTx -> Optional))
.modify(_.maxFundingSatoshis).setToIf(test.tags.contains("high-max-funding-satoshis"))(Btc(0.9))
.modify(_.channelConf.maxFundingSatoshis).setToIf(test.tags.contains("high-max-funding-satoshis"))(Btc(0.9))
.modify(_.autoReconnect).setToIf(test.tags.contains("auto_reconnect"))(true)
if (test.tags.contains("with_node_announcement")) {

View file

@ -75,7 +75,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
val register = TestProbe()
val eventListener = TestProbe()
system.eventStream.subscribe(eventListener.ref, classOf[PaymentEvent])
withFixture(test.toNoArgTest(FixtureParam(nodeParams, nodeParams.minFinalExpiryDelta.toCltvExpiry(nodeParams.currentBlockHeight), register, eventListener, TestProbe())))
withFixture(test.toNoArgTest(FixtureParam(nodeParams, nodeParams.channelConf.minFinalExpiryDelta.toCltvExpiry(nodeParams.currentBlockHeight), register, eventListener, TestProbe())))
}
}
@ -173,12 +173,12 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
sender.send(handlerWithMpp, ReceivePayment(Some(42000 msat), Left("1 coffee")))
val pr1 = sender.expectMsgType[PaymentRequest]
assert(pr1.minFinalCltvExpiryDelta === Some(nodeParams.minFinalExpiryDelta))
assert(pr1.minFinalCltvExpiryDelta === Some(nodeParams.channelConf.minFinalExpiryDelta))
assert(pr1.expiry === Some(Alice.nodeParams.paymentRequestExpiry.toSeconds))
sender.send(handlerWithMpp, ReceivePayment(Some(42000 msat), Left("1 coffee with custom expiry"), expirySeconds_opt = Some(60)))
val pr2 = sender.expectMsgType[PaymentRequest]
assert(pr2.minFinalCltvExpiryDelta === Some(nodeParams.minFinalExpiryDelta))
assert(pr2.minFinalCltvExpiryDelta === Some(nodeParams.channelConf.minFinalExpiryDelta))
assert(pr2.expiry === Some(60))
}
@ -285,7 +285,7 @@ class MultiPartHandlerSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
val pr = sender.expectMsgType[PaymentRequest]
assert(pr.features.allowMultiPart)
val lowCltvExpiry = nodeParams.fulfillSafetyBeforeTimeout.toCltvExpiry(nodeParams.currentBlockHeight)
val lowCltvExpiry = nodeParams.channelConf.fulfillSafetyBeforeTimeout.toCltvExpiry(nodeParams.currentBlockHeight)
val add = UpdateAddHtlc(ByteVector32.One, 0, 800 msat, pr.paymentHash, lowCltvExpiry, TestConstants.emptyOnionPacket)
sender.send(handlerWithMpp, IncomingPaymentPacket.FinalPacket(add, PaymentOnion.createMultiPartPayload(add.amountMsat, 1000 msat, add.cltvExpiry, pr.paymentSecret.get, pr.paymentMetadata)))
val cmd = register.expectMsgType[Register.Forward[CMD_FAIL_HTLC]].message

View file

@ -436,7 +436,7 @@ class PaymentInitiatorSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike
multiPartPayFsm.send(initiator, failed)
val failure = sender.expectMsgType[PaymentFailed]
assert(failure.failures === Seq(LocalFailure(finalAmount, Seq(NodeHop(nodeParams.nodeId, b, nodeParams.expiryDelta, 0 msat), NodeHop(b, c, CltvExpiryDelta(24), 25000 msat)), RouteNotFound)))
assert(failure.failures === Seq(LocalFailure(finalAmount, Seq(NodeHop(nodeParams.nodeId, b, nodeParams.channelConf.expiryDelta, 0 msat), NodeHop(b, c, CltvExpiryDelta(24), 25000 msat)), RouteNotFound)))
eventListener.expectMsg(failure)
sender.expectNoMessage(100 millis)
eventListener.expectNoMessage(100 millis)

View file

@ -99,7 +99,7 @@ class AnnouncementsSpec extends AnyFunSuite {
}
test("create valid signed channel update announcement") {
val ann = makeChannelUpdate(Block.RegtestGenesisBlock.hash, Alice.nodeParams.privateKey, randomKey().publicKey, ShortChannelId(45561L), Alice.nodeParams.expiryDelta, Alice.nodeParams.htlcMinimum, Alice.nodeParams.relayParams.publicChannelFees.feeBase, Alice.nodeParams.relayParams.publicChannelFees.feeProportionalMillionths, 500000000 msat)
val ann = makeChannelUpdate(Block.RegtestGenesisBlock.hash, Alice.nodeParams.privateKey, randomKey().publicKey, ShortChannelId(45561L), Alice.nodeParams.channelConf.expiryDelta, Alice.nodeParams.channelConf.htlcMinimum, Alice.nodeParams.relayParams.publicChannelFees.feeBase, Alice.nodeParams.relayParams.publicChannelFees.feeProportionalMillionths, 500000000 msat)
assert(checkSig(ann, Alice.nodeParams.nodeId))
assert(checkSig(ann, randomKey().publicKey) === false)
}