1
0
mirror of https://github.com/ACINQ/eclair.git synced 2024-11-19 01:43:22 +01:00

Automatically disable from_future_htlc when abused (#2928)

When providing on-the-fly funding with the `from_future_htlc` payment
type, the liquidity provider is paying mining fees for the funding
transaction while trusting that the remote node will accept the HTLCs
afterwards and thus pay a liquidity fees. If the remote node fails the
HTLCs, the liquidity provider doesn't get paid. At that point it can
disable the channel and try to actively double-spend it. When we detect
such behavior, we immediately disable `from_future_htlc` to limit the
exposure to liquidity griefing: it can then be re-enabled by using the
`enableFromFutureHtlc` RPC, or will be automatically re-enabled if the
remote node fulfills the HTLCs after a retry.
This commit is contained in:
Bastien Teinturier 2024-10-15 11:18:33 +02:00 committed by GitHub
parent b8e6800e9d
commit e09c830f10
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
10 changed files with 190 additions and 11 deletions

View File

@ -21,7 +21,7 @@ _eclair-cli()
*)
# works fine, but is too slow at the moment.
# allopts=$($eclaircli help 2>&1 | awk '$1 ~ /^"/ { sub(/,/, ""); print $1}' | sed 's/[":]//g')
allopts="allchannels allupdates audit bumpforceclose channel channelbalances channels channelstats close closedchannels connect cpfpbumpfees createinvoice deleteinvoice disconnect findroute findroutebetweennodes findroutetonode forceclose getdescriptors getinfo getinvoice getmasterxpub getnewaddress getreceivedinfo getsentinfo globalbalance listinvoices listpendinginvoices listreceivedpayments networkfees node nodes onchainbalance onchaintransactions open parseinvoice payinvoice payoffer peers rbfopen sendonchain sendonionmessage sendtonode sendtoroute signmessage splicein spliceout stop updaterelayfee usablebalances verifymessage"
allopts="allchannels allupdates audit bumpforceclose channel channelbalances channels channelstats close closedchannels connect cpfpbumpfees createinvoice deleteinvoice disconnect enableFromFutureHtlc findroute findroutebetweennodes findroutetonode forceclose getdescriptors getinfo getinvoice getmasterxpub getnewaddress getreceivedinfo getsentinfo globalbalance listinvoices listpendinginvoices listreceivedpayments networkfees node nodes onchainbalance onchaintransactions open parseinvoice payinvoice payoffer peers rbfopen sendonchain sendonionmessage sendtonode sendtoroute signmessage splicein spliceout stop updaterelayfee usablebalances verifymessage"
if ! [[ " $allopts " =~ " $prev " ]]; then # prevent double arguments
if [[ -z "$cur" || "$cur" =~ ^[a-z] ]]; then

View File

@ -96,6 +96,9 @@ and COMMAND is one of the available commands:
- getmasterxpub
- getdescriptors
=== Control ===
- enablefromfuturehtlc
Examples
--------
eclair-cli -a localhost:1234 peers list the peers of a node hosted on localhost:1234

View File

@ -69,6 +69,8 @@ case class SendOnionMessageResponsePayload(tlvs: TlvStream[OnionMessagePayloadTl
case class SendOnionMessageResponse(sent: Boolean, failureMessage: Option[String], response: Option[SendOnionMessageResponsePayload])
// @formatter:on
case class EnableFromFutureHtlcResponse(enabled: Boolean, failureMessage: Option[String])
object SignedMessage {
def signedBytes(message: ByteVector): ByteVector32 =
Crypto.hash256(ByteVector("Lightning Signed Message:".getBytes(StandardCharsets.UTF_8)) ++ message)
@ -186,6 +188,8 @@ trait Eclair {
def getDescriptors(account: Long): Descriptors
def enableFromFutureHtlc(): Future[EnableFromFutureHtlcResponse]
def stop(): Future[Unit]
}
@ -781,6 +785,16 @@ class EclairImpl(appKit: Kit) extends Eclair with Logging {
case _ => throw new RuntimeException("on-chain seed is not configured")
}
override def enableFromFutureHtlc(): Future[EnableFromFutureHtlcResponse] = {
appKit.nodeParams.willFundRates_opt match {
case Some(willFundRates) if willFundRates.paymentTypes.contains(LiquidityAds.PaymentType.FromFutureHtlc) =>
appKit.nodeParams.onTheFlyFundingConfig.enableFromFutureHtlc()
Future.successful(EnableFromFutureHtlcResponse(appKit.nodeParams.onTheFlyFundingConfig.isFromFutureHtlcAllowed, None))
case _ =>
Future.successful(EnableFromFutureHtlcResponse(enabled = false, Some("could not enable from_future_htlc: you must add it to eclair.liquidity-ads.payment-types in your eclair.conf file first")))
}
}
override def stop(): Future[Unit] = {
// README: do not make this smarter or more complex !
// eclair can simply and cleanly be stopped by killing its process without fear of losing data, payments, ... and it should remain this way.

View File

@ -218,7 +218,7 @@ class Peer(val nodeParams: NodeParams,
case Event(SpawnChannelNonInitiator(open, channelConfig, channelType, addFunding_opt, localParams, peerConnection), d: ConnectedData) =>
val temporaryChannelId = open.fold(_.temporaryChannelId, _.temporaryChannelId)
if (peerConnection == d.peerConnection) {
OnTheFlyFunding.validateOpen(open, pendingOnTheFlyFunding, feeCredit.getOrElse(0 msat)) match {
OnTheFlyFunding.validateOpen(nodeParams.onTheFlyFundingConfig, open, pendingOnTheFlyFunding, feeCredit.getOrElse(0 msat)) match {
case reject: OnTheFlyFunding.ValidationResult.Reject =>
log.warning("rejecting on-the-fly channel: {}", reject.cancel.toAscii)
self ! Peer.OutgoingMessage(reject.cancel, d.peerConnection)
@ -387,7 +387,7 @@ class Peer(val nodeParams: NodeParams,
log.info("rejecting open_channel2: feerate too low ({} < {})", msg.feerate, d.currentFeerates.fundingFeerate)
self ! Peer.OutgoingMessage(TxAbort(msg.channelId, FundingFeerateTooLow(msg.channelId, msg.feerate, d.currentFeerates.fundingFeerate).getMessage), d.peerConnection)
case Some(channel) =>
OnTheFlyFunding.validateSplice(msg, nodeParams.channelConf.htlcMinimum, pendingOnTheFlyFunding, feeCredit.getOrElse(0 msat)) match {
OnTheFlyFunding.validateSplice(nodeParams.onTheFlyFundingConfig, msg, nodeParams.channelConf.htlcMinimum, pendingOnTheFlyFunding, feeCredit.getOrElse(0 msat)) match {
case reject: OnTheFlyFunding.ValidationResult.Reject =>
log.warning("rejecting on-the-fly splice: {}", reject.cancel.toAscii)
self ! Peer.OutgoingMessage(reject.cancel, d.peerConnection)
@ -689,6 +689,8 @@ class Peer(val nodeParams: NodeParams,
pendingOnTheFlyFunding -= success.paymentHash
case None => ()
}
// If this is a payment that was initially rejected, it wasn't a malicious node, but rather a temporary issue.
nodeParams.onTheFlyFundingConfig.fromFutureHtlcFulfilled(success.paymentHash)
stay()
case OnTheFlyFunding.PaymentRelayer.RelayFailed(paymentHash, failure) =>
log.warning("on-the-fly HTLC failure for payment_hash={}: {}", paymentHash, failure.toString)
@ -696,6 +698,16 @@ class Peer(val nodeParams: NodeParams,
// We don't give up yet by relaying the failure upstream: we may have simply been disconnected, or the added
// liquidity may have been consumed by concurrent HTLCs. We'll retry at the next reconnection with that peer
// or after the next splice, and will only give up when the outgoing will_add_htlc timeout.
val fundingStatus = pendingOnTheFlyFunding.get(paymentHash).map(_.status)
failure match {
case OnTheFlyFunding.PaymentRelayer.RemoteFailure(_) if fundingStatus.collect { case s: OnTheFlyFunding.Status.Funded => s.remainingFees }.sum > 0.msat =>
// We are still owed some fees for the funding transaction we published: we need these HTLCs to succeed.
// They received the HTLCs but failed them, which means that they're likely malicious (but not always,
// they may have other pending HTLCs that temporarily prevent relaying the whole HTLC set because of
// channel limits). We disable funding from future HTLCs to limit our exposure to fee siphoning.
nodeParams.onTheFlyFundingConfig.fromFutureHtlcFailed(paymentHash, remoteNodeId)
case _ => ()
}
stay()
}

View File

@ -34,6 +34,7 @@ object Monitoring {
val SentPaymentDuration = Kamon.timer("payment.duration.sent", "Outgoing payment duration")
val ReceivedPaymentDuration = Kamon.timer("payment.duration.received", "Incoming payment duration")
val RelayedPaymentDuration = Kamon.timer("payment.duration.relayed", "Duration of pending downstream HTLCs during a relay")
val SuspiciousFromFutureHtlcRelays = Kamon.gauge("payment.on-the-fly-funding.suspicious-htlc-relays", "Number of pending on-the-fly HTLCs that are being rejected by seemingly malicious peers")
// The goal of this metric is to measure whether retrying MPP payments on failing channels yields useful results.
// Once enough data has been collected, we will update the MultiPartPaymentLifecycle logic accordingly.

View File

@ -25,6 +25,7 @@ import fr.acinq.bitcoin.scalacompat.{ByteVector32, Crypto, TxId}
import fr.acinq.eclair.blockchain.fee.FeeratePerKw
import fr.acinq.eclair.channel._
import fr.acinq.eclair.crypto.Sphinx
import fr.acinq.eclair.payment.Monitoring.Metrics
import fr.acinq.eclair.wire.protocol.LiquidityAds.PaymentDetails
import fr.acinq.eclair.wire.protocol._
import fr.acinq.eclair.{Logs, MilliSatoshi, MilliSatoshiLong, NodeParams, TimestampMilli, ToMilliSatoshiConversion}
@ -38,7 +39,38 @@ import scala.concurrent.duration.FiniteDuration
object OnTheFlyFunding {
case class Config(proposalTimeout: FiniteDuration)
case class Config(proposalTimeout: FiniteDuration) {
// When funding a transaction using from_future_htlc, we are taking the risk that the remote node doesn't fulfill
// the corresponding HTLCs. If we detect that our peer fails such HTLCs, we automatically disable from_future_htlc
// to limit our exposure.
// Note that this state is flushed when restarting: node operators should explicitly remove the from_future_htlc
// payment type from their liquidity ads configuration if they want to keep it disabled.
private val suspectFromFutureHtlcRelays = scala.collection.concurrent.TrieMap.empty[ByteVector32, PublicKey]
/** We allow using from_future_htlc if we don't have any pending payment that is abusing it. */
def isFromFutureHtlcAllowed: Boolean = suspectFromFutureHtlcRelays.isEmpty
/** An on-the-fly payment using from_future_htlc was failed by the remote node: they may be malicious. */
def fromFutureHtlcFailed(paymentHash: ByteVector32, remoteNodeId: PublicKey): Unit = {
suspectFromFutureHtlcRelays.addOne(paymentHash, remoteNodeId)
Metrics.SuspiciousFromFutureHtlcRelays.withoutTags().update(suspectFromFutureHtlcRelays.size)
}
/** If a fishy payment is fulfilled, we remove it from the list, which may re-enabled from_future_htlc. */
def fromFutureHtlcFulfilled(paymentHash: ByteVector32): Unit = {
suspectFromFutureHtlcRelays.remove(paymentHash).foreach { _ =>
// We only need to update the metric if an entry was actually removed.
Metrics.SuspiciousFromFutureHtlcRelays.withoutTags().update(suspectFromFutureHtlcRelays.size)
}
}
/** Remove all suspect payments and re-enable from_future_htlc. */
def enableFromFutureHtlc(): Unit = {
val pending = suspectFromFutureHtlcRelays.toList.map(_._1)
pending.foreach(paymentHash => suspectFromFutureHtlcRelays.remove(paymentHash))
Metrics.SuspiciousFromFutureHtlcRelays.withoutTags().update(0)
}
}
// @formatter:off
sealed trait Status
@ -114,25 +146,26 @@ object OnTheFlyFunding {
// @formatter:on
/** Validate an incoming channel that may use on-the-fly funding. */
def validateOpen(open: Either[OpenChannel, OpenDualFundedChannel], pendingOnTheFlyFunding: Map[ByteVector32, Pending], feeCredit: MilliSatoshi): ValidationResult = {
def validateOpen(cfg: Config, open: Either[OpenChannel, OpenDualFundedChannel], pendingOnTheFlyFunding: Map[ByteVector32, Pending], feeCredit: MilliSatoshi): ValidationResult = {
open match {
case Left(_) => ValidationResult.Accept(Set.empty, None)
case Right(open) => open.requestFunding_opt match {
case Some(requestFunding) => validate(open.temporaryChannelId, requestFunding, isChannelCreation = true, open.fundingFeerate, open.htlcMinimum, pendingOnTheFlyFunding, feeCredit)
case Some(requestFunding) => validate(cfg, open.temporaryChannelId, requestFunding, isChannelCreation = true, open.fundingFeerate, open.htlcMinimum, pendingOnTheFlyFunding, feeCredit)
case None => ValidationResult.Accept(Set.empty, None)
}
}
}
/** Validate an incoming splice that may use on-the-fly funding. */
def validateSplice(splice: SpliceInit, htlcMinimum: MilliSatoshi, pendingOnTheFlyFunding: Map[ByteVector32, Pending], feeCredit: MilliSatoshi): ValidationResult = {
def validateSplice(cfg: Config, splice: SpliceInit, htlcMinimum: MilliSatoshi, pendingOnTheFlyFunding: Map[ByteVector32, Pending], feeCredit: MilliSatoshi): ValidationResult = {
splice.requestFunding_opt match {
case Some(requestFunding) => validate(splice.channelId, requestFunding, isChannelCreation = false, splice.feerate, htlcMinimum, pendingOnTheFlyFunding, feeCredit)
case Some(requestFunding) => validate(cfg, splice.channelId, requestFunding, isChannelCreation = false, splice.feerate, htlcMinimum, pendingOnTheFlyFunding, feeCredit)
case None => ValidationResult.Accept(Set.empty, None)
}
}
private def validate(channelId: ByteVector32,
private def validate(cfg: Config,
channelId: ByteVector32,
requestFunding: LiquidityAds.RequestFunding,
isChannelCreation: Boolean,
feerate: FeeratePerKw,
@ -159,10 +192,12 @@ object OnTheFlyFunding {
}
val cancelAmountTooLow = CancelOnTheFlyFunding(channelId, paymentHashes, s"requested amount is too low to relay HTLCs: ${requestFunding.requestedAmount} < $totalPaymentAmount")
val cancelFeesTooLow = CancelOnTheFlyFunding(channelId, paymentHashes, s"htlc amount is too low to pay liquidity fees: $availableAmountForFees < $feesOwed")
val cancelDisabled = CancelOnTheFlyFunding(channelId, paymentHashes, "payments paid with future HTLCs are currently disabled")
requestFunding.paymentDetails match {
case PaymentDetails.FromChannelBalance => ValidationResult.Accept(Set.empty, None)
case _ if requestFunding.requestedAmount.toMilliSatoshi < totalPaymentAmount => ValidationResult.Reject(cancelAmountTooLow, paymentHashes.toSet)
case _: PaymentDetails.FromChannelBalanceForFutureHtlc => ValidationResult.Accept(Set.empty, useFeeCredit_opt)
case _: PaymentDetails.FromFutureHtlc if !cfg.isFromFutureHtlcAllowed => ValidationResult.Reject(cancelDisabled, paymentHashes.toSet)
case _: PaymentDetails.FromFutureHtlc if availableAmountForFees < feesOwed => ValidationResult.Reject(cancelFeesTooLow, paymentHashes.toSet)
case _: PaymentDetails.FromFutureHtlc => ValidationResult.Accept(Set.empty, useFeeCredit_opt)
case _: PaymentDetails.FromFutureHtlcWithPreimage if availableAmountForFees < feesOwed => ValidationResult.Reject(cancelFeesTooLow, paymentHashes.toSet)

View File

@ -36,6 +36,7 @@ import fr.acinq.eclair.wire.internal.channel.ChannelCodecsSpec
import fr.acinq.eclair.wire.internal.channel.ChannelCodecsSpec.localParams
import fr.acinq.eclair.wire.protocol
import fr.acinq.eclair.wire.protocol._
import org.scalatest.Inside.inside
import org.scalatest.{Tag, TestData}
import scodec.bits.ByteVector
@ -728,6 +729,29 @@ class PeerSpec extends FixtureSpec {
probe.expectTerminated(peer)
}
test("reject on-the-fly funding requests when from_future_htlc is disabled", Tag(ChannelStateTestsTags.DualFunding)) { f =>
import f._
// We make sure that from_future_htlc is disabled.
nodeParams.onTheFlyFundingConfig.fromFutureHtlcFailed(randomBytes32(), randomKey().publicKey)
assert(!nodeParams.onTheFlyFundingConfig.isFromFutureHtlcAllowed)
// We reject requests using from_future_htlc.
val paymentHash = randomBytes32()
connect(remoteNodeId, peer, peerConnection, switchboard, remoteInit = protocol.Init(Features(StaticRemoteKey -> Optional, AnchorOutputsZeroFeeHtlcTx -> Optional, DualFunding -> Optional)))
val requestFunds = LiquidityAds.RequestFunding(50_000 sat, LiquidityAds.FundingRate(10_000 sat, 100_000 sat, 0, 0, 0 sat, 0 sat), LiquidityAds.PaymentDetails.FromFutureHtlc(paymentHash :: Nil))
val open = inside(createOpenDualFundedChannelMessage()) { msg => msg.copy(tlvStream = TlvStream(ChannelTlv.RequestFundingTlv(requestFunds))) }
peerConnection.send(peer, open)
peerConnection.expectMsg(CancelOnTheFlyFunding(open.temporaryChannelId, paymentHash :: Nil, "payments paid with future HTLCs are currently disabled"))
channel.expectNoMessage(100 millis)
// Once enabled, we accept requests using from_future_htlc.
nodeParams.onTheFlyFundingConfig.enableFromFutureHtlc()
peerConnection.send(peer, open)
channel.expectMsgType[INPUT_INIT_CHANNEL_NON_INITIATOR]
channel.expectMsg(open)
}
}
object PeerSpec {

View File

@ -34,6 +34,7 @@ import fr.acinq.eclair.wire.protocol._
import fr.acinq.eclair.{Alias, BlockHeight, CltvExpiry, CltvExpiryDelta, FeatureSupport, Features, MilliSatoshi, MilliSatoshiLong, NodeParams, TestConstants, TestKitBaseClass, TimestampMilli, ToMilliSatoshiConversion, UInt64, randomBytes, randomBytes32, randomKey, randomLong}
import org.scalatest.funsuite.FixtureAnyFunSuiteLike
import org.scalatest.{Outcome, Tag}
import scodec.bits.ByteVector
import java.util.UUID
import scala.concurrent.duration.DurationInt
@ -1013,6 +1014,33 @@ class OnTheFlyFundingSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike {
awaitCond(nodeParams.db.liquidity.getFeeCredit(remoteNodeId) == 0.msat, interval = 100 millis)
}
test("disable from_future_htlc when remote rejects HTLCs") { f =>
import f._
connect(peer)
val preimage = randomBytes32()
val paymentHash = Crypto.sha256(preimage)
val upstream = upstreamChannel(11_000_000 msat, expiryIn, paymentHash)
proposeFunding(10_000_000 msat, expiryOut, paymentHash, upstream)
val fees = LiquidityAds.Fees(10_000 sat, 5_000 sat)
val purchase = signLiquidityPurchase(200_000 sat, LiquidityAds.PaymentDetails.FromFutureHtlc(List(paymentHash)), fees = fees)
// Once the channel is ready to relay payments, we forward HTLCs matching the proposed will_add_htlc.
peer ! ChannelReadyForPayments(channel.ref, remoteNodeId, purchase.channelId, fundingTxIndex = 0)
channel.expectMsgType[CMD_GET_CHANNEL_INFO]
// Our peer rejects the HTLC, so we automatically disable from_future_htlc.
assert(nodeParams.onTheFlyFundingConfig.isFromFutureHtlcAllowed)
val failure = HtlcResult.RemoteFail(UpdateFailHtlc(purchase.channelId, 2, ByteVector.empty))
peer ! OnTheFlyFunding.PaymentRelayer.RelayFailed(paymentHash, OnTheFlyFunding.PaymentRelayer.RemoteFailure(failure))
awaitCond(!nodeParams.onTheFlyFundingConfig.isFromFutureHtlcAllowed)
// When we retry relaying the HTLC, our peer fulfills it: we re-enable from_future_htlc.
peer ! OnTheFlyFunding.PaymentRelayer.RelaySuccess(purchase.channelId, paymentHash, preimage, fees.total.toMilliSatoshi)
awaitCond(nodeParams.onTheFlyFundingConfig.isFromFutureHtlcAllowed)
}
test("don't relay payments if added to fee credit while signing", Tag(withFeeCredit)) { f =>
import f._
@ -1156,6 +1184,34 @@ class OnTheFlyFundingSpec extends TestKitBaseClass with FixtureAnyFunSuiteLike {
probe.expectNoMessage(100 millis)
}
test("disable from_future_htlc when detecting abuse") { () =>
val cfg = OnTheFlyFunding.Config(90 seconds)
assert(cfg.isFromFutureHtlcAllowed)
val remoteNodeId = randomKey().publicKey
// We detect two payments that seem malicious.
val paymentHash1 = randomBytes32()
val paymentHash2 = randomBytes32()
cfg.fromFutureHtlcFailed(paymentHash1, remoteNodeId)
assert(!cfg.isFromFutureHtlcAllowed)
cfg.fromFutureHtlcFailed(paymentHash1, remoteNodeId) // noop
cfg.fromFutureHtlcFailed(paymentHash2, remoteNodeId)
assert(!cfg.isFromFutureHtlcAllowed)
// The first one wasn't malicious after all.
cfg.fromFutureHtlcFulfilled(paymentHash1)
assert(!cfg.isFromFutureHtlcAllowed)
// The second one wasn't malicious either: we reactivate from_future_htlc.
cfg.fromFutureHtlcFulfilled(paymentHash2)
assert(cfg.isFromFutureHtlcAllowed)
// We detect a bunch of potentially malicious payments but manually reactivate from_future_htlc.
cfg.fromFutureHtlcFailed(randomBytes32(), remoteNodeId)
cfg.fromFutureHtlcFailed(randomBytes32(), remoteNodeId)
assert(!cfg.isFromFutureHtlcAllowed)
cfg.enableFromFutureHtlc()
assert(cfg.isFromFutureHtlcAllowed)
}
}
object OnTheFlyFundingSpec {

View File

@ -23,7 +23,7 @@ import fr.acinq.eclair.api.directives.EclairDirectives
import fr.acinq.eclair.api.handlers._
import grizzled.slf4j.Logging
trait Service extends EclairDirectives with WebSocket with Node with Channel with Fees with PathFinding with Invoice with Payment with Message with OnChain with Logging {
trait Service extends EclairDirectives with WebSocket with Node with Control with Channel with Fees with PathFinding with Invoice with Payment with Message with OnChain with Logging {
/**
* Allows router access to the API password as configured in eclair.conf
@ -46,7 +46,7 @@ trait Service extends EclairDirectives with WebSocket with Node with Channel wit
* This is where we handle errors to ensure all routes are correctly tried before rejecting.
*/
def finalRoutes(extraRouteProviders: Seq[RouteProvider] = Nil): Route = securedHandler {
val baseRoutes = nodeRoutes ~ channelRoutes ~ feeRoutes ~ pathFindingRoutes ~ invoiceRoutes ~ paymentRoutes ~ messageRoutes ~ onChainRoutes ~ webSocket
val baseRoutes = nodeRoutes ~ controlRoutes ~ channelRoutes ~ feeRoutes ~ pathFindingRoutes ~ invoiceRoutes ~ paymentRoutes ~ messageRoutes ~ onChainRoutes ~ webSocket
extraRouteProviders.map(_.route(this)).foldLeft(baseRoutes)(_ ~ _)
}
}

View File

@ -0,0 +1,34 @@
/*
* Copyright 2024 ACINQ SAS
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package fr.acinq.eclair.api.handlers
import akka.http.scaladsl.server.Route
import fr.acinq.eclair.api.Service
import fr.acinq.eclair.api.directives.EclairDirectives
trait Control {
this: Service with EclairDirectives =>
import fr.acinq.eclair.api.serde.JsonSupport.{formats, marshaller, serialization}
val enableFromFutureHtlc: Route = postRequest("enablefromfuturehtlc") { implicit t =>
complete(eclairApi.enableFromFutureHtlc())
}
val controlRoutes: Route = enableFromFutureHtlc
}