From 83dc8176a9f0b7222085fb4ddf8ffc95ba6ba471 Mon Sep 17 00:00:00 2001 From: araspitzu Date: Fri, 26 Apr 2019 17:10:30 +0200 Subject: [PATCH] Various fix and improvements in time/timestamp handling (#971) This PR standardizes the way we compute the current time as unix timestamp - Scala's Platform is used and the conversion is done via scala's concurrent.duration facilities - Java's Instant has been replaced due to broken compatibility with android - AuditDB events use milliseconds (fixes #970) - PaymentDB events use milliseconds - Query filters for AuditDB and PaymentDB use seconds --- .../fr/acinq/eclair/channel/Channel.scala | 8 ++++---- .../eclair/db/sqlite/SqliteAuditDb.scala | 19 +++++++++--------- .../eclair/db/sqlite/SqlitePaymentsDb.scala | 15 +++++++------- .../eclair/payment/LocalPaymentHandler.scala | 4 ++-- .../acinq/eclair/payment/PaymentEvents.scala | 10 +++++----- .../eclair/payment/PaymentLifecycle.scala | 6 +++--- .../acinq/eclair/router/Announcements.scala | 7 +++---- .../scala/fr/acinq/eclair/router/Router.scala | 6 +++--- .../fr/acinq/eclair/wire/ChannelCodecs.scala | 4 +++- .../acinq/eclair/db/SqliteAuditDbSpec.scala | 20 +++++++++---------- .../eclair/db/SqlitePaymentsDbSpec.scala | 18 ++++++++--------- .../fr/acinq/eclair/router/RouterSpec.scala | 4 ++-- .../acinq/eclair/wire/ChannelCodecsSpec.scala | 3 ++- 13 files changed, 63 insertions(+), 61 deletions(-) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala index 01bf73a84..bc4277500 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/channel/Channel.scala @@ -228,7 +228,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId // this is a bit tricky: let's say we shut down eclair right after someone opened a channel to us, and didn't start it up before a very long time // we don't want the timeout to expire right away, because the watcher could be syncing or be busy, and may only notice the funding tx after some time // so we always give us 10 minutes before doing anything - val delay = Funding.computeFundingTimeout(Platform.currentTime / 1000, funding.waitingSince, delay = FUNDING_TIMEOUT_FUNDEE, minDelay = 10 minutes) + val delay = Funding.computeFundingTimeout(Platform.currentTime.milliseconds.toSeconds, funding.waitingSince, delay = FUNDING_TIMEOUT_FUNDEE, minDelay = 10 minutes) context.system.scheduler.scheduleOnce(delay, self, BITCOIN_FUNDING_TIMEOUT) } goto(OFFLINE) using data @@ -416,7 +416,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId log.info(s"waiting for them to publish the funding tx for channelId=$channelId fundingTxid=${commitInput.outPoint.txid}") blockchain ! WatchSpent(self, commitInput.outPoint.txid, commitInput.outPoint.index.toInt, commitments.commitInput.txOut.publicKeyScript, BITCOIN_FUNDING_SPENT) // TODO: should we wait for an acknowledgment from the watcher? blockchain ! WatchConfirmed(self, commitInput.outPoint.txid, commitments.commitInput.txOut.publicKeyScript, nodeParams.minDepthBlocks, BITCOIN_FUNDING_DEPTHOK) - val now = Platform.currentTime / 1000 + val now = Platform.currentTime.milliseconds.toSeconds context.system.scheduler.scheduleOnce(FUNDING_TIMEOUT_FUNDEE, self, BITCOIN_FUNDING_TIMEOUT) goto(WAIT_FOR_FUNDING_CONFIRMED) using store(DATA_WAIT_FOR_FUNDING_CONFIRMED(commitments, None, now, None, Right(fundingSigned))) sending fundingSigned } @@ -448,7 +448,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId originChannels = Map.empty, remoteNextCommitInfo = Right(randomKey.publicKey), // we will receive their next per-commitment point in the next message, so we temporarily put a random byte array commitInput, ShaChain.init, channelId = channelId) - val now = Platform.currentTime / 1000 + val now = Platform.currentTime.milliseconds.toSeconds context.system.eventStream.publish(ChannelSignatureReceived(self, commitments)) log.info(s"publishing funding tx for channelId=$channelId fundingTxid=${commitInput.outPoint.txid}") // we do this to make sure that the channel state has been written to disk when we publish the funding tx @@ -1467,7 +1467,7 @@ class Channel(val nodeParams: NodeParams, val wallet: EclairWallet, remoteNodeId } } // re-enable the channel - val timestamp = Platform.currentTime / 1000 match { + val timestamp = Platform.currentTime.milliseconds.toSeconds match { case ts if ts == d.channelUpdate.timestamp => ts + 1 // corner case: in case of quick reconnection, we bump the timestamp of the new channel_update, otherwise it will get ignored by the network case ts => ts } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteAuditDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteAuditDb.scala index baa24fb0f..c96da5470 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteAuditDb.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqliteAuditDb.scala @@ -18,7 +18,6 @@ package fr.acinq.eclair.db.sqlite import java.sql.{Connection, Statement} import java.util.UUID - import fr.acinq.bitcoin.Crypto.PublicKey import fr.acinq.bitcoin.MilliSatoshi import fr.acinq.eclair.channel.{AvailableBalanceChanged, Channel, ChannelErrorOccured, NetworkFeePaid} @@ -26,9 +25,9 @@ import fr.acinq.eclair.db.{AuditDb, ChannelLifecycleEvent, NetworkFee, Stats} import fr.acinq.eclair.payment.{PaymentReceived, PaymentRelayed, PaymentSent} import fr.acinq.eclair.wire.ChannelCodecs import grizzled.slf4j.Logging - import scala.collection.immutable.Queue import scala.compat.Platform +import concurrent.duration._ class SqliteAuditDb(sqlite: Connection) extends AuditDb with Logging { @@ -164,8 +163,8 @@ class SqliteAuditDb(sqlite: Connection) extends AuditDb with Logging { override def listSent(from: Long, to: Long): Seq[PaymentSent] = using(sqlite.prepareStatement("SELECT * FROM sent WHERE timestamp >= ? AND timestamp < ?")) { statement => - statement.setLong(1, from) - statement.setLong(2, to) + statement.setLong(1, from.seconds.toMillis) + statement.setLong(2, to.seconds.toMillis) val rs = statement.executeQuery() var q: Queue[PaymentSent] = Queue() while (rs.next()) { @@ -183,8 +182,8 @@ class SqliteAuditDb(sqlite: Connection) extends AuditDb with Logging { override def listReceived(from: Long, to: Long): Seq[PaymentReceived] = using(sqlite.prepareStatement("SELECT * FROM received WHERE timestamp >= ? AND timestamp < ?")) { statement => - statement.setLong(1, from) - statement.setLong(2, to) + statement.setLong(1, from.seconds.toMillis) + statement.setLong(2, to.seconds.toMillis) val rs = statement.executeQuery() var q: Queue[PaymentReceived] = Queue() while (rs.next()) { @@ -199,8 +198,8 @@ class SqliteAuditDb(sqlite: Connection) extends AuditDb with Logging { override def listRelayed(from: Long, to: Long): Seq[PaymentRelayed] = using(sqlite.prepareStatement("SELECT * FROM relayed WHERE timestamp >= ? AND timestamp < ?")) { statement => - statement.setLong(1, from) - statement.setLong(2, to) + statement.setLong(1, from.seconds.toMillis) + statement.setLong(2, to.seconds.toMillis) val rs = statement.executeQuery() var q: Queue[PaymentRelayed] = Queue() while (rs.next()) { @@ -217,8 +216,8 @@ class SqliteAuditDb(sqlite: Connection) extends AuditDb with Logging { override def listNetworkFees(from: Long, to: Long): Seq[NetworkFee] = using(sqlite.prepareStatement("SELECT * FROM network_fees WHERE timestamp >= ? AND timestamp < ?")) { statement => - statement.setLong(1, from) - statement.setLong(2, to) + statement.setLong(1, from.seconds.toMillis) + statement.setLong(2, to.seconds.toMillis) val rs = statement.executeQuery() var q: Queue[NetworkFee] = Queue() while (rs.next()) { diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqlitePaymentsDb.scala b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqlitePaymentsDb.scala index 5860402e0..7e7dc44d9 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqlitePaymentsDb.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/db/sqlite/SqlitePaymentsDb.scala @@ -17,7 +17,6 @@ package fr.acinq.eclair.db.sqlite import java.sql.Connection -import java.time.Instant import java.util.UUID import fr.acinq.bitcoin.ByteVector32 import fr.acinq.eclair.db.sqlite.SqliteUtils._ @@ -26,6 +25,8 @@ import fr.acinq.eclair.payment.PaymentRequest import grizzled.slf4j.Logging import scala.collection.immutable.Queue import OutgoingPaymentStatus._ +import concurrent.duration._ +import scala.compat.Platform class SqlitePaymentsDb(sqlite: Connection) extends PaymentsDb with Logging { @@ -58,7 +59,7 @@ class SqlitePaymentsDb(sqlite: Connection) extends PaymentsDb with Logging { require((newStatus == SUCCEEDED && preimage.isDefined) || (newStatus == FAILED && preimage.isEmpty), "Wrong combination of state/preimage") using(sqlite.prepareStatement("UPDATE sent_payments SET (completed_at, preimage, status) = (?, ?, ?) WHERE id = ? AND completed_at IS NULL")) { statement => - statement.setLong(1, Instant.now().getEpochSecond) + statement.setLong(1, Platform.currentTime) statement.setBytes(2, if (preimage.isEmpty) null else preimage.get.toArray) statement.setString(3, newStatus.toString) statement.setString(4, id.toString) @@ -135,8 +136,8 @@ class SqlitePaymentsDb(sqlite: Connection) extends PaymentsDb with Logging { statement.setBytes(1, pr.paymentHash.toArray) statement.setBytes(2, preimage.toArray) statement.setString(3, PaymentRequest.write(pr)) - statement.setLong(4, pr.timestamp) - pr.expiry.foreach { ex => statement.setLong(5, pr.timestamp + ex) } // we store "when" the invoice will expire + statement.setLong(4, pr.timestamp.seconds.toMillis) // BOLT11 timestamp is in seconds + pr.expiry.foreach { ex => statement.setLong(5, pr.timestamp.seconds.toMillis + ex.seconds.toMillis) } // we store "when" the invoice will expire, in milliseconds statement.executeUpdate() } } @@ -178,9 +179,9 @@ class SqlitePaymentsDb(sqlite: Connection) extends PaymentsDb with Logging { } using(sqlite.prepareStatement(queryStmt)) { statement => - statement.setLong(1, from) - statement.setLong(2, to) - if (pendingOnly) statement.setLong(3, Instant.now().getEpochSecond) + statement.setLong(1, from.seconds.toMillis) + statement.setLong(2, to.seconds.toMillis) + if (pendingOnly) statement.setLong(3, Platform.currentTime) val rs = statement.executeQuery() var q: Queue[PaymentRequest] = Queue() diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/LocalPaymentHandler.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/LocalPaymentHandler.scala index 985e38161..ff1ffa138 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/LocalPaymentHandler.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/LocalPaymentHandler.scala @@ -23,7 +23,7 @@ import fr.acinq.eclair.db.IncomingPayment import fr.acinq.eclair.payment.PaymentLifecycle.ReceivePayment import fr.acinq.eclair.wire._ import fr.acinq.eclair.{Globals, NodeParams, randomBytes32} - +import concurrent.duration._ import scala.compat.Platform import scala.concurrent.ExecutionContext import scala.util.{Failure, Success, Try} @@ -75,7 +75,7 @@ class LocalPaymentHandler(nodeParams: NodeParams) extends Actor with ActorLoggin case _ => log.info(s"received payment for paymentHash=${htlc.paymentHash} amountMsat=${htlc.amountMsat}") // amount is correct or was not specified in the payment request - nodeParams.db.payments.addIncomingPayment(IncomingPayment(htlc.paymentHash, htlc.amountMsat, Platform.currentTime / 1000)) + nodeParams.db.payments.addIncomingPayment(IncomingPayment(htlc.paymentHash, htlc.amountMsat, Platform.currentTime)) sender ! CMD_FULFILL_HTLC(htlc.id, paymentPreimage, commit = true) context.system.eventStream.publish(PaymentReceived(MilliSatoshi(htlc.amountMsat), htlc.paymentHash, htlc.channelId)) } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentEvents.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentEvents.scala index fe0289f72..1bb68fd7c 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentEvents.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentEvents.scala @@ -16,9 +16,9 @@ package fr.acinq.eclair.payment -import java.time.Instant import java.util.UUID import fr.acinq.bitcoin.{ByteVector32, MilliSatoshi} +import scala.compat.Platform /** * Created by PM on 01/02/2017. @@ -27,10 +27,10 @@ sealed trait PaymentEvent { val paymentHash: ByteVector32 } -case class PaymentSent(id: UUID, amount: MilliSatoshi, feesPaid: MilliSatoshi, paymentHash: ByteVector32, paymentPreimage: ByteVector32, toChannelId: ByteVector32, timestamp: Long = Instant.now().getEpochSecond) extends PaymentEvent +case class PaymentSent(id: UUID, amount: MilliSatoshi, feesPaid: MilliSatoshi, paymentHash: ByteVector32, paymentPreimage: ByteVector32, toChannelId: ByteVector32, timestamp: Long = Platform.currentTime) extends PaymentEvent -case class PaymentRelayed(amountIn: MilliSatoshi, amountOut: MilliSatoshi, paymentHash: ByteVector32, fromChannelId: ByteVector32, toChannelId: ByteVector32, timestamp: Long = Instant.now().getEpochSecond) extends PaymentEvent +case class PaymentRelayed(amountIn: MilliSatoshi, amountOut: MilliSatoshi, paymentHash: ByteVector32, fromChannelId: ByteVector32, toChannelId: ByteVector32, timestamp: Long = Platform.currentTime) extends PaymentEvent -case class PaymentReceived(amount: MilliSatoshi, paymentHash: ByteVector32, fromChannelId: ByteVector32, timestamp: Long = Instant.now().getEpochSecond) extends PaymentEvent +case class PaymentReceived(amount: MilliSatoshi, paymentHash: ByteVector32, fromChannelId: ByteVector32, timestamp: Long = Platform.currentTime) extends PaymentEvent -case class PaymentSettlingOnChain(id: UUID, amount: MilliSatoshi, paymentHash: ByteVector32, timestamp: Long = Instant.now().getEpochSecond) extends PaymentEvent +case class PaymentSettlingOnChain(id: UUID, amount: MilliSatoshi, paymentHash: ByteVector32, timestamp: Long = Platform.currentTime) extends PaymentEvent diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentLifecycle.scala b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentLifecycle.scala index ea64a20ba..c292c3803 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentLifecycle.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/payment/PaymentLifecycle.scala @@ -16,9 +16,7 @@ package fr.acinq.eclair.payment -import java.time.Instant import java.util.UUID - import akka.actor.{ActorRef, FSM, Props, Status} import fr.acinq.bitcoin.Crypto.PublicKey import fr.acinq.bitcoin.{ByteVector32, MilliSatoshi} @@ -33,6 +31,8 @@ import fr.acinq.eclair.router._ import fr.acinq.eclair.wire._ import scodec.Attempt import scodec.bits.ByteVector +import concurrent.duration._ +import scala.compat.Platform import scala.util.{Failure, Success} /** @@ -47,7 +47,7 @@ class PaymentLifecycle(nodeParams: NodeParams, id: UUID, router: ActorRef, regis when(WAITING_FOR_REQUEST) { case Event(c: SendPayment, WaitingForRequest) => router ! RouteRequest(nodeParams.nodeId, c.targetNodeId, c.amountMsat, c.assistedRoutes, routeParams = c.routeParams) - paymentsDb.addOutgoingPayment(OutgoingPayment(id, c.paymentHash, None, c.amountMsat, Instant.now().getEpochSecond, None, OutgoingPaymentStatus.PENDING)) + paymentsDb.addOutgoingPayment(OutgoingPayment(id, c.paymentHash, None, c.amountMsat, Platform.currentTime, None, OutgoingPaymentStatus.PENDING)) goto(WAITING_FOR_ROUTE) using WaitingForRoute(sender, c, failures = Nil) } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/router/Announcements.scala b/eclair-core/src/main/scala/fr/acinq/eclair/router/Announcements.scala index ab0488195..7e3e0b7f2 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/router/Announcements.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/router/Announcements.scala @@ -22,10 +22,9 @@ import fr.acinq.eclair.wire._ import fr.acinq.eclair.{ShortChannelId, serializationResult} import scodec.bits.{BitVector, ByteVector} import shapeless.HNil - +import scala.concurrent.duration._ import scala.compat.Platform - /** * Created by PM on 03/02/2017. */ @@ -73,7 +72,7 @@ object Announcements { ) } - def makeNodeAnnouncement(nodeSecret: PrivateKey, alias: String, color: Color, nodeAddresses: List[NodeAddress], timestamp: Long = Platform.currentTime / 1000): NodeAnnouncement = { + def makeNodeAnnouncement(nodeSecret: PrivateKey, alias: String, color: Color, nodeAddresses: List[NodeAddress], timestamp: Long = Platform.currentTime.milliseconds.toSeconds): NodeAnnouncement = { require(alias.size <= 32) val witness = nodeAnnouncementWitnessEncode(timestamp, nodeSecret.publicKey, color, alias, ByteVector.empty, nodeAddresses) val sig = Crypto.encodeSignature(Crypto.sign(witness, nodeSecret)) :+ 1.toByte @@ -119,7 +118,7 @@ object Announcements { def makeChannelFlags(isNode1: Boolean, enable: Boolean): Byte = BitVector.bits(!enable :: !isNode1 :: Nil).padLeft(8).toByte() - def makeChannelUpdate(chainHash: ByteVector32, nodeSecret: PrivateKey, remoteNodeId: PublicKey, shortChannelId: ShortChannelId, cltvExpiryDelta: Int, htlcMinimumMsat: Long, feeBaseMsat: Long, feeProportionalMillionths: Long, htlcMaximumMsat: Long, enable: Boolean = true, timestamp: Long = Platform.currentTime / 1000): ChannelUpdate = { + def makeChannelUpdate(chainHash: ByteVector32, nodeSecret: PrivateKey, remoteNodeId: PublicKey, shortChannelId: ShortChannelId, cltvExpiryDelta: Int, htlcMinimumMsat: Long, feeBaseMsat: Long, feeProportionalMillionths: Long, htlcMaximumMsat: Long, enable: Boolean = true, timestamp: Long = Platform.currentTime.milliseconds.toSeconds): ChannelUpdate = { val messageFlags = makeMessageFlags(hasOptionChannelHtlcMax = true) // NB: we always support option_channel_htlc_max val channelFlags = makeChannelFlags(isNode1 = isNode1(nodeSecret.publicKey, remoteNodeId), enable = enable) val htlcMaximumMsatOpt = Some(htlcMaximumMsat) diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/router/Router.scala b/eclair-core/src/main/scala/fr/acinq/eclair/router/Router.scala index e08a8e0e5..126c5d0a3 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/router/Router.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/router/Router.scala @@ -440,7 +440,7 @@ class Router(nodeParams: NodeParams, watcher: ActorRef, initialized: Option[Prom // the first_timestamp field to the current date/time and timestamp_range to the maximum value // NB: we can't just set firstTimestamp to 0, because in that case peer would send us all past messages matching // that (i.e. the whole routing table) - val filter = GossipTimestampFilter(nodeParams.chainHash, firstTimestamp = Platform.currentTime / 1000, timestampRange = Int.MaxValue) + val filter = GossipTimestampFilter(nodeParams.chainHash, firstTimestamp = Platform.currentTime.milliseconds.toSeconds, timestampRange = Int.MaxValue) remote ! filter // clean our sync state for this peer: we receive a SendChannelQuery just when we connect/reconnect to a peer and @@ -738,7 +738,7 @@ object Router { def toFakeUpdate(extraHop: ExtraHop): ChannelUpdate = // the `direction` bit in flags will not be accurate but it doesn't matter because it is not used // what matters is that the `disable` bit is 0 so that this update doesn't get filtered out - ChannelUpdate(signature = ByteVector.empty, chainHash = ByteVector32.Zeroes, extraHop.shortChannelId, Platform.currentTime / 1000, messageFlags = 0, channelFlags = 0, extraHop.cltvExpiryDelta, htlcMinimumMsat = 0L, extraHop.feeBaseMsat, extraHop.feeProportionalMillionths, None) + ChannelUpdate(signature = ByteVector.empty, chainHash = ByteVector32.Zeroes, extraHop.shortChannelId, Platform.currentTime.milliseconds.toSeconds, messageFlags = 0, channelFlags = 0, extraHop.cltvExpiryDelta, htlcMinimumMsat = 0L, extraHop.feeBaseMsat, extraHop.feeProportionalMillionths, None) def toFakeUpdates(extraRoute: Seq[ExtraHop], targetNodeId: PublicKey): Map[ChannelDesc, ChannelUpdate] = { // BOLT 11: "For each entry, the pubkey is the node ID of the start of the channel", and the last node is the destination @@ -760,7 +760,7 @@ object Router { def isStale(u: ChannelUpdate): Boolean = { // BOLT 7: "nodes MAY prune channels should the timestamp of the latest channel_update be older than 2 weeks (1209600 seconds)" // but we don't want to prune brand new channels for which we didn't yet receive a channel update - val staleThresholdSeconds = Platform.currentTime / 1000 - 1209600 + val staleThresholdSeconds = (Platform.currentTime.milliseconds - 14.days).toSeconds u.timestamp < staleThresholdSeconds } diff --git a/eclair-core/src/main/scala/fr/acinq/eclair/wire/ChannelCodecs.scala b/eclair-core/src/main/scala/fr/acinq/eclair/wire/ChannelCodecs.scala index 49ed5601c..0d013c5a2 100644 --- a/eclair-core/src/main/scala/fr/acinq/eclair/wire/ChannelCodecs.scala +++ b/eclair-core/src/main/scala/fr/acinq/eclair/wire/ChannelCodecs.scala @@ -31,6 +31,8 @@ import grizzled.slf4j.Logging import scodec.bits.BitVector import scodec.codecs._ import scodec.{Attempt, Codec} +import scala.concurrent.duration._ +import scala.compat.Platform /** * Created by PM on 02/06/2017. @@ -237,7 +239,7 @@ object ChannelCodecs extends Logging { val DATA_WAIT_FOR_FUNDING_CONFIRMED_COMPAT_01_Codec: Codec[DATA_WAIT_FOR_FUNDING_CONFIRMED] = ( ("commitments" | commitmentsCodec) :: ("fundingTx" | provide[Option[Transaction]](None)) :: - ("waitingSince" | provide(compat.Platform.currentTime / 1000)) :: + ("waitingSince" | provide(Platform.currentTime.milliseconds.toSeconds)) :: ("deferred" | optional(bool, fundingLockedCodec)) :: ("lastSent" | either(bool, fundingCreatedCodec, fundingSignedCodec))).as[DATA_WAIT_FOR_FUNDING_CONFIRMED].decodeOnly diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/db/SqliteAuditDbSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/db/SqliteAuditDbSpec.scala index 5bd26db5f..afb3699e5 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/db/SqliteAuditDbSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/db/SqliteAuditDbSpec.scala @@ -27,7 +27,7 @@ import fr.acinq.eclair.payment.{PaymentReceived, PaymentRelayed, PaymentSent} import fr.acinq.eclair.wire.ChannelCodecs import fr.acinq.eclair._ import org.scalatest.FunSuite - +import concurrent.duration._ import scala.compat.Platform @@ -48,7 +48,7 @@ class SqliteAuditDbSpec extends FunSuite { val e3 = PaymentRelayed(MilliSatoshi(42000), MilliSatoshi(1000), randomBytes32, randomBytes32, randomBytes32) val e4 = NetworkFeePaid(null, randomKey.publicKey, randomBytes32, Transaction(0, Seq.empty, Seq.empty, 0), Satoshi(42), "mutual") val e5 = PaymentSent(ChannelCodecs.UNKNOWN_UUID, MilliSatoshi(42000), MilliSatoshi(1000), randomBytes32, randomBytes32, randomBytes32, timestamp = 0) - val e6 = PaymentSent(ChannelCodecs.UNKNOWN_UUID, MilliSatoshi(42000), MilliSatoshi(1000), randomBytes32, randomBytes32, randomBytes32, timestamp = Platform.currentTime * 2) + val e6 = PaymentSent(ChannelCodecs.UNKNOWN_UUID, MilliSatoshi(42000), MilliSatoshi(1000), randomBytes32, randomBytes32, randomBytes32, timestamp = (Platform.currentTime.milliseconds + 10.minutes).toMillis) val e7 = AvailableBalanceChanged(null, randomBytes32, ShortChannelId(500000, 42, 1), 456123000, ChannelStateSpec.commitments) val e8 = ChannelLifecycleEvent(randomBytes32, randomKey.publicKey, 456123000, true, false, "mutual") val e9 = ChannelErrorOccured(null, randomBytes32, randomKey.publicKey, null, LocalError(new RuntimeException("oops")), true) @@ -65,12 +65,12 @@ class SqliteAuditDbSpec extends FunSuite { db.add(e9) db.add(e10) - assert(db.listSent(from = 0L, to = Long.MaxValue).toSet === Set(e1, e5, e6)) - assert(db.listSent(from = 100000L, to = Platform.currentTime + 1).toList === List(e1)) - assert(db.listReceived(from = 0L, to = Long.MaxValue).toList === List(e2)) - assert(db.listRelayed(from = 0L, to = Long.MaxValue).toList === List(e3)) - assert(db.listNetworkFees(from = 0L, to = Long.MaxValue).size === 1) - assert(db.listNetworkFees(from = 0L, to = Long.MaxValue).head.txType === "mutual") + assert(db.listSent(from = 0L, to = (Platform.currentTime.milliseconds + 15.minute).toSeconds).toSet === Set(e1, e5, e6)) + assert(db.listSent(from = 100000L, to = (Platform.currentTime.milliseconds + 1.minute).toSeconds).toList === List(e1)) + assert(db.listReceived(from = 0L, to = (Platform.currentTime.milliseconds + 1.minute).toSeconds).toList === List(e2)) + assert(db.listRelayed(from = 0L, to = (Platform.currentTime.milliseconds + 1.minute).toSeconds).toList === List(e3)) + assert(db.listNetworkFees(from = 0L, to = (Platform.currentTime.milliseconds + 1.minute).toSeconds).size === 1) + assert(db.listNetworkFees(from = 0L, to = (Platform.currentTime.milliseconds + 1.minute).toSeconds).head.txType === "mutual") } test("stats") { @@ -152,7 +152,7 @@ class SqliteAuditDbSpec extends FunSuite { } // existing rows will use 00000000-0000-0000-0000-000000000000 as default - assert(migratedDb.listSent(0, Long.MaxValue) == Seq(ps.copy(id = ChannelCodecs.UNKNOWN_UUID))) + assert(migratedDb.listSent(0, (Platform.currentTime.milliseconds + 1.minute).toSeconds) == Seq(ps.copy(id = ChannelCodecs.UNKNOWN_UUID))) val postMigrationDb = new SqliteAuditDb(connection) @@ -166,7 +166,7 @@ class SqliteAuditDbSpec extends FunSuite { postMigrationDb.add(e2) // the old record will have the UNKNOWN_UUID but the new ones will have their actual id - assert(postMigrationDb.listSent(0, Long.MaxValue) == Seq(ps.copy(id = ChannelCodecs.UNKNOWN_UUID), ps1, ps2)) + assert(postMigrationDb.listSent(0, (Platform.currentTime.milliseconds + 1.minute).toSeconds) == Seq(ps.copy(id = ChannelCodecs.UNKNOWN_UUID), ps1, ps2)) } test("handle migration version 2 -> 3") { diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/db/SqlitePaymentsDbSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/db/SqlitePaymentsDbSpec.scala index 67377b7d2..88ddcd39a 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/db/SqlitePaymentsDbSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/db/SqlitePaymentsDbSpec.scala @@ -16,9 +16,7 @@ package fr.acinq.eclair.db -import java.time.Instant import java.util.UUID - import fr.acinq.eclair.db.sqlite.SqliteUtils._ import fr.acinq.bitcoin.{Block, ByteVector32, MilliSatoshi} import fr.acinq.eclair.TestConstants.Bob @@ -28,9 +26,9 @@ import fr.acinq.eclair.payment.PaymentRequest import org.scalatest.FunSuite import scodec.bits._ import fr.acinq.eclair.randomBytes32 - import scala.compat.Platform import OutgoingPaymentStatus._ +import concurrent.duration._ class SqlitePaymentsDbSpec extends FunSuite { @@ -83,7 +81,7 @@ class SqlitePaymentsDbSpec extends FunSuite { assert(preMigrationDb.listIncomingPayments() == Seq(pr1)) assert(preMigrationDb.listOutgoingPayments() == Seq(ps1)) - assert(preMigrationDb.listPaymentRequests(0, Long.MaxValue) == Seq(i1)) + assert(preMigrationDb.listPaymentRequests(0, (Platform.currentTime.milliseconds + 1.minute).toSeconds) == Seq(i1)) val postMigrationDb = new SqlitePaymentsDb(connection) @@ -93,7 +91,7 @@ class SqlitePaymentsDbSpec extends FunSuite { assert(postMigrationDb.listIncomingPayments() == Seq(pr1)) assert(postMigrationDb.listOutgoingPayments() == Seq(ps1)) - assert(preMigrationDb.listPaymentRequests(0, Long.MaxValue) == Seq(i1)) + assert(preMigrationDb.listPaymentRequests(0, (Platform.currentTime.milliseconds + 1.minute).toSeconds) == Seq(i1)) } test("add/list received payments/find 1 payment that exists/find 1 payment that does not exist") { @@ -163,7 +161,7 @@ class SqlitePaymentsDbSpec extends FunSuite { val (paymentHash1, paymentHash2) = (randomBytes32, randomBytes32) val i1 = PaymentRequest(chainHash = Block.TestnetGenesisBlock.hash, amount = Some(MilliSatoshi(123)), paymentHash = paymentHash1, privateKey = bob.nodeKey.privateKey, description = "Some invoice", expirySeconds = None, timestamp = someTimestamp) - val i2 = PaymentRequest(chainHash = Block.TestnetGenesisBlock.hash, amount = None, paymentHash = paymentHash2, privateKey = bob.nodeKey.privateKey, description = "Some invoice", expirySeconds = Some(123456), timestamp = Instant.now().getEpochSecond) + val i2 = PaymentRequest(chainHash = Block.TestnetGenesisBlock.hash, amount = None, paymentHash = paymentHash2, privateKey = bob.nodeKey.privateKey, description = "Some invoice", expirySeconds = Some(123456), timestamp = Platform.currentTime.milliseconds.toSeconds) // i2 doesn't expire assert(i1.expiry.isEmpty && i2.expiry.isDefined) @@ -173,15 +171,17 @@ class SqlitePaymentsDbSpec extends FunSuite { db.addPaymentRequest(i2, ByteVector32.One) // order matters, i2 has a more recent timestamp than i1 - assert(db.listPaymentRequests(0, Long.MaxValue) == Seq(i2, i1)) + assert(db.listPaymentRequests(0, (Platform.currentTime.milliseconds + 1.minute).toSeconds) == Seq(i2, i1)) assert(db.getPaymentRequest(i1.paymentHash) == Some(i1)) assert(db.getPaymentRequest(i2.paymentHash) == Some(i2)) - assert(db.listPendingPaymentRequests(0, Long.MaxValue) == Seq(i2, i1)) + assert(db.listPendingPaymentRequests(0, (Platform.currentTime.milliseconds + 1.minute).toSeconds) == Seq(i2, i1)) assert(db.getPendingPaymentRequestAndPreimage(paymentHash1) == Some((ByteVector32.Zeroes, i1))) assert(db.getPendingPaymentRequestAndPreimage(paymentHash2) == Some((ByteVector32.One, i2))) - assert(db.listPaymentRequests(someTimestamp - 100, someTimestamp + 100) == Seq(i1)) + val from = (someTimestamp - 100).seconds.toSeconds + val to = (someTimestamp + 100).seconds.toSeconds + assert(db.listPaymentRequests(from, to) == Seq(i1)) } } diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/router/RouterSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/router/RouterSpec.scala index 7cafbf9fe..3ba1bbfc7 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/router/RouterSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/router/RouterSpec.scala @@ -242,7 +242,7 @@ class RouterSpec extends BaseRouterSpec { val blockHeight = Globals.blockCount.get().toInt - 2020 val channelId = ShortChannelId(blockHeight, 5, 0) val announcement = channelAnnouncement(channelId, priv_a, priv_c, priv_funding_a, priv_funding_c) - val timestamp = Platform.currentTime / 1000 - 1209600 - 1 + val timestamp = Platform.currentTime.millisecond.toSeconds - 1209600 - 1 val update = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_a, c, channelId, cltvExpiryDelta = 7, htlcMinimumMsat = 0, feeBaseMsat = 766000, feeProportionalMillionths = 10, htlcMaximumMsat = 5, timestamp = timestamp) val probe = TestProbe() probe.ignoreMsg { case _: TransportHandler.ReadAck => true } @@ -257,7 +257,7 @@ class RouterSpec extends BaseRouterSpec { val state = sender.expectMsgType[RoutingState] - val update1 = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_a, c, channelId, cltvExpiryDelta = 7, htlcMinimumMsat = 0, feeBaseMsat = 766000, feeProportionalMillionths = 10, htlcMaximumMsat = 500000000L, timestamp = Platform.currentTime / 1000) + val update1 = makeChannelUpdate(Block.RegtestGenesisBlock.hash, priv_a, c, channelId, cltvExpiryDelta = 7, htlcMinimumMsat = 0, feeBaseMsat = 766000, feeProportionalMillionths = 10, htlcMaximumMsat = 500000000L, timestamp = Platform.currentTime.millisecond.toSeconds) // we want to make sure that transport receives the query val transport = TestProbe() diff --git a/eclair-core/src/test/scala/fr/acinq/eclair/wire/ChannelCodecsSpec.scala b/eclair-core/src/test/scala/fr/acinq/eclair/wire/ChannelCodecsSpec.scala index 92b903ae0..b15ac9dba 100644 --- a/eclair-core/src/test/scala/fr/acinq/eclair/wire/ChannelCodecsSpec.scala +++ b/eclair-core/src/test/scala/fr/acinq/eclair/wire/ChannelCodecsSpec.scala @@ -30,6 +30,7 @@ import org.scalatest.FunSuite import scodec.bits._ import scala.compat.Platform import scala.util.Random +import scala.concurrent.duration._ /** * Created by PM on 31/05/2016. @@ -177,7 +178,7 @@ class ChannelCodecsSpec extends FunSuite { // let's decode the old data (this will use the old codec that provides default values for new fields) val data_new = stateDataCodec.decode(bin_old.toBitVector).require.value assert(data_new.asInstanceOf[DATA_WAIT_FOR_FUNDING_CONFIRMED].fundingTx === None) - assert(Platform.currentTime / 1000 - data_new.asInstanceOf[DATA_WAIT_FOR_FUNDING_CONFIRMED].waitingSince < 3600) // we just set this timestamp to current time + assert(Platform.currentTime.milliseconds.toSeconds - data_new.asInstanceOf[DATA_WAIT_FOR_FUNDING_CONFIRMED].waitingSince < 3600) // we just set this timestamp to current time // and re-encode it with the new codec val bin_new = ByteVector(stateDataCodec.encode(data_new).require.toByteVector.toArray) // data should now be encoded under the new format, with version=0 and type=8