From fa2fc3ed0782c13bddcbeea9f21922ae26cfaf8e Mon Sep 17 00:00:00 2001 From: Igor Artamonov Date: Mon, 10 Oct 2022 23:48:17 -0400 Subject: [PATCH 1/2] solution: newPendingTransactions subscription rel: #179 --- .../kotlin/io/emeraldpay/dshackle/Global.kt | 3 + .../dshackle/commons/DurableFlux.kt | 102 +++++++++++ .../dshackle/commons/ExpiringSet.kt | 153 ++++++++++++++++ .../dshackle/commons/SharedFluxHolder.kt | 67 +++++++ .../dshackle/proxy/WebsocketHandler.kt | 1 - .../dshackle/rpc/NativeSubscribe.kt | 2 +- .../dshackle/rpc/TrackERC20Address.kt | 5 +- .../upstream/NoUpstreamSubscriptions.kt | 28 +++ .../dshackle/upstream/SubscriptionConnect.kt | 27 +++ .../upstream/UpstreamSubscriptions.kt | 25 +++ .../upstream/ethereum/EthereumMultistream.kt | 26 ++- .../upstream/ethereum/EthereumRpcUpstream.kt | 14 +- ...ubscribe.kt => EthereumSubscriptionApi.kt} | 24 ++- .../upstream/ethereum/EthereumUpstream.kt | 4 + .../ethereum/EthereumUpstreamSubscriptions.kt | 24 +++ .../upstream/ethereum/EthereumWsFactory.kt | 4 +- .../upstream/ethereum/EthereumWsHead.kt | 60 ++++++- .../upstream/ethereum/EthereumWsUpstream.kt | 19 +- .../NoEthereumUpstreamSubscriptions.kt | 31 ++++ .../{WsConnection.kt => WsConnectionImpl.kt} | 163 +++++------------- .../upstream/ethereum/WsSubscriptions.kt | 42 +++++ .../upstream/ethereum/WsSubscriptionsImpl.kt | 55 ++++++ .../subscribe/AggregatedPendingTxes.kt | 45 +++++ .../ethereum/subscribe/ConnectBlockUpdates.kt | 5 +- .../ethereum/subscribe/ConnectLogs.kt | 19 +- .../ethereum/subscribe/ConnectNewHeads.kt | 5 +- .../ethereum/subscribe/ConnectSyncing.kt | 5 +- .../subscribe/DefaultPendingTxesSource.kt | 41 +++++ .../subscribe/DshacklePendingTxesSource.kt | 59 +++++++ .../EthereumDshackleSubscriptions.kt | 52 ++++++ .../subscribe/EthereumWsSubscriptions.kt | 46 +++++ .../ethereum/subscribe/NoPendingTxes.kt | 34 ++++ .../ethereum/subscribe/PendingTxesSource.kt | 25 +++ .../subscribe/WebsocketPendingTxes.kt | 48 ++++++ .../subscribe/json/TransactionIdSerializer.kt | 38 ++++ .../upstream/grpc/EthereumGrpcUpstream.kt | 12 +- .../upstream/rpcclient/JsonRpcWsClient.kt | 6 +- .../upstream/rpcclient/JsonRpcWsMessage.kt | 23 +++ .../dshackle/commons/DurableFluxSpec.groovy | 135 +++++++++++++++ .../dshackle/commons/ExpiringSetSpec.groovy | 104 +++++++++++ .../commons/SharedFluxHolderSpec.groovy | 79 +++++++++ .../dshackle/rpc/NativeSubscribeSpec.groovy | 10 +- .../dshackle/rpc/TrackERC20AddressSpec.groovy | 24 ++- ...ovy => EthereumSubscriptionApiSpec.groovy} | 19 +- .../ethereum/EthereumWsHeadSpec.groovy | 75 ++++++++ ...groovy => WsConnectionImplRealSpec.groovy} | 62 +++---- ...pec.groovy => WsConnectionImplSpec.groovy} | 48 +----- .../ethereum/WsSubscriptionsImplSpec.groovy | 90 ++++++++++ .../AggregatedPendingTxesSpec.groovy | 91 ++++++++++ .../DshacklePendingTxesSourceSpec.groovy | 124 +++++++++++++ .../subscribe/WebsocketPendingTxesSpec.groovy | 49 ++++++ .../rpcclient/JsonRpcWsClientSpec.groovy | 4 +- src/test/resources/log4j2.xml | 5 +- 53 files changed, 1983 insertions(+), 278 deletions(-) create mode 100644 src/main/kotlin/io/emeraldpay/dshackle/commons/DurableFlux.kt create mode 100644 src/main/kotlin/io/emeraldpay/dshackle/commons/ExpiringSet.kt create mode 100644 src/main/kotlin/io/emeraldpay/dshackle/commons/SharedFluxHolder.kt create mode 100644 src/main/kotlin/io/emeraldpay/dshackle/upstream/NoUpstreamSubscriptions.kt create mode 100644 src/main/kotlin/io/emeraldpay/dshackle/upstream/SubscriptionConnect.kt create mode 100644 src/main/kotlin/io/emeraldpay/dshackle/upstream/UpstreamSubscriptions.kt rename src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/{EthereumSubscribe.kt => EthereumSubscriptionApi.kt} (82%) create mode 100644 src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumUpstreamSubscriptions.kt create mode 100644 src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/NoEthereumUpstreamSubscriptions.kt rename src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/{WsConnection.kt => WsConnectionImpl.kt} (69%) create mode 100644 src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/WsSubscriptions.kt create mode 100644 src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/WsSubscriptionsImpl.kt create mode 100644 src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/AggregatedPendingTxes.kt create mode 100644 src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/DefaultPendingTxesSource.kt create mode 100644 src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/DshacklePendingTxesSource.kt create mode 100644 src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/EthereumDshackleSubscriptions.kt create mode 100644 src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/EthereumWsSubscriptions.kt create mode 100644 src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/NoPendingTxes.kt create mode 100644 src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/PendingTxesSource.kt create mode 100644 src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/WebsocketPendingTxes.kt create mode 100644 src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/json/TransactionIdSerializer.kt create mode 100644 src/main/kotlin/io/emeraldpay/dshackle/upstream/rpcclient/JsonRpcWsMessage.kt create mode 100644 src/test/groovy/io/emeraldpay/dshackle/commons/DurableFluxSpec.groovy create mode 100644 src/test/groovy/io/emeraldpay/dshackle/commons/ExpiringSetSpec.groovy create mode 100644 src/test/groovy/io/emeraldpay/dshackle/commons/SharedFluxHolderSpec.groovy rename src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/{EthereumSubscribeSpec.groovy => EthereumSubscriptionApiSpec.groovy} (80%) create mode 100644 src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/EthereumWsHeadSpec.groovy rename src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/{WsConnectionRealSpec.groovy => WsConnectionImplRealSpec.groovy} (72%) rename src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/{WsConnectionSpec.groovy => WsConnectionImplSpec.groovy} (68%) create mode 100644 src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/WsSubscriptionsImplSpec.groovy create mode 100644 src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/subscribe/AggregatedPendingTxesSpec.groovy create mode 100644 src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/subscribe/DshacklePendingTxesSourceSpec.groovy create mode 100644 src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/subscribe/WebsocketPendingTxesSpec.groovy diff --git a/src/main/kotlin/io/emeraldpay/dshackle/Global.kt b/src/main/kotlin/io/emeraldpay/dshackle/Global.kt index 1ec1c3df3..d3cb010a1 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/Global.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/Global.kt @@ -25,8 +25,10 @@ import io.emeraldpay.dshackle.upstream.bitcoin.data.EsploraUnspent import io.emeraldpay.dshackle.upstream.bitcoin.data.EsploraUnspentDeserializer import io.emeraldpay.dshackle.upstream.bitcoin.data.RpcUnspent import io.emeraldpay.dshackle.upstream.bitcoin.data.RpcUnspentDeserializer +import io.emeraldpay.dshackle.upstream.ethereum.subscribe.json.TransactionIdSerializer import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcRequest import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcResponse +import io.emeraldpay.etherjar.domain.TransactionId import io.emeraldpay.grpc.Chain import java.text.SimpleDateFormat import java.util.Locale @@ -79,6 +81,7 @@ class Global { private fun createObjectMapper(): ObjectMapper { val module = SimpleModule("EmeraldDshackle", Version(1, 0, 0, null, null, null)) module.addSerializer(JsonRpcResponse::class.java, JsonRpcResponse.ResponseJsonSerializer()) + module.addSerializer(TransactionId::class.java, TransactionIdSerializer()) module.addDeserializer(EsploraUnspent::class.java, EsploraUnspentDeserializer()) module.addDeserializer(RpcUnspent::class.java, RpcUnspentDeserializer()) diff --git a/src/main/kotlin/io/emeraldpay/dshackle/commons/DurableFlux.kt b/src/main/kotlin/io/emeraldpay/dshackle/commons/DurableFlux.kt new file mode 100644 index 000000000..d63429007 --- /dev/null +++ b/src/main/kotlin/io/emeraldpay/dshackle/commons/DurableFlux.kt @@ -0,0 +1,102 @@ +package io.emeraldpay.dshackle.commons + +import java.time.Duration +import org.slf4j.Logger +import org.slf4j.LoggerFactory +import org.springframework.util.backoff.BackOff +import org.springframework.util.backoff.BackOffExecution +import org.springframework.util.backoff.ExponentialBackOff +import org.springframework.util.backoff.FixedBackOff +import reactor.core.publisher.Flux +import reactor.core.publisher.Mono + +/** + * A flux holder that reconnects to it on failure taking into account a back off strategy + */ +class DurableFlux( + private val provider: () -> Flux, + private val errorBackOff: BackOff, + private val log: Logger, +) { + + companion object { + private val defaultLog = LoggerFactory.getLogger(DurableFlux::class.java) + + @JvmStatic + fun newBuilder(): Builder<*> { + return Builder() + } + } + + private var messagesSinceStart = 0 + private var errorBackOffExecution = errorBackOff.start() + + fun connect(): Flux { + return provider.invoke() + .doOnNext { + if (messagesSinceStart == 0) { + errorBackOffExecution = errorBackOff.start() + } + messagesSinceStart++ + } + .doOnSubscribe { + messagesSinceStart = 0 + } + .onErrorResume { t -> + val backoff = errorBackOffExecution.nextBackOff() + if (backoff != BackOffExecution.STOP) { + log.warn("Connection closed with ${t.message}. Reconnecting in ${backoff}ms") + connect().delaySubscription(Duration.ofMillis(backoff)) + } else { + log.warn("Connection closed with ${t.message}. Not reconnecting") + Mono.error(t) + } + } + } + + class Builder { + + private var provider: (() -> Flux)? = null + + protected var errorBackOff: BackOff = FixedBackOff(1_000, Long.MAX_VALUE) + protected var log: Logger = DurableFlux.defaultLog + + @Suppress("UNCHECKED_CAST") + fun using(provider: () -> Flux): Builder { + this.provider = provider as () -> Flux + return this as Builder + } + + fun backoffOnError(time: Duration): Builder { + errorBackOff = FixedBackOff(time.toMillis(), Long.MAX_VALUE) + return this + } + + fun backoffOnError(time: Duration, multiplier: Double, max: Duration? = null): Builder { + errorBackOff = ExponentialBackOff(time.toMillis(), multiplier).also { + if (max != null) { + it.maxInterval = max.toMillis() + } + } + return this + } + + fun backoffOnError(backOff: BackOff): Builder { + errorBackOff = backOff + return this + } + + fun logTo(log: Logger): Builder { + this.log = log + return this + } + + fun build(): DurableFlux { + if (provider == null) { + throw IllegalStateException("No provider for original Flux") + } + return DurableFlux(provider!!,errorBackOff, log) + } + } + +} \ No newline at end of file diff --git a/src/main/kotlin/io/emeraldpay/dshackle/commons/ExpiringSet.kt b/src/main/kotlin/io/emeraldpay/dshackle/commons/ExpiringSet.kt new file mode 100644 index 000000000..f62c7a3de --- /dev/null +++ b/src/main/kotlin/io/emeraldpay/dshackle/commons/ExpiringSet.kt @@ -0,0 +1,153 @@ +package io.emeraldpay.dshackle.commons + +import java.time.Duration +import java.util.LinkedList +import java.util.TreeSet +import java.util.concurrent.locks.ReentrantLock +import kotlin.concurrent.withLock +import org.apache.commons.collections4.iterators.UnmodifiableIterator +import org.slf4j.LoggerFactory + +/** + * A naive implementation of a Set with a limit for elements and an expiration time. Supposed to be used a filter for uniqueness. + * Internally it uses a TreeSet and a journal of added elements, which is used ot remove elements when they expire or the list grows too large. + * It's tread safe, but may be suboptimal to use in multithreaded scenario because of internal locks. + */ +class ExpiringSet( + ttl: Duration, + comparator: Comparator, + val limit: Int, +): MutableSet { + + companion object { + private val log = LoggerFactory.getLogger(ExpiringSet::class.java) + } + + private val tree = TreeSet(comparator) + private val lock = ReentrantLock() + private val journal = LinkedList>() + private var count = 0 + + private val ttl = ttl.toMillis() + + data class JournalItem( + val since: Long = System.currentTimeMillis(), + val value: T + ) { + fun isExpired(ttl: Long): Boolean { + return System.currentTimeMillis() > since + ttl + } + } + + override val size: Int + get() = count + + override fun clear() { + lock.withLock { + tree.clear() + journal.clear() + count = 0 + } + } + + override fun addAll(elements: Collection): Boolean { + var changed = false + elements.forEach { + changed = changed || add(it) + } + return changed + } + + override fun add(element: T): Boolean { + lock.withLock { + val added = tree.add(element) + if (added) { + journal.offer(JournalItem(value = element)) + count++ + shrink() + } + return added + } + } + + override fun isEmpty(): Boolean { + return count == 0 + } + + override fun iterator(): MutableIterator { + // not mutable + return UnmodifiableIterator.unmodifiableIterator(tree.iterator()) + } + + override fun retainAll(elements: Collection): Boolean { + lock.withLock { + var changed = false + val iter = tree.iterator() + while (iter.hasNext()) { + val next = iter.next() + if (!elements.contains(next)) { + changed = true + iter.remove() + count-- + } + } + return changed + } + } + + override fun removeAll(elements: Collection): Boolean { + var changed = false + elements.forEach { + changed = changed || remove(it) + } + return changed + } + + override fun remove(element: T): Boolean { + lock.withLock { + val changed = tree.remove(element) + if (changed) { + count-- + } + return changed + } + } + + override fun containsAll(elements: Collection): Boolean { + return elements.all { contains(it) } + } + + override fun contains(element: T): Boolean { + lock.withLock { + return tree.contains(element) + } + } + + fun shrink() { + lock.withLock { + val iter = journal.iterator() + val removeAtLeast = (count - limit).coerceAtLeast(0) + var removed = 0 + var stop = false + while (!stop && iter.hasNext()) { + val next = iter.next() + val overflow = removeAtLeast > removed + val expired = next.isExpired(ttl) + if (overflow || expired) { + iter.remove() + if (tree.remove(next.value)) { + removed++ + } + } + // we always delete expired elements so don't stop on that + if (!expired) { + // but if we already deleted all non-expired element (i.e., started because it grew too large) + // then we stop as soon as we don't have any overflow + stop = !overflow + } + } + count -= removed + } + } + +} \ No newline at end of file diff --git a/src/main/kotlin/io/emeraldpay/dshackle/commons/SharedFluxHolder.kt b/src/main/kotlin/io/emeraldpay/dshackle/commons/SharedFluxHolder.kt new file mode 100644 index 000000000..df8605e9e --- /dev/null +++ b/src/main/kotlin/io/emeraldpay/dshackle/commons/SharedFluxHolder.kt @@ -0,0 +1,67 @@ +package io.emeraldpay.dshackle.commons + +import java.util.concurrent.atomic.AtomicLong +import java.util.concurrent.locks.ReentrantReadWriteLock +import kotlin.concurrent.read +import kotlin.concurrent.write +import org.slf4j.LoggerFactory +import reactor.core.publisher.Flux + +/** + * A flux holder that that creates it only if requested. Keeps it for the following calls, so all the following calls will + * reuse it. Forgets as soon as it completes/cancelled, so it will be recreated again if needed. + */ +class SharedFluxHolder( + /** + * Provider for the flux. Note that it can be called multiple times but only one is used at the same time. + * I.e., if there is a few calls because of a thread-race only one is kept. + * But once it's completed a new one may be created if requested. + */ + private val provider: () -> Flux +) { + + companion object { + private val log = LoggerFactory.getLogger(SharedFluxHolder::class.java) + } + + private val ids = AtomicLong() + private val lock = ReentrantReadWriteLock() + private var current: Holder? = null + + fun get(): Flux { + lock.read { + if (current != null) { + return current!!.flux + } + } + // The following doesn't consume resources because it's just create a Flux without actual subscription + // So even for the case of a thread race it's okay to create many. B/c only one is going to be kept as `current` and subscribed + val id = ids.incrementAndGet() + val created = Holder( + provider.invoke() + .share() + .doFinally { onClose(id) }, + id + ) + lock.write { + if (current != null) { + return current!!.flux + } + current = created + } + return created.flux + } + + private fun onClose(id: Long) { + lock.write { + if (current?.id == id) { + current = null + } + } + } + + data class Holder( + val flux: Flux, + val id: Long, + ) +} \ No newline at end of file diff --git a/src/main/kotlin/io/emeraldpay/dshackle/proxy/WebsocketHandler.kt b/src/main/kotlin/io/emeraldpay/dshackle/proxy/WebsocketHandler.kt index 55b62f0c2..8ab1a36d9 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/proxy/WebsocketHandler.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/proxy/WebsocketHandler.kt @@ -115,7 +115,6 @@ class WebsocketHandler( ): Flux { return requests.flatMap { call -> val method = call.method - if (method == "eth_subscribe") { val methodParams = splitMethodParams(call.params) if (methodParams != null) { diff --git a/src/main/kotlin/io/emeraldpay/dshackle/rpc/NativeSubscribe.kt b/src/main/kotlin/io/emeraldpay/dshackle/rpc/NativeSubscribe.kt index 63640e3a9..1c363fab7 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/rpc/NativeSubscribe.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/rpc/NativeSubscribe.kt @@ -84,7 +84,7 @@ open class NativeSubscribe( open fun subscribe(chain: Chain, method: String, params: Any?): Flux { val up = multistreamHolder.getUpstream(chain) ?: return Flux.error(SilentException.UnsupportedBlockchain(chain)) return (up as EthereumMultistream) - .getSubscribe() + .getSubscribtionApi() .subscribe(method, params) } diff --git a/src/main/kotlin/io/emeraldpay/dshackle/rpc/TrackERC20Address.kt b/src/main/kotlin/io/emeraldpay/dshackle/rpc/TrackERC20Address.kt index 5a968c449..a0e6d44d0 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/rpc/TrackERC20Address.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/rpc/TrackERC20Address.kt @@ -86,11 +86,12 @@ class TrackERC20Address( val asset = request.asset.code.lowercase(Locale.getDefault()) val tokenDefinition = tokens[TokenId(chain, asset)] ?: return Flux.empty() val logs = getUpstream(chain) - .getSubscribe().logs - .start( + .getSubscribtionApi().logs + .create( listOf(tokenDefinition.token.contract), listOf(EventId.fromSignature("Transfer", "address", "address", "uint256")) ) + .connect() return ethereumAddresses.extract(request.address) .map { TrackedAddress(chain, it, tokenDefinition.token, tokenDefinition.name) } diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/NoUpstreamSubscriptions.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/NoUpstreamSubscriptions.kt new file mode 100644 index 000000000..a41f58356 --- /dev/null +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/NoUpstreamSubscriptions.kt @@ -0,0 +1,28 @@ +/** + * Copyright (c) 2022 EmeraldPay, Inc + * + * 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 io.emeraldpay.dshackle.upstream + +open class NoUpstreamSubscriptions : UpstreamSubscriptions { + + companion object { + val DEFAULT = NoUpstreamSubscriptions() + } + + override fun get(method: String): SubscriptionConnect? { + return null + } + +} \ No newline at end of file diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/SubscriptionConnect.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/SubscriptionConnect.kt new file mode 100644 index 000000000..27b54b50d --- /dev/null +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/SubscriptionConnect.kt @@ -0,0 +1,27 @@ +/** + * Copyright (c) 2022 EmeraldPay, Inc + * + * 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 io.emeraldpay.dshackle.upstream + +import reactor.core.publisher.Flux + +/** + * Note that T is supposed to be serializable as JSON + */ +interface SubscriptionConnect { + + fun connect(): Flux + +} \ No newline at end of file diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/UpstreamSubscriptions.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/UpstreamSubscriptions.kt new file mode 100644 index 000000000..4a6c6d964 --- /dev/null +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/UpstreamSubscriptions.kt @@ -0,0 +1,25 @@ +/** + * Copyright (c) 2022 EmeraldPay, Inc + * + * 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 io.emeraldpay.dshackle.upstream + +/** + * Subscriptions available on the current upstream + */ +interface UpstreamSubscriptions { + + fun get(method: String): SubscriptionConnect? + +} \ No newline at end of file diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumMultistream.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumMultistream.kt index 0e4fcdfee..24adae990 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumMultistream.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumMultistream.kt @@ -25,6 +25,9 @@ import io.emeraldpay.dshackle.upstream.MergedHead import io.emeraldpay.dshackle.upstream.Multistream import io.emeraldpay.dshackle.upstream.Selector import io.emeraldpay.dshackle.upstream.Upstream +import io.emeraldpay.dshackle.upstream.ethereum.subscribe.AggregatedPendingTxes +import io.emeraldpay.dshackle.upstream.ethereum.subscribe.NoPendingTxes +import io.emeraldpay.dshackle.upstream.ethereum.subscribe.PendingTxesSource import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcRequest import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcResponse import io.emeraldpay.grpc.Chain @@ -46,7 +49,8 @@ open class EthereumMultistream( private var head: Head? = null private val reader: EthereumReader = EthereumReader(this, this.caches, getMethodsFactory()) - private val subscribe = EthereumSubscribe(this) + + private var subscribe = EthereumSubscriptionApi(this, NoPendingTxes()) private val supportsEIP1559 = when (chain) { Chain.ETHEREUM, Chain.TESTNET_ROPSTEN, Chain.TESTNET_GOERLI, Chain.TESTNET_RINKEBY -> true else -> false @@ -65,6 +69,24 @@ open class EthereumMultistream( super.init() } + override fun onUpstreamsUpdated() { + super.onUpstreamsUpdated() + + val pendingTxes: PendingTxesSource = upstreams + .mapNotNull { + it.getUpstreamSubscriptions().getPendingTxes() + }.let { + if (it.isEmpty()) { + NoPendingTxes() + } else if (it.size == 1) { + it.first() + } else { + AggregatedPendingTxes(it) + } + } + subscribe = EthereumSubscriptionApi(this, pendingTxes) + } + override fun start() { super.start() reader.start() @@ -137,7 +159,7 @@ open class EthereumMultistream( return Mono.just(LocalCallRouter(reader, getMethods(), getHead())) } - open fun getSubscribe(): EthereumSubscribe { + open fun getSubscribtionApi(): EthereumSubscriptionApi { return subscribe } diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumRpcUpstream.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumRpcUpstream.kt index 818de4a39..a70835993 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumRpcUpstream.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumRpcUpstream.kt @@ -8,12 +8,15 @@ import io.emeraldpay.dshackle.startup.QuorumForLabels import io.emeraldpay.dshackle.upstream.ForkWatch import io.emeraldpay.dshackle.upstream.Head import io.emeraldpay.dshackle.upstream.MergedHead +import io.emeraldpay.dshackle.upstream.NoUpstreamSubscriptions import io.emeraldpay.dshackle.upstream.Upstream import io.emeraldpay.dshackle.upstream.UpstreamAvailability +import io.emeraldpay.dshackle.upstream.UpstreamSubscriptions import io.emeraldpay.dshackle.upstream.calls.CallMethods import io.emeraldpay.dshackle.upstream.calls.DirectCallMethods import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcRequest import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcResponse +import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcWsClient import io.emeraldpay.grpc.Chain import org.slf4j.LoggerFactory import org.springframework.context.Lifecycle @@ -59,6 +62,10 @@ open class EthereumRpcUpstream( } } + override fun getUpstreamSubscriptions(): EthereumUpstreamSubscriptions { + return NoEthereumUpstreamSubscriptions.DEFAULT + } + override fun start() { log.info("Configured for ${chain.chainName}") super.start() @@ -90,13 +97,14 @@ open class EthereumRpcUpstream( open fun createHead(): Head { return if (ethereumWsFactory != null) { // do not set upstream to the WS, since it doesn't control the RPC upstream - val ws = ethereumWsFactory.create(null, null).apply { + val ws = ethereumWsFactory.create(null).apply { connect() } - val wsHead = EthereumWsHead(ws).apply { + val subscriptions = WsSubscriptionsImpl(ws) + val wsHead = EthereumWsHead(getApi(), subscriptions).apply { start() } - // receive bew blocks through WebSockets, but also periodically verify with RPC in case if WS failed + // receive all new blocks through WebSockets, but also periodically verify with RPC in case if WS failed val rpcHead = EthereumRpcHead(getApi(), Duration.ofSeconds(60)).apply { start() } diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumSubscribe.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumSubscriptionApi.kt similarity index 82% rename from src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumSubscribe.kt rename to src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumSubscriptionApi.kt index 69487b1e6..200ac8ea4 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumSubscribe.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumSubscriptionApi.kt @@ -3,17 +3,24 @@ package io.emeraldpay.dshackle.upstream.ethereum import io.emeraldpay.dshackle.upstream.ethereum.subscribe.ConnectLogs import io.emeraldpay.dshackle.upstream.ethereum.subscribe.ConnectNewHeads import io.emeraldpay.dshackle.upstream.ethereum.subscribe.ConnectSyncing +import io.emeraldpay.dshackle.upstream.ethereum.subscribe.PendingTxesSource import io.emeraldpay.etherjar.domain.Address import io.emeraldpay.etherjar.hex.Hex32 import org.slf4j.LoggerFactory import reactor.core.publisher.Flux -open class EthereumSubscribe( - val upstream: EthereumMultistream +open class EthereumSubscriptionApi( + val upstream: EthereumMultistream, + val pendingTxesSource: PendingTxesSource, ) { companion object { - private val log = LoggerFactory.getLogger(EthereumSubscribe::class.java) + private val log = LoggerFactory.getLogger(EthereumSubscriptionApi::class.java) + + const val METHOD_NEW_HEADS = "newHeads" + const val METHOD_LOGS = "logs" + const val METHOD_SYNCING = "syncing" + const val METHOD_PENDING_TXES = "newPendingTransactions" } private val newHeads = ConnectNewHeads(upstream) @@ -22,10 +29,10 @@ open class EthereumSubscribe( @Suppress("UNCHECKED_CAST") open fun subscribe(method: String, params: Any?): Flux { - if (method == "newHeads") { + if (method == METHOD_NEW_HEADS) { return newHeads.connect() } - if (method == "logs") { + if (method == METHOD_LOGS) { val paramsMap = try { if (params != null && Map::class.java.isAssignableFrom(params.javaClass)) { readLogsRequest(params as Map) @@ -35,11 +42,14 @@ open class EthereumSubscribe( } catch (t: Throwable) { return Flux.error(UnsupportedOperationException("Invalid parameter for $method. Error: ${t.message}")) } - return logs.start(paramsMap.address, paramsMap.topics) + return logs.create(paramsMap.address, paramsMap.topics).connect() } - if (method == "syncing") { + if (method == METHOD_SYNCING) { return syncing.connect() } + if (method == METHOD_PENDING_TXES) { + return pendingTxesSource.connect() + } return Flux.error(UnsupportedOperationException("Method $method is not supported")) } diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumUpstream.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumUpstream.kt index 334f3d7d1..92e9e5c21 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumUpstream.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumUpstream.kt @@ -21,7 +21,9 @@ import io.emeraldpay.dshackle.startup.QuorumForLabels import io.emeraldpay.dshackle.upstream.Capability import io.emeraldpay.dshackle.upstream.DefaultUpstream import io.emeraldpay.dshackle.upstream.ForkWatch +import io.emeraldpay.dshackle.upstream.UpstreamSubscriptions import io.emeraldpay.dshackle.upstream.calls.CallMethods +import io.emeraldpay.dshackle.upstream.ethereum.subscribe.PendingTxesSource abstract class EthereumUpstream( id: String, @@ -45,4 +47,6 @@ abstract class EthereumUpstream( override fun getLabels(): Collection { return node?.let { listOf(it.labels) } ?: emptyList() } + + abstract fun getUpstreamSubscriptions(): EthereumUpstreamSubscriptions } diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumUpstreamSubscriptions.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumUpstreamSubscriptions.kt new file mode 100644 index 000000000..eb29a2d7f --- /dev/null +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumUpstreamSubscriptions.kt @@ -0,0 +1,24 @@ +/** + * Copyright (c) 2022 EmeraldPay, Inc + * + * 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 io.emeraldpay.dshackle.upstream.ethereum + +import io.emeraldpay.dshackle.upstream.UpstreamSubscriptions +import io.emeraldpay.dshackle.upstream.ethereum.subscribe.PendingTxesSource + +interface EthereumUpstreamSubscriptions: UpstreamSubscriptions { + + fun getPendingTxes(): PendingTxesSource? +} \ No newline at end of file diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumWsFactory.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumWsFactory.kt index 00def0bfd..1fb07c313 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumWsFactory.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumWsFactory.kt @@ -58,11 +58,11 @@ class EthereumWsFactory( ) } - fun create(upstream: DefaultUpstream?, validator: EthereumUpstreamValidator?): WsConnection { + fun create(upstream: DefaultUpstream?): WsConnectionImpl { require(upstream == null || upstream.getId() == id) { "Creating instance for different upstream. ${upstream?.getId()} != id" } - return WsConnection(uri, origin, basicAuth, metrics, upstream, validator).also { ws -> + return WsConnectionImpl(uri, origin, basicAuth, metrics, upstream).also { ws -> config?.frameSize?.let { ws.frameSize = it } diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumWsHead.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumWsHead.kt index 117e901b6..4f2cfd2c8 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumWsHead.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumWsHead.kt @@ -16,14 +16,28 @@ */ package io.emeraldpay.dshackle.upstream.ethereum +import io.emeraldpay.dshackle.Defaults +import io.emeraldpay.dshackle.Global +import io.emeraldpay.dshackle.SilentException +import io.emeraldpay.dshackle.data.BlockContainer +import io.emeraldpay.dshackle.reader.Reader +import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcRequest +import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcResponse import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcWsClient +import io.emeraldpay.etherjar.rpc.json.BlockJson +import io.emeraldpay.etherjar.rpc.json.TransactionRefJson +import java.time.Duration import org.slf4j.LoggerFactory import org.springframework.context.Lifecycle import reactor.core.Disposable import reactor.core.publisher.Flux +import reactor.core.publisher.Mono +import reactor.core.scheduler.Schedulers +import reactor.retry.Repeat class EthereumWsHead( - private val ws: WsConnection + private val api: Reader, + private val wsSubscriptions: WsSubscriptions, ) : DefaultEthereumHead(), Lifecycle { private val log = LoggerFactory.getLogger(EthereumWsHead::class.java) @@ -38,12 +52,52 @@ class EthereumWsHead( this.subscription?.dispose() val heads = Flux.merge( // get the current block, not just wait for the next update - getLatestBlock(JsonRpcWsClient(ws)), - ws.getBlocksFlux() + getLatestBlock(api), + listenNewHeads() ) this.subscription = super.follow(heads) } + fun listenNewHeads(): Flux { + return wsSubscriptions.subscribe("newHeads") + .map { + Global.objectMapper.readValue(it, BlockJson::class.java) as BlockJson + } + .flatMap { block -> + // newHeads returns incomplete blocks, i.e. without some fields and without transaction hashes, + // so we need to fetch the full block data + if (block.difficulty == null || block.transactions == null) { + enhanceRealBlock(block) + } else { + Mono.just(BlockContainer.from(block)) + } + } + } + + fun enhanceRealBlock(block: BlockJson): Mono { + return Mono.just(block.hash) + .flatMap { hash -> + api.read(JsonRpcRequest("eth_getBlockByHash", listOf(hash.toHex(), false))) + .flatMap { resp -> + if (resp.isNull()) { + Mono.error(SilentException("Received null for block $hash")) + } else { + Mono.just(resp) + } + } + .flatMap(JsonRpcResponse::requireResult) + .map { BlockContainer.fromEthereumJson(it) } + .subscribeOn(Schedulers.boundedElastic()) + .timeout(Defaults.timeoutInternal, Mono.empty()) + }.repeatWhenEmpty { n -> + Repeat.times(5) + .exponentialBackoff(Duration.ofMillis(50), Duration.ofMillis(500)) + .apply(n) + } + .timeout(Defaults.timeout, Mono.empty()) + .onErrorResume { Mono.empty() } + } + override fun stop() { subscription?.dispose() subscription = null diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumWsUpstream.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumWsUpstream.kt index 27e14e262..f606d759a 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumWsUpstream.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumWsUpstream.kt @@ -23,6 +23,7 @@ import io.emeraldpay.dshackle.upstream.Head import io.emeraldpay.dshackle.upstream.Upstream import io.emeraldpay.dshackle.upstream.UpstreamAvailability import io.emeraldpay.dshackle.upstream.calls.CallMethods +import io.emeraldpay.dshackle.upstream.ethereum.subscribe.EthereumWsSubscriptions import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcRequest import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcResponse import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcSwitchClient @@ -49,22 +50,24 @@ class EthereumWsUpstream( } private val head: EthereumWsHead - private val connection: WsConnection + private val connection: WsConnectionImpl private val api: Reader + private val subscriptions: EthereumUpstreamSubscriptions private var validatorSubscription: Disposable? = null - private val validator: EthereumUpstreamValidator init { - validator = EthereumUpstreamValidator(this, getOptions()) - connection = ethereumWsFactory.create(this, validator) - head = EthereumWsHead(connection) + connection = ethereumWsFactory.create(this) + val wsSubscriptions = WsSubscriptionsImpl(connection) // Sometimes the server may close the WebSocket connection during the execution of a call, for example if the response // is too large for WebSockets Frame (and Geth is unable to split messages into separate frames) // In this case the failed request must be rerouted to the HTTP connection, because otherwise it would always fail api = JsonRpcSwitchClient( JsonRpcWsClient(connection), httpConnection ) + + head = EthereumWsHead(getApi(), wsSubscriptions) + subscriptions = EthereumWsSubscriptions(wsSubscriptions) } override fun getHead(): Head { @@ -79,6 +82,10 @@ class EthereumWsUpstream( return false } + override fun getUpstreamSubscriptions(): EthereumUpstreamSubscriptions { + return subscriptions + } + @Suppress("UNCHECKED_CAST") override fun cast(selfType: Class): T { if (!selfType.isAssignableFrom(this.javaClass)) { @@ -92,7 +99,7 @@ class EthereumWsUpstream( connection.connect() head.start() - if (getOptions().disableValidation != null && getOptions().disableValidation!!) { + if (getOptions().disableValidation) { log.warn("Disable validation for upstream ${this.getId()}") this.setLag(0) this.setStatus(UpstreamAvailability.OK) diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/NoEthereumUpstreamSubscriptions.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/NoEthereumUpstreamSubscriptions.kt new file mode 100644 index 000000000..332e459e7 --- /dev/null +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/NoEthereumUpstreamSubscriptions.kt @@ -0,0 +1,31 @@ +/** + * Copyright (c) 2022 EmeraldPay, Inc + * + * 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 io.emeraldpay.dshackle.upstream.ethereum + +import io.emeraldpay.dshackle.upstream.NoUpstreamSubscriptions +import io.emeraldpay.dshackle.upstream.ethereum.subscribe.PendingTxesSource + +class NoEthereumUpstreamSubscriptions: NoUpstreamSubscriptions(), EthereumUpstreamSubscriptions { + + companion object { + val DEFAULT = NoEthereumUpstreamSubscriptions() + } + + override fun getPendingTxes(): PendingTxesSource? { + return null + } + +} \ No newline at end of file diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/WsConnection.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/WsConnectionImpl.kt similarity index 69% rename from src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/WsConnection.kt rename to src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/WsConnectionImpl.kt index e7972bd4f..618abced3 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/WsConnection.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/WsConnectionImpl.kt @@ -17,11 +17,10 @@ package io.emeraldpay.dshackle.upstream.ethereum import io.emeraldpay.dshackle.Defaults import io.emeraldpay.dshackle.Global -import io.emeraldpay.dshackle.SilentException import io.emeraldpay.dshackle.config.AuthConfig -import io.emeraldpay.dshackle.data.BlockContainer import io.emeraldpay.dshackle.upstream.DefaultUpstream import io.emeraldpay.dshackle.upstream.UpstreamAvailability +import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcWsMessage import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcError import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcException import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcRequest @@ -29,8 +28,6 @@ import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcResponse import io.emeraldpay.dshackle.upstream.rpcclient.ResponseWSParser import io.emeraldpay.dshackle.upstream.rpcclient.RpcMetrics import io.emeraldpay.etherjar.rpc.RpcResponseError -import io.emeraldpay.etherjar.rpc.json.BlockJson -import io.emeraldpay.etherjar.rpc.json.TransactionRefJson import io.netty.buffer.ByteBuf import io.netty.buffer.ByteBufInputStream import io.netty.buffer.Unpooled @@ -51,7 +48,6 @@ import reactor.netty.http.client.HttpClient import reactor.netty.http.client.WebsocketClientSpec import reactor.netty.http.websocket.WebsocketInbound import reactor.netty.http.websocket.WebsocketOutbound -import reactor.retry.Repeat import reactor.util.function.Tuples import java.net.URI import java.time.Duration @@ -59,25 +55,21 @@ import java.time.Instant import java.util.Base64 import java.util.concurrent.Executors import java.util.concurrent.TimeUnit -import java.util.concurrent.TimeoutException import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicInteger -open class WsConnection( +open class WsConnectionImpl( private val uri: URI, private val origin: URI, private val basicAuth: AuthConfig.ClientBasicAuth?, private val rpcMetrics: RpcMetrics?, private val upstream: DefaultUpstream?, - private val validator: EthereumUpstreamValidator? -) : AutoCloseable { +): AutoCloseable { companion object { - private val log = LoggerFactory.getLogger(WsConnection::class.java) + private val log = LoggerFactory.getLogger(WsConnectionImpl::class.java) private const val IDS_START = 100 - private const val START_REQUEST = - "{\"jsonrpc\":\"2.0\", \"method\":\"eth_subscribe\", \"id\":\"blocks\", \"params\":[\"newHeads\"]}" // WebSocket Frame limit. // Default is 65_536, but Geth responds with larger frames, @@ -102,18 +94,16 @@ open class WsConnection( private val parser = ResponseWSParser() - private val blocks = Sinks + private val messages = Sinks .many() .multicast() - .directBestEffort() + .directBestEffort() + private var rpcSend = Sinks .many() .unicast() .onBackpressureBuffer() - private val rpcReceive = Sinks - .many() - .multicast() - .directBestEffort() + private val disconnects = Sinks .many() .multicast() @@ -239,11 +229,7 @@ open class WsConnection( read = true } } - if (msg.type == ResponseWSParser.Type.SUBSCRIPTION) { - onSubscription(msg) - } else { - onRpc(msg) - } + onMessage(msg) } catch (t: Throwable) { log.warn("Failed to process WS message. ${t.javaClass}: ${t.message}") Mono.empty() @@ -265,109 +251,52 @@ open class WsConnection( return outbound.send( Flux.merge( - startWhenValidated(), calls.subscribeOn(Schedulers.boundedElastic()), consumer.then(Mono.empty()).subscribeOn(Schedulers.boundedElastic()) ) ) } - /** - * Starts subscriptions ('newHeads') when the upstream is fully validated. If upstream is invalid it breaks flow with an Error. - * I.e., the first requests are made from a Validator and when it returns OK the Connection continues with other stuff. - */ - fun startWhenValidated(): Publisher { - val start = Mono.just(START_REQUEST).map { - Unpooled.wrappedBuffer(it.toByteArray()) - } - - return if (validator != null) { - validator.validate() - .timeout( - Defaults.timeoutInternal, - Mono.fromCallable { log.warn("Not received a validation result from $uri") }.then(Mono.error(TimeoutException())) - ) - .flatMap { - if (it == UpstreamAvailability.OK) { - start - } else { - tryReconnectLater() - Mono.error(IllegalStateException("Upstream $uri is not ready")) - } - } - } else { - start - } - } - - fun onRpc(msg: ResponseWSParser.WsResponse): Mono { - return if (msg.id.isNumber()) { - val resp = JsonRpcResponse( - msg.value, msg.error, msg.id, null - ) - Mono.fromCallable { - val status = rpcReceive.tryEmitNext(resp) - if (status.isFailure) { - if (status == Sinks.EmitResult.FAIL_ZERO_SUBSCRIBER) { - log.debug("No subscribers to WS response") - } else { - log.warn("Failed to proceed with a RPC message: $status") - } + fun onMessage(msg: ResponseWSParser.WsResponse): Mono { + return Mono.fromCallable { + val status = messages.tryEmitNext(msg) + if (status.isFailure) { + if (status == Sinks.EmitResult.FAIL_ZERO_SUBSCRIBER) { + log.debug("No subscribers to WS response") + } else { + log.warn("Failed to proceed with a WS message: $status") } - }.then() - } else { - // it's a response to the newHeads subscription, just ignore it - Mono.empty() - } + } + }.then() } - fun onSubscription(msg: ResponseWSParser.WsResponse): Mono { - if (msg.error != null) { - return Mono.error(IllegalStateException("Received error from WS upstream: ${msg.error.message}")) - } - // we always expect an answer to the `newHeads`, since we are not initiating any other subscriptions - return Mono.fromCallable { - Global.objectMapper.readValue(msg.value, BlockJson::class.java) as BlockJson - }.flatMap { onNewHeads(it) }.then() + open fun getRpcResponses(): Flux { + return Flux.from(messages.asFlux()) + .publishOn(Schedulers.boundedElastic()) + .filter { + it.type == ResponseWSParser.Type.RPC + } + .map { msg -> + JsonRpcResponse( + msg.value, msg.error, msg.id, null + ) + } } - fun onNewHeads(block: BlockJson): Mono { - // newHeads returns incomplete blocks, i.e. without some fields and without transaction hashes, - // so we need to fetch the full block data - return if (block.difficulty == null || block.transactions == null) { - Mono.just(block.hash) - .flatMap { hash -> - call(JsonRpcRequest("eth_getBlockByHash", listOf(hash.toHex(), false))) - .flatMap { resp -> - if (resp.isNull()) { - Mono.error(SilentException("Received null for block $hash")) - } else { - Mono.just(resp) - } - } - .flatMap(JsonRpcResponse::requireResult) - .map { BlockContainer.fromEthereumJson(it) } - .subscribeOn(Schedulers.boundedElastic()) - .timeout(Defaults.timeoutInternal, Mono.empty()) - }.repeatWhenEmpty { n -> - Repeat.times(5) - .exponentialBackoff(Duration.ofMillis(50), Duration.ofMillis(500)) - .apply(n) - } - .timeout(Defaults.timeout, Mono.empty()) - .onErrorResume { Mono.empty() } - .doOnNext { - blocks.tryEmitNext(it) - } - .then() - } else { - Mono.fromCallable { - blocks.tryEmitNext(BlockContainer.from(block)) - }.then() - } + open fun getSubscribeResponses(): Flux { + return Flux.from(messages.asFlux()) + .publishOn(Schedulers.boundedElastic()) + .filter { + it.type == ResponseWSParser.Type.SUBSCRIPTION + } + .map { msg -> + JsonRpcWsMessage( + msg.value, msg.error, msg.id.asString(), + ) + } } - fun call(originalRequest: JsonRpcRequest): Mono { + open fun callRpc(originalRequest: JsonRpcRequest): Mono { return Mono.fromCallable { val startTime = System.nanoTime() // use an internal id sequence, to avoid id conflicts with user calls @@ -379,7 +308,7 @@ open class WsConnection( } } - fun sendRpc(request: JsonRpcRequest) { + private fun sendRpc(request: JsonRpcRequest) { // submit to upstream in a separate thread, to free current thread (needs for subscription, etc) sendExecutor.execute { val result = rpcSend.tryEmitNext(request) @@ -399,7 +328,7 @@ open class WsConnection( ) ) - val response = Flux.from(rpcReceive.asFlux()) + val response = Flux.from(getRpcResponses()) .doOnSubscribe { sendRpc(request) } .filter { resp -> resp.id.asNumber() == expectedId } .take(Defaults.timeout) @@ -426,10 +355,6 @@ open class WsConnection( .switchIfEmpty(Mono.error(noResponse)) } - fun getBlocksFlux(): Flux { - return this.blocks.asFlux() - } - override fun close() { log.info("Closing connection to WebSocket $uri") keepConnection = false diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/WsSubscriptions.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/WsSubscriptions.kt new file mode 100644 index 000000000..c4b3eb222 --- /dev/null +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/WsSubscriptions.kt @@ -0,0 +1,42 @@ +/** + * Copyright (c) 2022 EmeraldPay, Inc + * + * 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 io.emeraldpay.dshackle.upstream.ethereum + +import reactor.core.publisher.Flux + +/** + * A JSON-RPC Subscription client. + * In general, it's a Websocket extension for JSON RPC that allows multiple responses to the same method. + * + * Example: + * + *

+ * > {"id": 1, "method": "eth_subscribe", "params": ["newPendingTransactions"]}
+ * > {"jsonrpc":"2.0","id":1,"result":"0xcff45d00e77e8e050d919daf284516c8"}
+ * > {"jsonrpc":"2.0","method":"eth_subscription","params":{"subscription":"0xcff45d00e77e8e050d919daf284516c8","result":"0xa61bab14fc9720ea8725622688c2f964666d7c2afdae38af7dad53f12f242d5c"}}
+ * > {"jsonrpc":"2.0","method":"eth_subscription","params":{"subscription":"0xcff45d00e77e8e050d919daf284516c8","result":"0x911548eb0f3bf353a54e03a3506c7c3e747470d6c201f03babbc07ff6e14cd6e"}}
+ * > {"jsonrpc":"2.0","method":"eth_subscription","params":{"subscription":"0xcff45d00e77e8e050d919daf284516c8","result":"0x67f22a3b441ea312306f97694ca8159f8d6faaccf0f5ce6442c84b13991f1d23"}}
+ * 
+ * + * */ +interface WsSubscriptions { + + /** + * Subscribe on remote + */ + fun subscribe(method: String): Flux + +} \ No newline at end of file diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/WsSubscriptionsImpl.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/WsSubscriptionsImpl.kt new file mode 100644 index 000000000..5ba098305 --- /dev/null +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/WsSubscriptionsImpl.kt @@ -0,0 +1,55 @@ +/** + * Copyright (c) 2022 EmeraldPay, Inc + * + * 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 io.emeraldpay.dshackle.upstream.ethereum + +import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcException +import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcRequest +import java.util.concurrent.atomic.AtomicLong +import java.util.concurrent.atomic.AtomicReference +import org.apache.commons.lang3.StringUtils +import org.slf4j.LoggerFactory +import reactor.core.publisher.Flux +import reactor.core.publisher.Mono + +class WsSubscriptionsImpl( + val conn: WsConnectionImpl, +): WsSubscriptions { + + companion object { + private val log = LoggerFactory.getLogger(WsSubscriptionsImpl::class.java) + } + + private val ids = AtomicLong(1) + + override fun subscribe(method: String): Flux { + val subscriptionId = AtomicReference("") + val messages = conn.getSubscribeResponses() + .filter { it.subscriptionId == subscriptionId.get() } + .filter { it.result != null } // should never happen + .map { it.result!! } + + return conn.callRpc(JsonRpcRequest("eth_subscribe", listOf(method), ids.incrementAndGet())) + .flatMapMany { + if (it.hasError()) { + log.warn("Failed to establish ETH Subscription: ${it.error?.message}") + Mono.error(JsonRpcException(it.id, it.error!!)) + } else { + subscriptionId.set(it.getResultAsProcessedString()) + messages + } + } + } +} \ No newline at end of file diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/AggregatedPendingTxes.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/AggregatedPendingTxes.kt new file mode 100644 index 000000000..ad28fcff5 --- /dev/null +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/AggregatedPendingTxes.kt @@ -0,0 +1,45 @@ +/** + * Copyright (c) 2022 EmeraldPay, Inc + * + * 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 io.emeraldpay.dshackle.upstream.ethereum.subscribe + +import io.emeraldpay.dshackle.commons.ExpiringSet +import io.emeraldpay.etherjar.domain.TransactionId +import io.emeraldpay.etherjar.hex.HexDataComparator +import java.time.Duration +import org.slf4j.LoggerFactory +import reactor.core.publisher.Flux + +class AggregatedPendingTxes( + private val sources: List +): PendingTxesSource { + + companion object { + private val log = LoggerFactory.getLogger(AggregatedPendingTxes::class.java) + } + + private val track = ExpiringSet( + Duration.ofSeconds(30), + HexDataComparator() as Comparator, + 10_000 + ) + + override fun connect(): Flux { + return Flux.merge( + sources.map(PendingTxesSource::connect) + ).filter(track::add) + } + +} \ No newline at end of file diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/ConnectBlockUpdates.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/ConnectBlockUpdates.kt index 3c59b7de4..a5a9c642d 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/ConnectBlockUpdates.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/ConnectBlockUpdates.kt @@ -19,6 +19,7 @@ import io.emeraldpay.dshackle.data.BlockContainer import io.emeraldpay.dshackle.data.BlockId import io.emeraldpay.dshackle.data.TxId import io.emeraldpay.dshackle.upstream.Head +import io.emeraldpay.dshackle.upstream.SubscriptionConnect import io.emeraldpay.dshackle.upstream.ethereum.EthereumMultistream import org.slf4j.LoggerFactory import reactor.core.publisher.Flux @@ -33,7 +34,7 @@ import kotlin.concurrent.write class ConnectBlockUpdates( private val upstream: EthereumMultistream -) { +): SubscriptionConnect { companion object { private val log = LoggerFactory.getLogger(ConnectBlockUpdates::class.java) @@ -50,7 +51,7 @@ class ConnectBlockUpdates( private var connected: Flux? = null private val connectLock = ReentrantLock() - fun connect(): Flux { + override fun connect(): Flux { val current = connected if (current != null) { return current diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/ConnectLogs.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/ConnectLogs.kt index 1309e4c25..e9233194b 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/ConnectLogs.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/ConnectLogs.kt @@ -15,6 +15,7 @@ */ package io.emeraldpay.dshackle.upstream.ethereum.subscribe +import io.emeraldpay.dshackle.upstream.SubscriptionConnect import io.emeraldpay.dshackle.upstream.ethereum.EthereumMultistream import io.emeraldpay.dshackle.upstream.ethereum.subscribe.json.LogMessage import io.emeraldpay.etherjar.domain.Address @@ -44,14 +45,18 @@ open class ConnectLogs( return produceLogs.produce(connectBlockUpdates.connect()) } - open fun start(addresses: List
, topics: List): Flux { - // shortcut to the whole output if we don't have any filters - if (addresses.isEmpty() && topics.isEmpty()) { - return start() + open fun create(addresses: List
, topics: List): SubscriptionConnect { + return object : SubscriptionConnect { + override fun connect(): Flux { + // shortcut to the whole output if we don't have any filters + if (addresses.isEmpty() && topics.isEmpty()) { + return start() + } + // filtered output + return start() + .transform(filtered(addresses, topics)) + } } - // filtered output - return start() - .transform(filtered(addresses, topics)) } fun filtered(addresses: List
, topics: List): Function, Flux> { diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/ConnectNewHeads.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/ConnectNewHeads.kt index 0bcc1f71b..3c28b6566 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/ConnectNewHeads.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/ConnectNewHeads.kt @@ -15,6 +15,7 @@ */ package io.emeraldpay.dshackle.upstream.ethereum.subscribe +import io.emeraldpay.dshackle.upstream.SubscriptionConnect import io.emeraldpay.dshackle.upstream.ethereum.EthereumMultistream import io.emeraldpay.dshackle.upstream.ethereum.subscribe.json.NewHeadMessage import org.slf4j.LoggerFactory @@ -29,7 +30,7 @@ import kotlin.concurrent.withLock */ class ConnectNewHeads( private val upstream: EthereumMultistream -) { +): SubscriptionConnect { companion object { private val log = LoggerFactory.getLogger(ConnectNewHeads::class.java) @@ -38,7 +39,7 @@ class ConnectNewHeads( private var connected: Flux? = null private val connectLock = ReentrantLock() - fun connect(): Flux { + override fun connect(): Flux { val current = connected if (current != null) { return current diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/ConnectSyncing.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/ConnectSyncing.kt index f1f2e8217..c086d12f0 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/ConnectSyncing.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/ConnectSyncing.kt @@ -15,6 +15,7 @@ */ package io.emeraldpay.dshackle.upstream.ethereum.subscribe +import io.emeraldpay.dshackle.upstream.SubscriptionConnect import io.emeraldpay.dshackle.upstream.UpstreamAvailability import io.emeraldpay.dshackle.upstream.ethereum.EthereumMultistream import org.slf4j.LoggerFactory @@ -25,7 +26,7 @@ import kotlin.concurrent.withLock class ConnectSyncing( private val upstream: EthereumMultistream -) { +): SubscriptionConnect { companion object { private val log = LoggerFactory.getLogger(ConnectSyncing::class.java) @@ -34,7 +35,7 @@ class ConnectSyncing( private var connected: Flux? = null private val connectLock = ReentrantLock() - fun connect(): Flux { + override fun connect(): Flux { val current = connected if (current != null) { return current diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/DefaultPendingTxesSource.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/DefaultPendingTxesSource.kt new file mode 100644 index 000000000..d7fdc1ac2 --- /dev/null +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/DefaultPendingTxesSource.kt @@ -0,0 +1,41 @@ +/** + * Copyright (c) 2022 EmeraldPay, Inc + * + * 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 io.emeraldpay.dshackle.upstream.ethereum.subscribe + +import io.emeraldpay.dshackle.commons.DurableFlux +import io.emeraldpay.dshackle.commons.SharedFluxHolder +import io.emeraldpay.dshackle.upstream.SubscriptionConnect +import io.emeraldpay.etherjar.domain.TransactionId +import java.time.Duration +import reactor.core.publisher.Flux + +abstract class DefaultPendingTxesSource: SubscriptionConnect, PendingTxesSource { + + private val connectionSource = DurableFlux + .newBuilder() + .using(::createConnection) + .backoffOnError(Duration.ofMillis(100), 1.5, Duration.ofSeconds(60)) + .build() + private val holder = SharedFluxHolder( + connectionSource::connect + ) + + override fun connect(): Flux { + return holder.get() + } + + abstract fun createConnection(): Flux +} \ No newline at end of file diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/DshacklePendingTxesSource.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/DshacklePendingTxesSource.kt new file mode 100644 index 000000000..46af5671b --- /dev/null +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/DshacklePendingTxesSource.kt @@ -0,0 +1,59 @@ +/** + * Copyright (c) 2022 EmeraldPay, Inc + * + * 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 io.emeraldpay.dshackle.upstream.ethereum.subscribe + +import io.emeraldpay.api.proto.BlockchainOuterClass +import io.emeraldpay.api.proto.BlockchainOuterClass.NativeSubscribeReplyItem +import io.emeraldpay.api.proto.BlockchainOuterClass.NativeSubscribeRequest +import io.emeraldpay.api.proto.ReactorBlockchainGrpc +import io.emeraldpay.dshackle.upstream.ethereum.EthereumSubscriptionApi +import io.emeraldpay.etherjar.domain.TransactionId +import io.emeraldpay.grpc.Chain +import reactor.core.publisher.Flux + +class DshacklePendingTxesSource( + private val blockchain: Chain, + private val conn: ReactorBlockchainGrpc.ReactorBlockchainStub, +): PendingTxesSource, DefaultPendingTxesSource() { + + private val request = NativeSubscribeRequest.newBuilder() + .setChainValue(blockchain.id) + .setMethod(EthereumSubscriptionApi.METHOD_PENDING_TXES) + .build() + + var available = false + + override fun createConnection(): Flux { + if (!available) { + return Flux.empty() + } + return conn + .nativeSubscribe(request) + .map(::readResponse) + .map(TransactionId::from) + } + + fun readResponse(resp: NativeSubscribeReplyItem): String { + // comes as a string, so cut off the quotes + return resp.payload.substring(1, resp.payload.size() - 1).toStringUtf8() + } + + fun update(conf: BlockchainOuterClass.DescribeChain) { + available = conf.supportedMethodsList.any { + it == EthereumSubscriptionApi.METHOD_PENDING_TXES + } + } +} \ No newline at end of file diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/EthereumDshackleSubscriptions.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/EthereumDshackleSubscriptions.kt new file mode 100644 index 000000000..b968fceb7 --- /dev/null +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/EthereumDshackleSubscriptions.kt @@ -0,0 +1,52 @@ +/** + * Copyright (c) 2022 EmeraldPay, Inc + * + * 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 io.emeraldpay.dshackle.upstream.ethereum.subscribe + +import io.emeraldpay.api.proto.BlockchainOuterClass +import io.emeraldpay.api.proto.ReactorBlockchainGrpc +import io.emeraldpay.dshackle.upstream.SubscriptionConnect +import io.emeraldpay.dshackle.upstream.UpstreamSubscriptions +import io.emeraldpay.dshackle.upstream.ethereum.EthereumSubscriptionApi +import io.emeraldpay.dshackle.upstream.ethereum.EthereumUpstreamSubscriptions +import io.emeraldpay.grpc.Chain +import org.slf4j.LoggerFactory + +class EthereumDshackleSubscriptions( + private val blockchain: Chain, + private val conn: ReactorBlockchainGrpc.ReactorBlockchainStub, +) : UpstreamSubscriptions, EthereumUpstreamSubscriptions { + + companion object { + private val log = LoggerFactory.getLogger(EthereumDshackleSubscriptions::class.java) + } + + private val pendingTxes = DshacklePendingTxesSource(blockchain, conn) + + override fun get(method: String): SubscriptionConnect? { + if (method == EthereumSubscriptionApi.METHOD_PENDING_TXES) { + return pendingTxes as SubscriptionConnect + } + return null + } + + fun update(conf: BlockchainOuterClass.DescribeChain) { + pendingTxes.update(conf) + } + + override fun getPendingTxes(): PendingTxesSource? { + return pendingTxes + } +} \ No newline at end of file diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/EthereumWsSubscriptions.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/EthereumWsSubscriptions.kt new file mode 100644 index 000000000..b57919848 --- /dev/null +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/EthereumWsSubscriptions.kt @@ -0,0 +1,46 @@ +/** + * Copyright (c) 2022 EmeraldPay, Inc + * + * 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 io.emeraldpay.dshackle.upstream.ethereum.subscribe + +import io.emeraldpay.dshackle.upstream.SubscriptionConnect +import io.emeraldpay.dshackle.upstream.UpstreamSubscriptions +import io.emeraldpay.dshackle.upstream.ethereum.EthereumSubscriptionApi +import io.emeraldpay.dshackle.upstream.ethereum.EthereumUpstreamSubscriptions +import io.emeraldpay.dshackle.upstream.ethereum.WsSubscriptions +import org.slf4j.LoggerFactory + +class EthereumWsSubscriptions( + private val conn: WsSubscriptions +): UpstreamSubscriptions, EthereumUpstreamSubscriptions { + + companion object { + private val log = LoggerFactory.getLogger(EthereumWsSubscriptions::class.java) + } + + private val pendingTxes = WebsocketPendingTxes(conn) + + override fun get(method: String): SubscriptionConnect? { + if (method == EthereumSubscriptionApi.METHOD_PENDING_TXES) { + return pendingTxes as SubscriptionConnect + } + return null + } + + override fun getPendingTxes(): PendingTxesSource? { + return pendingTxes + } + +} \ No newline at end of file diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/NoPendingTxes.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/NoPendingTxes.kt new file mode 100644 index 000000000..909efb76b --- /dev/null +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/NoPendingTxes.kt @@ -0,0 +1,34 @@ +/** + * Copyright (c) 2022 EmeraldPay, Inc + * + * 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 io.emeraldpay.dshackle.upstream.ethereum.subscribe + +import io.emeraldpay.etherjar.domain.TransactionId +import org.slf4j.LoggerFactory +import reactor.core.publisher.Flux + +class NoPendingTxes: PendingTxesSource { + + companion object { + private val log = LoggerFactory.getLogger(NoPendingTxes::class.java) + + val DEFAULT = NoPendingTxes() + } + + override fun connect(): Flux { + return Flux.empty() + } + +} \ No newline at end of file diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/PendingTxesSource.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/PendingTxesSource.kt new file mode 100644 index 000000000..22bd4dba3 --- /dev/null +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/PendingTxesSource.kt @@ -0,0 +1,25 @@ +/** + * Copyright (c) 2022 EmeraldPay, Inc + * + * 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 io.emeraldpay.dshackle.upstream.ethereum.subscribe + +import io.emeraldpay.dshackle.upstream.SubscriptionConnect +import io.emeraldpay.etherjar.domain.TransactionId + +/** + * A source to subscribe to newPendingTransactions. + * When using a Websocket RPC on a node, it produces hashes of new transactions received. + */ +interface PendingTxesSource: SubscriptionConnect \ No newline at end of file diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/WebsocketPendingTxes.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/WebsocketPendingTxes.kt new file mode 100644 index 000000000..474239a6e --- /dev/null +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/WebsocketPendingTxes.kt @@ -0,0 +1,48 @@ +/** + * Copyright (c) 2022 EmeraldPay, Inc + * + * 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 io.emeraldpay.dshackle.upstream.ethereum.subscribe + +import io.emeraldpay.dshackle.upstream.SubscriptionConnect +import io.emeraldpay.dshackle.upstream.ethereum.EthereumSubscriptionApi +import io.emeraldpay.dshackle.upstream.ethereum.WsSubscriptions +import io.emeraldpay.etherjar.domain.TransactionId +import java.time.Duration +import org.slf4j.LoggerFactory +import reactor.core.publisher.Flux +import reactor.core.publisher.Mono + +class WebsocketPendingTxes( + private val wsSubscriptions: WsSubscriptions +) : DefaultPendingTxesSource(), SubscriptionConnect { + + companion object { + private val log = LoggerFactory.getLogger(WebsocketPendingTxes::class.java) + } + + override fun createConnection(): Flux { + return wsSubscriptions.subscribe(EthereumSubscriptionApi.METHOD_PENDING_TXES) + .timeout(Duration.ofSeconds(60), Mono.empty()) + .map { + // comes as a JS string, i.e., within quotes + val value = ByteArray(it.size - 2) + System.arraycopy(it, 1, value, 0, value.size) + TransactionId.from(String(value)) + } + .doOnError { t -> log.warn("Invalid pending transaction", t) } + .onErrorResume { Mono.empty() } + } + +} \ No newline at end of file diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/json/TransactionIdSerializer.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/json/TransactionIdSerializer.kt new file mode 100644 index 000000000..96adf5985 --- /dev/null +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/json/TransactionIdSerializer.kt @@ -0,0 +1,38 @@ +/** + * Copyright (c) 2022 EmeraldPay, Inc + * + * 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 io.emeraldpay.dshackle.upstream.ethereum.subscribe.json + +import com.fasterxml.jackson.core.JsonGenerator +import com.fasterxml.jackson.databind.JsonSerializer +import com.fasterxml.jackson.databind.SerializerProvider +import io.emeraldpay.etherjar.domain.TransactionId +import org.slf4j.LoggerFactory + +class TransactionIdSerializer: JsonSerializer() { + + companion object { + private val log = LoggerFactory.getLogger(TransactionIdSerializer::class.java) + } + + override fun serialize(value: TransactionId?, gen: JsonGenerator, serializers: SerializerProvider) { + if (value == null) { + gen.writeNull() + return + } + gen.writeString(value.toHex()) + } + +} \ No newline at end of file diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/grpc/EthereumGrpcUpstream.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/grpc/EthereumGrpcUpstream.kt index e6ba2ee7b..be043b899 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/grpc/EthereumGrpcUpstream.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/grpc/EthereumGrpcUpstream.kt @@ -32,8 +32,11 @@ import io.emeraldpay.dshackle.upstream.OptionalHead import io.emeraldpay.dshackle.upstream.Selector import io.emeraldpay.dshackle.upstream.Upstream import io.emeraldpay.dshackle.upstream.UpstreamAvailability +import io.emeraldpay.dshackle.upstream.UpstreamSubscriptions import io.emeraldpay.dshackle.upstream.calls.CallMethods import io.emeraldpay.dshackle.upstream.ethereum.EthereumUpstream +import io.emeraldpay.dshackle.upstream.ethereum.EthereumUpstreamSubscriptions +import io.emeraldpay.dshackle.upstream.ethereum.subscribe.EthereumDshackleSubscriptions import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcGrpcClient import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcRequest import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcResponse @@ -56,7 +59,7 @@ open class EthereumGrpcUpstream( role: UpstreamsConfig.UpstreamRole, private val chain: Chain, options: UpstreamsConfig.Options, - private val remote: ReactorBlockchainGrpc.ReactorBlockchainStub, + private val remote: ReactorBlockchainStub, private val client: JsonRpcGrpcClient ) : EthereumUpstream( "${parentId}_${chain.chainCode.lowercase(Locale.getDefault())}", @@ -113,8 +116,9 @@ open class EthereumGrpcUpstream( private val defaultReader: Reader = client.forSelector(Selector.empty) var timeout = Defaults.timeout + private val ethereumSubscriptions = EthereumDshackleSubscriptions(chain, remote) - override fun getBlockchainApi(): ReactorBlockchainGrpc.ReactorBlockchainStub { + override fun getBlockchainApi(): ReactorBlockchainStub { return remote } @@ -165,6 +169,10 @@ open class EthereumGrpcUpstream( return defaultReader } + override fun getUpstreamSubscriptions(): EthereumUpstreamSubscriptions { + return ethereumSubscriptions + } + @Suppress("UNCHECKED_CAST") override fun cast(selfType: Class): T { if (!selfType.isAssignableFrom(this.javaClass)) { diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/rpcclient/JsonRpcWsClient.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/rpcclient/JsonRpcWsClient.kt index 6f65d0a2e..a286d7f3e 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/rpcclient/JsonRpcWsClient.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/rpcclient/JsonRpcWsClient.kt @@ -16,12 +16,12 @@ package io.emeraldpay.dshackle.upstream.rpcclient import io.emeraldpay.dshackle.reader.Reader -import io.emeraldpay.dshackle.upstream.ethereum.WsConnection +import io.emeraldpay.dshackle.upstream.ethereum.WsConnectionImpl import io.emeraldpay.etherjar.rpc.RpcResponseError import reactor.core.publisher.Mono class JsonRpcWsClient( - private val ws: WsConnection + private val ws: WsConnectionImpl ) : Reader { override fun read(key: JsonRpcRequest): Mono { @@ -36,6 +36,6 @@ class JsonRpcWsClient( ) ) } - return ws.call(key) + return ws.callRpc(key) } } diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/rpcclient/JsonRpcWsMessage.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/rpcclient/JsonRpcWsMessage.kt new file mode 100644 index 000000000..e701ffbef --- /dev/null +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/rpcclient/JsonRpcWsMessage.kt @@ -0,0 +1,23 @@ +/** + * Copyright (c) 2022 EmeraldPay, Inc + * + * 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 io.emeraldpay.dshackle.upstream.rpcclient + +class JsonRpcWsMessage( + val result: ByteArray?, + val error: JsonRpcError?, + val subscriptionId: String, +) { +} \ No newline at end of file diff --git a/src/test/groovy/io/emeraldpay/dshackle/commons/DurableFluxSpec.groovy b/src/test/groovy/io/emeraldpay/dshackle/commons/DurableFluxSpec.groovy new file mode 100644 index 000000000..fa3d4b2c6 --- /dev/null +++ b/src/test/groovy/io/emeraldpay/dshackle/commons/DurableFluxSpec.groovy @@ -0,0 +1,135 @@ +/** + * Copyright (c) 2022 EmeraldPay, Inc + * + * 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 io.emeraldpay.dshackle.commons + +import org.springframework.util.backoff.ExponentialBackOff +import reactor.core.publisher.Flux +import reactor.core.publisher.Mono +import reactor.test.StepVerifier +import spock.lang.Retry +import spock.lang.Specification + +import java.time.Duration + +class DurableFluxSpec extends Specification { + + private static CLOCK_ALLOW_ERROR_MS = 10 + + def "Normal subscribe works"() { + when: + def flux = DurableFlux.newBuilder() + .using({ + Flux.fromIterable([1, 2, 3]) + }) + .build() + def values = flux.connect().collectList().block(Duration.ofSeconds(1)) + + then: + values == [1, 2, 3] + } + + def "Reconnects when broken"() { + when: + def connects = 0 + def flux = DurableFlux.newBuilder() + .using({ + connects++ + def id = connects + Flux.fromIterable([100+id, 200+id, 300+id]) + .concatWith(Mono.error(new RuntimeException("[TEST Reached the end of $id]"))) + }) + .backoffOnError(Duration.ofMillis(50)) + .build() + def values = flux.connect() + .take(5) + .collectList().block(Duration.ofSeconds(1)) + + then: + values == [101, 201, 301, 102, 202] + } + + @Retry // sometimes it goes too fast or too slow + def "No continuous backoff if restored"() { + when: + def connects = 0 + def flux = DurableFlux.newBuilder() + .using({ + connects++ + def id = connects + Flux.fromIterable([100+id, 200+id]) + .concatWith(Mono.error(new RuntimeException("[TEST ERROR $id]"))) + }) + .backoffOnError(new ExponentialBackOff(100, 2)) + .build() + def values = flux.connect() + .take(7) + + def verifier = StepVerifier.create(values) + .expectNext(101, 201) + .expectNoEvent(Duration.ofMillis(100 - CLOCK_ALLOW_ERROR_MS)) + .expectNext(102, 202) + .expectNoEvent(Duration.ofMillis(100 - CLOCK_ALLOW_ERROR_MS)) + .expectNext(103, 203) + .expectNoEvent(Duration.ofMillis(100 - CLOCK_ALLOW_ERROR_MS)) + .expectNext(104) + .expectComplete() + .verifyLater() + + then: + verifier.verify(Duration.ofSeconds(3)) + } + + @Retry // sometimes it goes too fast or too slow + def "Continue backoff if not restored immediately"() { + when: + def connects = 0 + def flux = DurableFlux.newBuilder() + .using({ + connects++ + def id = connects + if (id in [2, 4,5,6 ]) { + Flux.error(new RuntimeException("[TEST ERROR $id]")) + } else { + Flux.fromIterable([100+id, 200+id]) + .concatWith(Mono.error(new RuntimeException("[TEST ERROR $id]"))) + } + }) + .backoffOnError(new ExponentialBackOff(100, 2)) + .build() + def values = flux.connect() + .take(7) + + def verifier = StepVerifier.create(values) + .expectNext(101, 201) + .as("first batch") + .expectNoEvent(Duration.ofMillis(100 + 200 - CLOCK_ALLOW_ERROR_MS)) + .as("immediate fail on #2") + .expectNext(103, 203) + .as("second batch") + .expectNoEvent(Duration.ofMillis(100 + 200 + 400 + 800 - CLOCK_ALLOW_ERROR_MS)) + .as("three fails in row (after the original) as #4, #5, #6") + .expectNext(107, 207) + .as("third batch") + .expectNoEvent(Duration.ofMillis(100 - CLOCK_ALLOW_ERROR_MS)) + .expectNext(108) + .expectComplete() + .verifyLater() + + then: + verifier.verify(Duration.ofSeconds(3)) + + } +} diff --git a/src/test/groovy/io/emeraldpay/dshackle/commons/ExpiringSetSpec.groovy b/src/test/groovy/io/emeraldpay/dshackle/commons/ExpiringSetSpec.groovy new file mode 100644 index 000000000..ec5e23a37 --- /dev/null +++ b/src/test/groovy/io/emeraldpay/dshackle/commons/ExpiringSetSpec.groovy @@ -0,0 +1,104 @@ +/** + * Copyright (c) 2022 EmeraldPay, Inc + * + * 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 io.emeraldpay.dshackle.commons + +import io.emeraldpay.etherjar.domain.TransactionId +import io.emeraldpay.etherjar.hex.HexDataComparator +import io.emeraldpay.etherjar.tx.Transaction +import spock.lang.Specification + +import java.time.Duration + +class ExpiringSetSpec extends Specification { + + def "Add and check item"() { + setup: + def set = new ExpiringSet(Duration.ofSeconds(60), new HexDataComparator() as Comparator, 100) + + when: + def firstAdded = set.add(TransactionId.from("0xc008b506367e1f96fcfdf6b683e84601434f8b655334bc61dae7970f0fb7d02c")) + def firstExists = set.contains(TransactionId.from("0xc008b506367e1f96fcfdf6b683e84601434f8b655334bc61dae7970f0fb7d02c")) + def secondAdded = set.add(TransactionId.from("0x424e36776777ddd2877df0f9d278c37077d1e00afa84defcfb6b367880d8eb6d")) + def secondExists = set.contains(TransactionId.from("0x424e36776777ddd2877df0f9d278c37077d1e00afa84defcfb6b367880d8eb6d")) + def thirdExists = set.contains(TransactionId.from("0xe218f09c3060099cb7302304e2a29ed7a8693bd5dfe3a55cce96fabce27012e0")) + + then: + firstAdded + secondAdded + firstExists + secondExists + !thirdExists + + set.size == 2 + } + + def "Doesn't grow after limit"() { + setup: + def set = new ExpiringSet(Duration.ofSeconds(60), new HexDataComparator() as Comparator, 3) + + when: + set.add(TransactionId.from("0xc008b506367e1f96fcfdf6b683e84601434f8b655334bc61dae7970f0fb7d02c")) + set.add(TransactionId.from("0x424e36776777ddd2877df0f9d278c37077d1e00afa84defcfb6b367880d8eb6d")) + set.add(TransactionId.from("0xe218f09c3060099cb7302304e2a29ed7a8693bd5dfe3a55cce96fabce27012e0")) + set.add(TransactionId.from("0xd3082daa344a64369c8aace137f22beb5085351bf111859202bac66b70b28bdd")) + + then: + set.size == 3 + + when: + def firstExists = set.contains(TransactionId.from("0xc008b506367e1f96fcfdf6b683e84601434f8b655334bc61dae7970f0fb7d02c")) + def secondExists = set.contains(TransactionId.from("0x424e36776777ddd2877df0f9d278c37077d1e00afa84defcfb6b367880d8eb6d")) + def thirdExists = set.contains(TransactionId.from("0xe218f09c3060099cb7302304e2a29ed7a8693bd5dfe3a55cce96fabce27012e0")) + + then: + !firstExists + secondExists + thirdExists + } + + def "Remove expired"() { + setup: + def set = new ExpiringSet(Duration.ofMillis(100), new HexDataComparator() as Comparator, 100) + + when: + set.add(TransactionId.from("0x1118b506367e1f96fcfdf6b683e84601434f8b655334bc61dae7970f0fb7d02c")) + set.add(TransactionId.from("0x222e36776777ddd2877df0f9d278c37077d1e00afa84defcfb6b367880d8eb6d")) + + then: + set.size == 2 + + when: + Thread.sleep(60) + set.add(TransactionId.from("0x3338f09c3060099cb7302304e2a29ed7a8693bd5dfe3a55cce96fabce27012e0")) + set.add(TransactionId.from("0x44482daa344a64369c8aace137f22beb5085351bf111859202bac66b70b28bdd")) + + then: + set.size == 4 + + when: + Thread.sleep(60) + set.add(TransactionId.from("0x55531d466acf4ef72f7e0fbc60a5c2c9d2845b90a3d27b6d7581575cb119cac9")) + set.add(TransactionId.from("0x6665e1f32cb21aee5f27d804cfc65781d5c140b002776bc073f9405479e8b1e5")) + + then: + set.size == 4 + set.contains(TransactionId.from("0x3338f09c3060099cb7302304e2a29ed7a8693bd5dfe3a55cce96fabce27012e0")) + set.contains(TransactionId.from("0x44482daa344a64369c8aace137f22beb5085351bf111859202bac66b70b28bdd")) + set.contains(TransactionId.from("0x55531d466acf4ef72f7e0fbc60a5c2c9d2845b90a3d27b6d7581575cb119cac9")) + set.contains(TransactionId.from("0x6665e1f32cb21aee5f27d804cfc65781d5c140b002776bc073f9405479e8b1e5")) + } + +} diff --git a/src/test/groovy/io/emeraldpay/dshackle/commons/SharedFluxHolderSpec.groovy b/src/test/groovy/io/emeraldpay/dshackle/commons/SharedFluxHolderSpec.groovy new file mode 100644 index 000000000..60d9c4ac2 --- /dev/null +++ b/src/test/groovy/io/emeraldpay/dshackle/commons/SharedFluxHolderSpec.groovy @@ -0,0 +1,79 @@ +/** + * Copyright (c) 2022 EmeraldPay, Inc + * + * 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 io.emeraldpay.dshackle.commons + +import reactor.core.publisher.Flux +import spock.lang.Specification + +import java.time.Duration + +class SharedFluxHolderSpec extends Specification { + + def "Keeps one flux"() { + when: + def called = 0 + def shared = new SharedFluxHolder({ + Flux.fromIterable([100+called, 200+called, 300+called, 400+called, 500+called]) + .delayElements(Duration.ofMillis(100)) + .doOnSubscribe {called++ } + }) + List values1 = [] + List values2 = [] + new Thread({ + values1 = shared.get().collectList().block(Duration.ofSeconds(1)) + }).start() + new Thread({ + values2 = shared.get().collectList().block(Duration.ofSeconds(1)) + }).start() + Thread.sleep(1000) + + then: + values1 == [100, 200, 300, 400, 500] + values2 == [100, 200, 300, 400, 500] + called == 1 + } + + def "Create a new flux if existing completes"() { + when: + def called = 0 + def shared = new SharedFluxHolder({ + Flux.fromIterable([100+called, 200+called, 300+called, 400+called, 500+called]) + .delayElements(Duration.ofMillis(100)) + .doOnSubscribe {called++ } + }) + List values1 = [] + List values2 = [] + List values3 = [] + new Thread({ + values1 = shared.get().collectList().block(Duration.ofSeconds(1)) + }).start() + new Thread({ + values2 = shared.get().collectList().block(Duration.ofSeconds(1)) + }).start() + Thread.sleep(1000) + + new Thread({ + values3 = shared.get().collectList().block(Duration.ofSeconds(1)) + }).start() + Thread.sleep(1000) + + then: + values1 == [100, 200, 300, 400, 500] + values2 == [100, 200, 300, 400, 500] + values3 == [101, 201, 301, 401, 501] + } + +} diff --git a/src/test/groovy/io/emeraldpay/dshackle/rpc/NativeSubscribeSpec.groovy b/src/test/groovy/io/emeraldpay/dshackle/rpc/NativeSubscribeSpec.groovy index 4e176e2a7..d7e19d843 100644 --- a/src/test/groovy/io/emeraldpay/dshackle/rpc/NativeSubscribeSpec.groovy +++ b/src/test/groovy/io/emeraldpay/dshackle/rpc/NativeSubscribeSpec.groovy @@ -19,7 +19,7 @@ import com.google.protobuf.ByteString import io.emeraldpay.api.proto.BlockchainOuterClass import io.emeraldpay.dshackle.test.MultistreamHolderMock import io.emeraldpay.dshackle.upstream.ethereum.EthereumMultistream -import io.emeraldpay.dshackle.upstream.ethereum.EthereumSubscribe +import io.emeraldpay.dshackle.upstream.ethereum.EthereumSubscriptionApi import io.emeraldpay.grpc.Chain import reactor.core.publisher.Flux import reactor.test.StepVerifier @@ -31,11 +31,11 @@ class NativeSubscribeSpec extends Specification { def "Call with empty params when not provided"() { setup: - def subscribe = Mock(EthereumSubscribe) { + def subscribe = Mock(EthereumSubscriptionApi) { 1 * it.subscribe("newHeads", null) >> Flux.just("{}") } def up = Mock(EthereumMultistream) { - 1 * it.getSubscribe() >> subscribe + 1 * it.getSubscribtionApi() >> subscribe } def nativeSubscribe = new NativeSubscribe(new MultistreamHolderMock(Chain.ETHEREUM, up)) @@ -55,7 +55,7 @@ class NativeSubscribeSpec extends Specification { def "Call with params when provided"() { setup: - def subscribe = Mock(EthereumSubscribe) { + def subscribe = Mock(EthereumSubscriptionApi) { 1 * it.subscribe("logs", { params -> println("params: $params") def ok = params instanceof Map && @@ -67,7 +67,7 @@ class NativeSubscribeSpec extends Specification { }) >> Flux.just("{}") } def up = Mock(EthereumMultistream) { - 1 * it.getSubscribe() >> subscribe + 1 * it.getSubscribtionApi() >> subscribe } def nativeSubscribe = new NativeSubscribe(new MultistreamHolderMock(Chain.ETHEREUM, up)) diff --git a/src/test/groovy/io/emeraldpay/dshackle/rpc/TrackERC20AddressSpec.groovy b/src/test/groovy/io/emeraldpay/dshackle/rpc/TrackERC20AddressSpec.groovy index 062820edc..34d1f01c1 100644 --- a/src/test/groovy/io/emeraldpay/dshackle/rpc/TrackERC20AddressSpec.groovy +++ b/src/test/groovy/io/emeraldpay/dshackle/rpc/TrackERC20AddressSpec.groovy @@ -3,18 +3,13 @@ package io.emeraldpay.dshackle.rpc import io.emeraldpay.api.proto.BlockchainOuterClass import io.emeraldpay.api.proto.Common import io.emeraldpay.dshackle.config.TokensConfig -import io.emeraldpay.dshackle.test.EthereumUpstreamMock -import io.emeraldpay.dshackle.test.MultistreamHolderMock -import io.emeraldpay.dshackle.test.ReaderMock import io.emeraldpay.dshackle.upstream.MultistreamHolder +import io.emeraldpay.dshackle.upstream.SubscriptionConnect import io.emeraldpay.dshackle.upstream.ethereum.ERC20Balance import io.emeraldpay.dshackle.upstream.ethereum.EthereumMultistream -import io.emeraldpay.dshackle.upstream.ethereum.EthereumSubscribe -import io.emeraldpay.dshackle.upstream.ethereum.EthereumUpstream +import io.emeraldpay.dshackle.upstream.ethereum.EthereumSubscriptionApi import io.emeraldpay.dshackle.upstream.ethereum.subscribe.ConnectLogs import io.emeraldpay.dshackle.upstream.ethereum.subscribe.json.LogMessage -import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcRequest -import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcResponse import io.emeraldpay.etherjar.domain.BlockHash import io.emeraldpay.etherjar.domain.TransactionId import io.emeraldpay.etherjar.hex.Hex32 @@ -22,11 +17,9 @@ import io.emeraldpay.grpc.Chain import io.emeraldpay.etherjar.domain.Address import io.emeraldpay.etherjar.erc20.ERC20Token import io.emeraldpay.etherjar.hex.HexData -import io.emeraldpay.etherjar.rpc.json.TransactionCallJson import reactor.core.publisher.Flux import reactor.core.publisher.Mono import reactor.test.StepVerifier -import spock.lang.Ignore import spock.lang.Specification import java.time.Duration @@ -174,20 +167,23 @@ class TrackERC20AddressSpec extends Specification { false ) ] + def logsConnect = Mock(SubscriptionConnect) { + 1 * connect() >> Flux.fromIterable(events) + } def logs = Mock(ConnectLogs) { - 1 * start( + 1 * create( [Address.from("0x54EedeAC495271d0F6B175474E89094C44Da98b9")], [Hex32.from("0xddf252ad1be2c89b69c2b068fc378daa952ba7f163c4a11628f55a4df523b3ef")] ) >> { args -> - println("ConnectLogs.start $args") - Flux.fromIterable(events) + println("ConnectLogs.create $args") + logsConnect } } - def sub = Mock(EthereumSubscribe) { + def sub = Mock(EthereumSubscriptionApi) { 1 * getLogs() >> logs } def up = Mock(EthereumMultistream) { - 1 * getSubscribe() >> sub + 1 * getSubscribtionApi() >> sub _ * cast(EthereumMultistream) >> { args -> it } diff --git a/src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/EthereumSubscribeSpec.groovy b/src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/EthereumSubscriptionApiSpec.groovy similarity index 80% rename from src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/EthereumSubscribeSpec.groovy rename to src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/EthereumSubscriptionApiSpec.groovy index 3a2e116c2..b5fb3bb4b 100644 --- a/src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/EthereumSubscribeSpec.groovy +++ b/src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/EthereumSubscriptionApiSpec.groovy @@ -16,15 +16,16 @@ package io.emeraldpay.dshackle.upstream.ethereum import io.emeraldpay.dshackle.test.TestingCommons +import io.emeraldpay.dshackle.upstream.ethereum.subscribe.PendingTxesSource import io.emeraldpay.etherjar.domain.Address import io.emeraldpay.etherjar.hex.Hex32 import spock.lang.Specification -class EthereumSubscribeSpec extends Specification { +class EthereumSubscriptionApiSpec extends Specification { def "read empty logs request"() { setup: - def ethereumSubscribe = new EthereumSubscribe(TestingCommons.emptyMultistream() as EthereumMultistream) + def ethereumSubscribe = new EthereumSubscriptionApi(TestingCommons.emptyMultistream() as EthereumMultistream, Stub(PendingTxesSource)) when: def act = ethereumSubscribe.readLogsRequest([:]) @@ -35,7 +36,7 @@ class EthereumSubscribeSpec extends Specification { def "read single address logs request"() { setup: - def ethereumSubscribe = new EthereumSubscribe(TestingCommons.emptyMultistream() as EthereumMultistream) + def ethereumSubscribe = new EthereumSubscriptionApi(TestingCommons.emptyMultistream() as EthereumMultistream, Stub(PendingTxesSource)) when: def act = ethereumSubscribe.readLogsRequest([ address: "0x829bd824b016326a401d083b33d092293333a830" @@ -60,7 +61,7 @@ class EthereumSubscribeSpec extends Specification { def "ignores invalid address for logs request"() { setup: - def ethereumSubscribe = new EthereumSubscribe(TestingCommons.emptyMultistream() as EthereumMultistream) + def ethereumSubscribe = new EthereumSubscriptionApi(TestingCommons.emptyMultistream() as EthereumMultistream, Stub(PendingTxesSource)) when: def act = ethereumSubscribe.readLogsRequest([ address: "829bd824b016326a401d083b33d092293333a830" @@ -73,7 +74,7 @@ class EthereumSubscribeSpec extends Specification { def "read multi address logs request"() { setup: - def ethereumSubscribe = new EthereumSubscribe(TestingCommons.emptyMultistream() as EthereumMultistream) + def ethereumSubscribe = new EthereumSubscriptionApi(TestingCommons.emptyMultistream() as EthereumMultistream, Stub(PendingTxesSource)) when: def act = ethereumSubscribe.readLogsRequest([ address: ["0x829bd824b016326a401d083b33d092293333a830", "0x401d083b33d092293333a83829bd824b016326a0"] @@ -89,7 +90,7 @@ class EthereumSubscribeSpec extends Specification { def "read single topic logs request"() { setup: - def ethereumSubscribe = new EthereumSubscribe(TestingCommons.emptyMultistream() as EthereumMultistream) + def ethereumSubscribe = new EthereumSubscriptionApi(TestingCommons.emptyMultistream() as EthereumMultistream, Stub(PendingTxesSource)) when: def act = ethereumSubscribe.readLogsRequest([ topics: "0xddf252ad1be2c89b69c2b068fc378daa952ba7f163c4a11628f55a4df523b3ef" @@ -114,7 +115,7 @@ class EthereumSubscribeSpec extends Specification { def "read invalid topic for request"() { setup: - def ethereumSubscribe = new EthereumSubscribe(TestingCommons.emptyMultistream() as EthereumMultistream) + def ethereumSubscribe = new EthereumSubscriptionApi(TestingCommons.emptyMultistream() as EthereumMultistream, Stub(PendingTxesSource)) when: def act = ethereumSubscribe.readLogsRequest([ topics: [ @@ -132,7 +133,7 @@ class EthereumSubscribeSpec extends Specification { def "read multi topic logs request"() { setup: - def ethereumSubscribe = new EthereumSubscribe(TestingCommons.emptyMultistream() as EthereumMultistream) + def ethereumSubscribe = new EthereumSubscriptionApi(TestingCommons.emptyMultistream() as EthereumMultistream, Stub(PendingTxesSource)) when: def act = ethereumSubscribe.readLogsRequest([ topics: [ @@ -151,7 +152,7 @@ class EthereumSubscribeSpec extends Specification { def "read full logs request"() { setup: - def ethereumSubscribe = new EthereumSubscribe(TestingCommons.emptyMultistream() as EthereumMultistream) + def ethereumSubscribe = new EthereumSubscriptionApi(TestingCommons.emptyMultistream() as EthereumMultistream, Stub(PendingTxesSource)) when: def act = ethereumSubscribe.readLogsRequest([ address: "0x298d492e8c1d909d3f63bc4a36c66c64acb3d695", diff --git a/src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/EthereumWsHeadSpec.groovy b/src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/EthereumWsHeadSpec.groovy new file mode 100644 index 000000000..a252098f3 --- /dev/null +++ b/src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/EthereumWsHeadSpec.groovy @@ -0,0 +1,75 @@ +/** + * Copyright (c) 2022 EmeraldPay, Inc + * + * 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 io.emeraldpay.dshackle.upstream.ethereum + +import io.emeraldpay.dshackle.Global +import io.emeraldpay.dshackle.data.BlockContainer +import io.emeraldpay.dshackle.test.TestingCommons +import io.emeraldpay.etherjar.domain.BlockHash +import io.emeraldpay.etherjar.domain.TransactionId +import io.emeraldpay.etherjar.rpc.json.BlockJson +import io.emeraldpay.etherjar.rpc.json.TransactionRefJson +import io.emeraldpay.grpc.Chain +import reactor.core.publisher.Flux +import reactor.test.StepVerifier +import spock.lang.Specification + +import java.time.Duration +import java.time.Instant +import java.time.temporal.ChronoUnit + +class EthereumWsHeadSpec extends Specification { + + def "Fetch block"() { + setup: + def block = new BlockJson() + block.number = 100 + block.hash = BlockHash.from("0x3ec2ebf5d0ec474d0ac6bc50d2770d8409ad76e119968e7919f85d5ec8915200") + block.timestamp = Instant.now().truncatedTo(ChronoUnit.SECONDS) + block.transactions = [ + new TransactionRefJson(TransactionId.from("0x29229361dc5aa1ec66c323dc7a299e2b61a8c8dd2a3522d41255ec10eca25dd8")), + new TransactionRefJson(TransactionId.from("0xebe8f22a55a9e26892a8545b93cbb2bfa4fd81c3184e50e5cf6276025bb42b93")) + ] + block.uncles = [] + block.totalDifficulty = BigInteger.ONE + + def headBlock = block.copy().tap { + it.transactions = null + }.with { + Global.objectMapper.writeValueAsBytes(it) + } + + def apiMock = TestingCommons.api() + apiMock.answerOnce("eth_getBlockByHash", ["0x3ec2ebf5d0ec474d0ac6bc50d2770d8409ad76e119968e7919f85d5ec8915200", false], block) + + def ws = Mock(WsSubscriptions) + + def head = new EthereumWsHead(apiMock, ws) + + when: + def act = head.listenNewHeads().blockFirst() + + then: + act == BlockContainer.from(block) + act.transactions.size() == 2 + act.transactions[0].toHexWithPrefix() == "0x29229361dc5aa1ec66c323dc7a299e2b61a8c8dd2a3522d41255ec10eca25dd8" + act.transactions[1].toHexWithPrefix() == "0xebe8f22a55a9e26892a8545b93cbb2bfa4fd81c3184e50e5cf6276025bb42b93" + + 1 * ws.subscribe("newHeads") >> Flux.fromIterable([ + headBlock + ]) + } +} diff --git a/src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/WsConnectionRealSpec.groovy b/src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/WsConnectionImplRealSpec.groovy similarity index 72% rename from src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/WsConnectionRealSpec.groovy rename to src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/WsConnectionImplRealSpec.groovy index eb3f22c0c..0b6e45910 100644 --- a/src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/WsConnectionRealSpec.groovy +++ b/src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/WsConnectionImplRealSpec.groovy @@ -11,7 +11,7 @@ import spock.lang.Specification import java.time.Duration -class WsConnectionRealSpec extends Specification { +class WsConnectionImplRealSpec extends Specification { static SLEEP = 500 @@ -19,7 +19,7 @@ class WsConnectionRealSpec extends Specification { @Shared MockWSServer server @Shared - WsConnection conn + WsConnectionImpl conn def setup() { if (System.getenv("CI") == "true") { @@ -31,7 +31,7 @@ class WsConnectionRealSpec extends Specification { server = new MockWSServer(port) server.start() Thread.sleep(SLEEP) - conn = new EthereumWsFactory("test", Chain.ETHEREUM, "ws://localhost:${port}".toURI(), "http://localhost:${port}".toURI()).create(null, null) + conn = new EthereumWsFactory("test", Chain.ETHEREUM, "ws://localhost:${port}".toURI(), "http://localhost:${port}".toURI()).create(null) } def cleanup() { @@ -39,22 +39,10 @@ class WsConnectionRealSpec extends Specification { server.stop() } - def "Connects to server"() { + def "Can make a RPC request"() { when: conn.connect() - Thread.sleep(SLEEP) - println("verify....") - def act = server.received - then: - act.size() > 0 - act[0].value.contains("\"method\":\"eth_subscribe\"") - act[0].value.contains("\"params\":[\"newHeads\"]") - } - - def "Makes RPC request"() { - when: - conn.connect() - def resp = conn.call(new JsonRpcRequest("foo_bar", [])) + def resp = conn.callRpc(new JsonRpcRequest("foo_bar", [])) then: StepVerifier.create(resp) .then { @@ -70,8 +58,8 @@ class WsConnectionRealSpec extends Specification { Thread.sleep(SLEEP) def act = server.received then: - act.size() == 2 - act[1].value.contains("\"method\":\"foo_bar\"") + act.size() == 1 + act[0].value.contains("\"method\":\"foo_bar\"") } def "Reconnects after server disconnect"() { @@ -83,15 +71,15 @@ class WsConnectionRealSpec extends Specification { Thread.sleep(SLEEP) server = new MockWSServer(port) server.start() - def resp = conn.call(new JsonRpcRequest("foo_bar", [])) + server.onNextReply('{"jsonrpc":"2.0","id":100,"result":1}') // reconnects in 2 seconds, give 1 extra Thread.sleep(3_000) + def resp = conn.callRpc(new JsonRpcRequest("foo_bar", [])).block(Duration.ofSeconds(1)) def act = server.received then: - act.size() > 0 - act[0].value.contains("\"method\":\"eth_subscribe\"") - act[0].value.contains("\"params\":[\"newHeads\"]") + act.size() == 1 + act[0].value.contains("\"method\":\"foo_bar\"") } def "Error on request when server disconnects"() { @@ -99,7 +87,7 @@ class WsConnectionRealSpec extends Specification { conn.connect() conn.reconnectIntervalSeconds = 2 - def resp = conn.call(new JsonRpcRequest("foo_bar", [])) + def resp = conn.callRpc(new JsonRpcRequest("foo_bar", [])) then: StepVerifier.create(resp) @@ -113,7 +101,7 @@ class WsConnectionRealSpec extends Specification { def up = Mock(DefaultUpstream) { _ * getId() >> "test" } - conn = new EthereumWsFactory("test", Chain.ETHEREUM, "ws://localhost:${port}".toURI(), "http://localhost:${port}".toURI()).create(up, null) + conn = new EthereumWsFactory("test", Chain.ETHEREUM, "ws://localhost:${port}".toURI(), "http://localhost:${port}".toURI()).create(up) when: conn.connect() conn.reconnectIntervalSeconds = 10 @@ -125,18 +113,6 @@ class WsConnectionRealSpec extends Specification { 1 * up.setStatus(UpstreamAvailability.UNAVAILABLE) } - def "Validates after connect"() { - setup: - def validator = Mock(EthereumUpstreamValidator) - conn = new EthereumWsFactory("test", Chain.ETHEREUM, "ws://localhost:${port}".toURI(), "http://localhost:${port}".toURI()).create(null, validator) - when: - conn.connect() - Thread.sleep(100) - - then: - 1 * validator.validate() - } - def "Try to connects to server until it's available"() { when: server.stop() @@ -146,12 +122,14 @@ class WsConnectionRealSpec extends Specification { Thread.sleep(3_000) server = new MockWSServer(port) server.start() - Thread.sleep(2_000) + server.onNextReply('{"jsonrpc":"2.0","id":100,"result":1}') + Thread.sleep(3_000) + + def resp = conn.callRpc(new JsonRpcRequest("foo_bar", [])).block(Duration.ofSeconds(1)) def act = server.received then: - act.size() > 0 - act[0].value.contains("\"method\":\"eth_subscribe\"") - act[0].value.contains("\"params\":[\"newHeads\"]") + act.size() == 1 + act[0].value.contains("\"method\":\"foo_bar\"") } def "Call after reconnect"() { @@ -166,7 +144,7 @@ class WsConnectionRealSpec extends Specification { // reconnects in 2 seconds, give 1 extra Thread.sleep(3_000) - def resp = conn.call(new JsonRpcRequest("foo_bar", [])) + def resp = conn.callRpc(new JsonRpcRequest("foo_bar", [])) then: StepVerifier.create(resp) .then { diff --git a/src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/WsConnectionSpec.groovy b/src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/WsConnectionImplSpec.groovy similarity index 68% rename from src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/WsConnectionSpec.groovy rename to src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/WsConnectionImplSpec.groovy index b8996287c..501085c7a 100644 --- a/src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/WsConnectionSpec.groovy +++ b/src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/WsConnectionImplSpec.groovy @@ -34,48 +34,14 @@ import java.time.Duration import java.time.Instant import java.time.temporal.ChronoUnit -class WsConnectionSpec extends Specification { - - def "Fetch block"() { - setup: - def wsf = new EthereumWsFactory("test", Chain.ETHEREUM, new URI("http://localhost"), new URI("http://localhost")) - - def block = new BlockJson() - block.number = 100 - block.hash = BlockHash.from("0x3ec2ebf5d0ec474d0ac6bc50d2770d8409ad76e119968e7919f85d5ec8915200") - block.timestamp = Instant.now().truncatedTo(ChronoUnit.SECONDS) - block.transactions = [] - block.uncles = [] - block.totalDifficulty = BigInteger.ONE - - def headBlock = block.copy().tap { - it.transactions = null - } - - def apiMock = TestingCommons.api() - def wsApiMock = apiMock.asWebsocket() - def ws = wsf.create(null, null) - - apiMock.answerOnce("eth_getBlockByHash", ["0x3ec2ebf5d0ec474d0ac6bc50d2770d8409ad76e119968e7919f85d5ec8915200", false], block) - - when: - Flux.from(ws.handle(wsApiMock.inbound, wsApiMock.outbound)).subscribe() - def act = Flux.from(ws.getBlocksFlux()) - - then: - StepVerifier.create(act) - .then { ws.onNewHeads(headBlock).subscribe() } - .expectNext(BlockContainer.from(block)) - .thenCancel() - .verify(Duration.ofSeconds(5)) - } +class WsConnectionImplSpec extends Specification { def "Makes a RPC call"() { setup: def wsf = new EthereumWsFactory("test", Chain.ETHEREUM, new URI("http://localhost"), new URI("http://localhost")) def apiMock = TestingCommons.api() def wsApiMock = apiMock.asWebsocket() - def ws = wsf.create(null, null) + def ws = wsf.create(null) def tx = new TransactionJson().tap { hash = TransactionId.from("0x3ec2ebf5d0ec474d0ac6bc50d2770d8409ad76e119968e7919f85d5ec8915200") @@ -84,7 +50,7 @@ class WsConnectionSpec extends Specification { when: Flux.from(ws.handle(wsApiMock.inbound, wsApiMock.outbound)).subscribe() - def act = ws.call(new JsonRpcRequest("eth_getTransactionByHash", ["0x3ec2ebf5d0ec474d0ac6bc50d2770d8409ad76e119968e7919f85d5ec8915200"], 15, null)) + def act = ws.callRpc(new JsonRpcRequest("eth_getTransactionByHash", ["0x3ec2ebf5d0ec474d0ac6bc50d2770d8409ad76e119968e7919f85d5ec8915200"], 15, null)) then: StepVerifier.create(act) @@ -100,13 +66,13 @@ class WsConnectionSpec extends Specification { def wsf = new EthereumWsFactory("test", Chain.ETHEREUM, new URI("http://localhost"), new URI("http://localhost")) def apiMock = TestingCommons.api() def wsApiMock = apiMock.asWebsocket() - def ws = wsf.create(null, null) + def ws = wsf.create(null) apiMock.answerOnce("eth_getTransactionByHash", ["0x3ec2ebf5d0ec474d0ac6bc50d2770d8409ad76e119968e7919f85d5ec8915200"], null) when: Flux.from(ws.handle(wsApiMock.inbound, wsApiMock.outbound)).subscribe() - def act = ws.call(new JsonRpcRequest("eth_getTransactionByHash", ["0x3ec2ebf5d0ec474d0ac6bc50d2770d8409ad76e119968e7919f85d5ec8915200"], 15, null)) + def act = ws.callRpc(new JsonRpcRequest("eth_getTransactionByHash", ["0x3ec2ebf5d0ec474d0ac6bc50d2770d8409ad76e119968e7919f85d5ec8915200"], 15, null)) then: StepVerifier.create(act) @@ -123,14 +89,14 @@ class WsConnectionSpec extends Specification { def wsf = new EthereumWsFactory("test", Chain.ETHEREUM, new URI("http://localhost"), new URI("http://localhost")) def apiMock = TestingCommons.api() def wsApiMock = apiMock.asWebsocket() - def ws = wsf.create(null, null) + def ws = wsf.create(null) apiMock.answerOnce("eth_getTransactionByHash", ["0x3ec2ebf5d0ec474d0ac6bc50d2770d8409ad76e119968e7919f85d5ec8915200"], new RpcResponseError(RpcResponseError.CODE_METHOD_NOT_EXIST, "test")) when: Flux.from(ws.handle(wsApiMock.inbound, wsApiMock.outbound)).subscribe() - def act = ws.call(new JsonRpcRequest("eth_getTransactionByHash", ["0x3ec2ebf5d0ec474d0ac6bc50d2770d8409ad76e119968e7919f85d5ec8915200"], 15, null)) + def act = ws.callRpc(new JsonRpcRequest("eth_getTransactionByHash", ["0x3ec2ebf5d0ec474d0ac6bc50d2770d8409ad76e119968e7919f85d5ec8915200"], 15, null)) then: StepVerifier.create(act) diff --git a/src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/WsSubscriptionsImplSpec.groovy b/src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/WsSubscriptionsImplSpec.groovy new file mode 100644 index 000000000..7a096106f --- /dev/null +++ b/src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/WsSubscriptionsImplSpec.groovy @@ -0,0 +1,90 @@ +/** + * Copyright (c) 2022 EmeraldPay, Inc + * + * 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 io.emeraldpay.dshackle.upstream.ethereum + +import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcRequest +import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcResponse +import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcWsMessage +import reactor.core.publisher.Flux +import reactor.core.publisher.Mono +import reactor.core.publisher.Sinks +import reactor.test.StepVerifier +import spock.lang.Specification + +import java.time.Duration + +class WsSubscriptionsImplSpec extends Specification { + + def "Makes a subscription"() { + setup: + def answers = Flux.fromIterable( + [ + new JsonRpcWsMessage("100".bytes, null, "0xcff45d00e7"), + new JsonRpcWsMessage("101".bytes, null, "0xcff45d00e7"), + new JsonRpcWsMessage("102".bytes, null, "0xcff45d00e7"), + ] + ) + + def conn = Mock(WsConnectionImpl) + def ws = new WsSubscriptionsImpl(conn) + + when: + def act = ws.subscribe("foo_bar") + .map { new String(it) } + .take(3) + .collectList().block(Duration.ofSeconds(1)) + + then: + act == ["100", "101", "102"] + + 1 * conn.callRpc({ JsonRpcRequest req -> + req.method == "eth_subscribe" && req.params == ["foo_bar"] + }) >> Mono.just(new JsonRpcResponse('"0xcff45d00e7"'.bytes, null)) + 1 * conn.getSubscribeResponses() >> answers + } + + def "Produces only messages to the actual subscription"() { + setup: + def answers = Flux.fromIterable( + [ + new JsonRpcWsMessage("100".bytes, null, "0xcff45d00e7"), + new JsonRpcWsMessage("AAA".bytes, null, "0x000001a0e7"), + new JsonRpcWsMessage("101".bytes, null, "0xcff45d00e7"), + new JsonRpcWsMessage("BBB".bytes, null, "0x000001a0e7"), + new JsonRpcWsMessage("CCC".bytes, null, "0x000001a0e7"), + new JsonRpcWsMessage("102".bytes, null, "0xcff45d00e7"), + new JsonRpcWsMessage("DDD".bytes, null, "0x000001a0e7"), + ] + ) + + def conn = Mock(WsConnectionImpl) + def ws = new WsSubscriptionsImpl(conn) + + when: + def act = ws.subscribe("foo_bar") + .map { new String(it) } + .take(3) + .collectList().block(Duration.ofSeconds(1)) + + then: + act == ["100", "101", "102"] + + 1 * conn.callRpc({ JsonRpcRequest req -> + req.method == "eth_subscribe" && req.params == ["foo_bar"] + }) >> Mono.just(new JsonRpcResponse('"0xcff45d00e7"'.bytes, null)) + 1 * conn.getSubscribeResponses() >> answers + } +} diff --git a/src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/subscribe/AggregatedPendingTxesSpec.groovy b/src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/subscribe/AggregatedPendingTxesSpec.groovy new file mode 100644 index 000000000..6ac30414a --- /dev/null +++ b/src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/subscribe/AggregatedPendingTxesSpec.groovy @@ -0,0 +1,91 @@ +/** + * Copyright (c) 2022 EmeraldPay, Inc + * + * 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 io.emeraldpay.dshackle.upstream.ethereum.subscribe + +import io.emeraldpay.etherjar.domain.TransactionId +import reactor.core.publisher.Flux +import spock.lang.Specification + +import java.time.Duration + +class AggregatedPendingTxesSpec extends Specification { + + def "Produces values from two sources"() { + setup: + def source1 = Mock(PendingTxesSource) + def source2 = Mock(PendingTxesSource) + + when: + def aggregate = new AggregatedPendingTxes([source1, source2]) + def values = aggregate.connect() + .collectList().block(Duration.ofSeconds(1)) + + then: + 1 * source1.connect() >> Flux.fromIterable( + [ + "0xa61bab14fc9720ea8725622688c2f964666d7c2afdae38af7dad53f12f242d5c", + "0x911548eb0f3bf353a54e03a3506c7c3e747470d6c201f03babbc07ff6e14cd6e" + ].collect { TransactionId.from(it) } + ) + 1 * source2.connect() >> Flux.fromIterable( + [ + "0x9a9d4618b12d36d17a63d48c5b5efc05b461feead124ddd86803d8bca4015248", + "0xa38173981f8eab96ee70cefe42735af0f574b7ef354565f2fea32a28e5ed9bd2" + ].collect { TransactionId.from(it) } + ) + + values.collect { it.toHex() }.toSorted() == [ + "0xa61bab14fc9720ea8725622688c2f964666d7c2afdae38af7dad53f12f242d5c", + "0x911548eb0f3bf353a54e03a3506c7c3e747470d6c201f03babbc07ff6e14cd6e", + "0x9a9d4618b12d36d17a63d48c5b5efc05b461feead124ddd86803d8bca4015248", + "0xa38173981f8eab96ee70cefe42735af0f574b7ef354565f2fea32a28e5ed9bd2" + ].toSorted() + } + + def "Skip duplicates"() { + setup: + def source1 = Mock(PendingTxesSource) + def source2 = Mock(PendingTxesSource) + + when: + def aggregate = new AggregatedPendingTxes([source1, source2]) + def values = aggregate.connect() + .collectList().block(Duration.ofSeconds(1)) + + then: + 1 * source1.connect() >> Flux.fromIterable( + [ + "0xa61bab14fc9720ea8725622688c2f964666d7c2afdae38af7dad53f12f242d5c", + "0x9a9d4618b12d36d17a63d48c5b5efc05b461feead124ddd86803d8bca4015248", + "0x911548eb0f3bf353a54e03a3506c7c3e747470d6c201f03babbc07ff6e14cd6e" + ].collect { TransactionId.from(it) } + ) + 1 * source2.connect() >> Flux.fromIterable( + [ + "0x9a9d4618b12d36d17a63d48c5b5efc05b461feead124ddd86803d8bca4015248", + "0xa61bab14fc9720ea8725622688c2f964666d7c2afdae38af7dad53f12f242d5c", + "0xa38173981f8eab96ee70cefe42735af0f574b7ef354565f2fea32a28e5ed9bd2" + ].collect { TransactionId.from(it) } + ) + + values.collect { it.toHex() }.toSorted() == [ + "0xa61bab14fc9720ea8725622688c2f964666d7c2afdae38af7dad53f12f242d5c", + "0x911548eb0f3bf353a54e03a3506c7c3e747470d6c201f03babbc07ff6e14cd6e", + "0x9a9d4618b12d36d17a63d48c5b5efc05b461feead124ddd86803d8bca4015248", + "0xa38173981f8eab96ee70cefe42735af0f574b7ef354565f2fea32a28e5ed9bd2" + ].toSorted() + } +} diff --git a/src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/subscribe/DshacklePendingTxesSourceSpec.groovy b/src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/subscribe/DshacklePendingTxesSourceSpec.groovy new file mode 100644 index 000000000..b8994a846 --- /dev/null +++ b/src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/subscribe/DshacklePendingTxesSourceSpec.groovy @@ -0,0 +1,124 @@ +/** + * Copyright (c) 2022 EmeraldPay, Inc + * + * 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 io.emeraldpay.dshackle.upstream.ethereum.subscribe + +import com.google.protobuf.ByteString +import io.emeraldpay.api.proto.BlockchainGrpc +import io.emeraldpay.api.proto.BlockchainOuterClass +import io.emeraldpay.api.proto.ReactorBlockchainGrpc +import io.emeraldpay.grpc.Chain +import io.grpc.Channel +import io.grpc.ManagedChannel +import io.grpc.Server +import io.grpc.inprocess.InProcessChannelBuilder +import io.grpc.inprocess.InProcessServerBuilder +import io.grpc.stub.StreamObserver +import spock.lang.Specification + +import java.time.Duration + +class DshacklePendingTxesSourceSpec extends Specification { + + def "Produces values"() { + setup: + BlockchainOuterClass.NativeSubscribeRequest receivedRequest + + String uniqueName = InProcessServerBuilder.generateName(); + Server server = InProcessServerBuilder.forName(uniqueName) + .directExecutor() + .addService(new BlockchainGrpc.BlockchainImplBase() { + @Override + void nativeSubscribe(BlockchainOuterClass.NativeSubscribeRequest request, StreamObserver responseObserver) { + receivedRequest = request + [ + '"0xa61bab14fc9720ea8725622688c2f964666d7c2afdae38af7dad53f12f242d5c"', + '"0x911548eb0f3bf353a54e03a3506c7c3e747470d6c201f03babbc07ff6e14cd6e"', + '"0x9a9d4618b12d36d17a63d48c5b5efc05b461feead124ddd86803d8bca4015248"', + '"0xa38173981f8eab96ee70cefe42735af0f574b7ef354565f2fea32a28e5ed9bd2"' + ] + .collect { it.bytes } + .collect { + BlockchainOuterClass.NativeSubscribeReplyItem.newBuilder() + .setPayload(ByteString.copyFrom(it)) + .build() + }.forEach { + responseObserver.onNext(it) + } + responseObserver.onCompleted() + } + }) + .build().start() + ManagedChannel channel = InProcessChannelBuilder.forName(uniqueName) + .directExecutor() + .build() + + def remote = ReactorBlockchainGrpc.newReactorStub(channel) + def pending = new DshacklePendingTxesSource(Chain.ETHEREUM, remote) + + when: + pending.available = true + def txes = pending.connect().take(3) + .collectList().block(Duration.ofSeconds(1)) + + then: + receivedRequest != null + receivedRequest.chainValue == Chain.ETHEREUM.id + receivedRequest.method == "newPendingTransactions" + txes.collect {it.toHex() } == [ + "0xa61bab14fc9720ea8725622688c2f964666d7c2afdae38af7dad53f12f242d5c", + "0x911548eb0f3bf353a54e03a3506c7c3e747470d6c201f03babbc07ff6e14cd6e", + "0x9a9d4618b12d36d17a63d48c5b5efc05b461feead124ddd86803d8bca4015248", + ] + } + + def "available when method is enabled on remote"() { + setup: + def pending = new DshacklePendingTxesSource(Chain.ETHEREUM, ReactorBlockchainGrpc.newReactorStub(Stub(Channel))) + pending.available = false + when: + pending.update( + BlockchainOuterClass.DescribeChain.newBuilder() + .addAllSupportedMethods(["newPendingTransactions"]) + .build() + ) + then: + pending.available + } + + def "unavailable when not method is enabled on remote"() { + setup: + def pending = new DshacklePendingTxesSource(Chain.ETHEREUM, ReactorBlockchainGrpc.newReactorStub(Stub(Channel))) + pending.available = false + when: + pending.update( + BlockchainOuterClass.DescribeChain.newBuilder() + .addAllSupportedMethods(["other_method"]) + .build() + ) + then: + !pending.available + + when: "It was enabled before getting an update" + pending.available = true + pending.update( + BlockchainOuterClass.DescribeChain.newBuilder() + .addAllSupportedMethods(["other_method"]) + .build() + ) + then: + !pending.available + } +} diff --git a/src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/subscribe/WebsocketPendingTxesSpec.groovy b/src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/subscribe/WebsocketPendingTxesSpec.groovy new file mode 100644 index 000000000..c6e37b9b0 --- /dev/null +++ b/src/test/groovy/io/emeraldpay/dshackle/upstream/ethereum/subscribe/WebsocketPendingTxesSpec.groovy @@ -0,0 +1,49 @@ +/** + * Copyright (c) 2022 EmeraldPay, Inc + * + * 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 io.emeraldpay.dshackle.upstream.ethereum.subscribe + +import io.emeraldpay.dshackle.upstream.ethereum.WsSubscriptions +import reactor.core.publisher.Flux +import spock.lang.Specification + +import java.time.Duration + +class WebsocketPendingTxesSpec extends Specification { + + def "Produces values"() { + setup: + def responses = [ + '"0xa61bab14fc9720ea8725622688c2f964666d7c2afdae38af7dad53f12f242d5c"', + '"0x911548eb0f3bf353a54e03a3506c7c3e747470d6c201f03babbc07ff6e14cd6e"', + '"0x67f22a3b441ea312306f97694ca8159f8d6faaccf0f5ce6442c84b13991f1d23"', + '"0xa38173981f8eab96ee70cefe42735af0f574b7ef354565f2fea32a28e5ed9bd2"', + ].collect { it.bytes } + def ws = Mock(WsSubscriptions) + def pending = new WebsocketPendingTxes(ws) + + when: + def txes = pending.connect().take(3) + .collectList().block(Duration.ofSeconds(1)) + + then: + 1 * ws.subscribe("newPendingTransactions") >> Flux.fromIterable(responses) + txes.collect {it.toHex() } == [ + "0xa61bab14fc9720ea8725622688c2f964666d7c2afdae38af7dad53f12f242d5c", + "0x911548eb0f3bf353a54e03a3506c7c3e747470d6c201f03babbc07ff6e14cd6e", + "0x67f22a3b441ea312306f97694ca8159f8d6faaccf0f5ce6442c84b13991f1d23", + ] + } +} diff --git a/src/test/groovy/io/emeraldpay/dshackle/upstream/rpcclient/JsonRpcWsClientSpec.groovy b/src/test/groovy/io/emeraldpay/dshackle/upstream/rpcclient/JsonRpcWsClientSpec.groovy index e0ac8a1bf..77b304352 100644 --- a/src/test/groovy/io/emeraldpay/dshackle/upstream/rpcclient/JsonRpcWsClientSpec.groovy +++ b/src/test/groovy/io/emeraldpay/dshackle/upstream/rpcclient/JsonRpcWsClientSpec.groovy @@ -1,6 +1,6 @@ package io.emeraldpay.dshackle.upstream.rpcclient -import io.emeraldpay.dshackle.upstream.ethereum.WsConnection +import io.emeraldpay.dshackle.upstream.ethereum.WsConnectionImpl import reactor.core.Exceptions import spock.lang.Specification @@ -10,7 +10,7 @@ class JsonRpcWsClientSpec extends Specification { def "Produce error if WS is not connected"() { setup: - def ws = Mock(WsConnection) + def ws = Mock(WsConnectionImpl) def client = new JsonRpcWsClient(ws) when: client.read(new JsonRpcRequest("foo_bar", [], 1)) diff --git a/src/test/resources/log4j2.xml b/src/test/resources/log4j2.xml index aa4a58a87..406b11e13 100644 --- a/src/test/resources/log4j2.xml +++ b/src/test/resources/log4j2.xml @@ -22,7 +22,10 @@ - + + + + From 776e461004a17167cbcd9ba8c53ddf69be873825 Mon Sep 17 00:00:00 2001 From: Igor Artamonov Date: Mon, 10 Oct 2022 23:55:45 -0400 Subject: [PATCH 2/2] problem: formatting --- .../kotlin/io/emeraldpay/dshackle/commons/DurableFlux.kt | 7 +++---- .../kotlin/io/emeraldpay/dshackle/commons/ExpiringSet.kt | 9 ++++----- .../io/emeraldpay/dshackle/commons/SharedFluxHolder.kt | 6 +++--- .../dshackle/upstream/NoUpstreamSubscriptions.kt | 3 +-- .../emeraldpay/dshackle/upstream/SubscriptionConnect.kt | 3 +-- .../dshackle/upstream/UpstreamSubscriptions.kt | 3 +-- .../dshackle/upstream/ethereum/EthereumRpcUpstream.kt | 3 --- .../dshackle/upstream/ethereum/EthereumUpstream.kt | 2 -- .../upstream/ethereum/EthereumUpstreamSubscriptions.kt | 4 ++-- .../dshackle/upstream/ethereum/EthereumWsHead.kt | 3 +-- .../upstream/ethereum/NoEthereumUpstreamSubscriptions.kt | 5 ++--- .../dshackle/upstream/ethereum/WsConnectionImpl.kt | 4 ++-- .../dshackle/upstream/ethereum/WsSubscriptions.kt | 3 +-- .../dshackle/upstream/ethereum/WsSubscriptionsImpl.kt | 9 ++++----- .../upstream/ethereum/subscribe/AggregatedPendingTxes.kt | 7 +++---- .../upstream/ethereum/subscribe/ConnectBlockUpdates.kt | 2 +- .../upstream/ethereum/subscribe/ConnectNewHeads.kt | 2 +- .../upstream/ethereum/subscribe/ConnectSyncing.kt | 2 +- .../ethereum/subscribe/DefaultPendingTxesSource.kt | 6 +++--- .../ethereum/subscribe/DshacklePendingTxesSource.kt | 4 ++-- .../ethereum/subscribe/EthereumDshackleSubscriptions.kt | 2 +- .../ethereum/subscribe/EthereumWsSubscriptions.kt | 5 ++--- .../upstream/ethereum/subscribe/NoPendingTxes.kt | 5 ++--- .../upstream/ethereum/subscribe/PendingTxesSource.kt | 2 +- .../upstream/ethereum/subscribe/WebsocketPendingTxes.kt | 5 ++--- .../ethereum/subscribe/json/TransactionIdSerializer.kt | 5 ++--- .../dshackle/upstream/grpc/EthereumGrpcUpstream.kt | 2 -- .../dshackle/upstream/rpcclient/JsonRpcWsMessage.kt | 3 +-- 28 files changed, 47 insertions(+), 69 deletions(-) diff --git a/src/main/kotlin/io/emeraldpay/dshackle/commons/DurableFlux.kt b/src/main/kotlin/io/emeraldpay/dshackle/commons/DurableFlux.kt index d63429007..505304c37 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/commons/DurableFlux.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/commons/DurableFlux.kt @@ -1,6 +1,5 @@ package io.emeraldpay.dshackle.commons -import java.time.Duration import org.slf4j.Logger import org.slf4j.LoggerFactory import org.springframework.util.backoff.BackOff @@ -9,6 +8,7 @@ import org.springframework.util.backoff.ExponentialBackOff import org.springframework.util.backoff.FixedBackOff import reactor.core.publisher.Flux import reactor.core.publisher.Mono +import java.time.Duration /** * A flux holder that reconnects to it on failure taking into account a back off strategy @@ -95,8 +95,7 @@ class DurableFlux( if (provider == null) { throw IllegalStateException("No provider for original Flux") } - return DurableFlux(provider!!,errorBackOff, log) + return DurableFlux(provider!!, errorBackOff, log) } } - -} \ No newline at end of file +} diff --git a/src/main/kotlin/io/emeraldpay/dshackle/commons/ExpiringSet.kt b/src/main/kotlin/io/emeraldpay/dshackle/commons/ExpiringSet.kt index f62c7a3de..8686d8eed 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/commons/ExpiringSet.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/commons/ExpiringSet.kt @@ -1,12 +1,12 @@ package io.emeraldpay.dshackle.commons +import org.apache.commons.collections4.iterators.UnmodifiableIterator +import org.slf4j.LoggerFactory import java.time.Duration import java.util.LinkedList import java.util.TreeSet import java.util.concurrent.locks.ReentrantLock import kotlin.concurrent.withLock -import org.apache.commons.collections4.iterators.UnmodifiableIterator -import org.slf4j.LoggerFactory /** * A naive implementation of a Set with a limit for elements and an expiration time. Supposed to be used a filter for uniqueness. @@ -17,7 +17,7 @@ class ExpiringSet( ttl: Duration, comparator: Comparator, val limit: Int, -): MutableSet { +) : MutableSet { companion object { private val log = LoggerFactory.getLogger(ExpiringSet::class.java) @@ -149,5 +149,4 @@ class ExpiringSet( count -= removed } } - -} \ No newline at end of file +} diff --git a/src/main/kotlin/io/emeraldpay/dshackle/commons/SharedFluxHolder.kt b/src/main/kotlin/io/emeraldpay/dshackle/commons/SharedFluxHolder.kt index df8605e9e..dc0bcb1b9 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/commons/SharedFluxHolder.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/commons/SharedFluxHolder.kt @@ -1,11 +1,11 @@ package io.emeraldpay.dshackle.commons +import org.slf4j.LoggerFactory +import reactor.core.publisher.Flux import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.locks.ReentrantReadWriteLock import kotlin.concurrent.read import kotlin.concurrent.write -import org.slf4j.LoggerFactory -import reactor.core.publisher.Flux /** * A flux holder that that creates it only if requested. Keeps it for the following calls, so all the following calls will @@ -64,4 +64,4 @@ class SharedFluxHolder( val flux: Flux, val id: Long, ) -} \ No newline at end of file +} diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/NoUpstreamSubscriptions.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/NoUpstreamSubscriptions.kt index a41f58356..245823d5a 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/NoUpstreamSubscriptions.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/NoUpstreamSubscriptions.kt @@ -24,5 +24,4 @@ open class NoUpstreamSubscriptions : UpstreamSubscriptions { override fun get(method: String): SubscriptionConnect? { return null } - -} \ No newline at end of file +} diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/SubscriptionConnect.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/SubscriptionConnect.kt index 27b54b50d..dc56817c0 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/SubscriptionConnect.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/SubscriptionConnect.kt @@ -23,5 +23,4 @@ import reactor.core.publisher.Flux interface SubscriptionConnect { fun connect(): Flux - -} \ No newline at end of file +} diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/UpstreamSubscriptions.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/UpstreamSubscriptions.kt index 4a6c6d964..b1159c5fb 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/UpstreamSubscriptions.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/UpstreamSubscriptions.kt @@ -21,5 +21,4 @@ package io.emeraldpay.dshackle.upstream interface UpstreamSubscriptions { fun get(method: String): SubscriptionConnect? - -} \ No newline at end of file +} diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumRpcUpstream.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumRpcUpstream.kt index a70835993..9283a1c3e 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumRpcUpstream.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumRpcUpstream.kt @@ -8,15 +8,12 @@ import io.emeraldpay.dshackle.startup.QuorumForLabels import io.emeraldpay.dshackle.upstream.ForkWatch import io.emeraldpay.dshackle.upstream.Head import io.emeraldpay.dshackle.upstream.MergedHead -import io.emeraldpay.dshackle.upstream.NoUpstreamSubscriptions import io.emeraldpay.dshackle.upstream.Upstream import io.emeraldpay.dshackle.upstream.UpstreamAvailability -import io.emeraldpay.dshackle.upstream.UpstreamSubscriptions import io.emeraldpay.dshackle.upstream.calls.CallMethods import io.emeraldpay.dshackle.upstream.calls.DirectCallMethods import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcRequest import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcResponse -import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcWsClient import io.emeraldpay.grpc.Chain import org.slf4j.LoggerFactory import org.springframework.context.Lifecycle diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumUpstream.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumUpstream.kt index 92e9e5c21..5f078ffbe 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumUpstream.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumUpstream.kt @@ -21,9 +21,7 @@ import io.emeraldpay.dshackle.startup.QuorumForLabels import io.emeraldpay.dshackle.upstream.Capability import io.emeraldpay.dshackle.upstream.DefaultUpstream import io.emeraldpay.dshackle.upstream.ForkWatch -import io.emeraldpay.dshackle.upstream.UpstreamSubscriptions import io.emeraldpay.dshackle.upstream.calls.CallMethods -import io.emeraldpay.dshackle.upstream.ethereum.subscribe.PendingTxesSource abstract class EthereumUpstream( id: String, diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumUpstreamSubscriptions.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumUpstreamSubscriptions.kt index eb29a2d7f..f646ff684 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumUpstreamSubscriptions.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumUpstreamSubscriptions.kt @@ -18,7 +18,7 @@ package io.emeraldpay.dshackle.upstream.ethereum import io.emeraldpay.dshackle.upstream.UpstreamSubscriptions import io.emeraldpay.dshackle.upstream.ethereum.subscribe.PendingTxesSource -interface EthereumUpstreamSubscriptions: UpstreamSubscriptions { +interface EthereumUpstreamSubscriptions : UpstreamSubscriptions { fun getPendingTxes(): PendingTxesSource? -} \ No newline at end of file +} diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumWsHead.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumWsHead.kt index 4f2cfd2c8..397a05e99 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumWsHead.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/EthereumWsHead.kt @@ -23,10 +23,8 @@ import io.emeraldpay.dshackle.data.BlockContainer import io.emeraldpay.dshackle.reader.Reader import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcRequest import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcResponse -import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcWsClient import io.emeraldpay.etherjar.rpc.json.BlockJson import io.emeraldpay.etherjar.rpc.json.TransactionRefJson -import java.time.Duration import org.slf4j.LoggerFactory import org.springframework.context.Lifecycle import reactor.core.Disposable @@ -34,6 +32,7 @@ import reactor.core.publisher.Flux import reactor.core.publisher.Mono import reactor.core.scheduler.Schedulers import reactor.retry.Repeat +import java.time.Duration class EthereumWsHead( private val api: Reader, diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/NoEthereumUpstreamSubscriptions.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/NoEthereumUpstreamSubscriptions.kt index 332e459e7..ded75eece 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/NoEthereumUpstreamSubscriptions.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/NoEthereumUpstreamSubscriptions.kt @@ -18,7 +18,7 @@ package io.emeraldpay.dshackle.upstream.ethereum import io.emeraldpay.dshackle.upstream.NoUpstreamSubscriptions import io.emeraldpay.dshackle.upstream.ethereum.subscribe.PendingTxesSource -class NoEthereumUpstreamSubscriptions: NoUpstreamSubscriptions(), EthereumUpstreamSubscriptions { +class NoEthereumUpstreamSubscriptions : NoUpstreamSubscriptions(), EthereumUpstreamSubscriptions { companion object { val DEFAULT = NoEthereumUpstreamSubscriptions() @@ -27,5 +27,4 @@ class NoEthereumUpstreamSubscriptions: NoUpstreamSubscriptions(), EthereumUpstre override fun getPendingTxes(): PendingTxesSource? { return null } - -} \ No newline at end of file +} diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/WsConnectionImpl.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/WsConnectionImpl.kt index 618abced3..2b90fd475 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/WsConnectionImpl.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/WsConnectionImpl.kt @@ -20,11 +20,11 @@ import io.emeraldpay.dshackle.Global import io.emeraldpay.dshackle.config.AuthConfig import io.emeraldpay.dshackle.upstream.DefaultUpstream import io.emeraldpay.dshackle.upstream.UpstreamAvailability -import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcWsMessage import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcError import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcException import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcRequest import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcResponse +import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcWsMessage import io.emeraldpay.dshackle.upstream.rpcclient.ResponseWSParser import io.emeraldpay.dshackle.upstream.rpcclient.RpcMetrics import io.emeraldpay.etherjar.rpc.RpcResponseError @@ -64,7 +64,7 @@ open class WsConnectionImpl( private val basicAuth: AuthConfig.ClientBasicAuth?, private val rpcMetrics: RpcMetrics?, private val upstream: DefaultUpstream?, -): AutoCloseable { +) : AutoCloseable { companion object { private val log = LoggerFactory.getLogger(WsConnectionImpl::class.java) diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/WsSubscriptions.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/WsSubscriptions.kt index c4b3eb222..34589a338 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/WsSubscriptions.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/WsSubscriptions.kt @@ -38,5 +38,4 @@ interface WsSubscriptions { * Subscribe on remote */ fun subscribe(method: String): Flux - -} \ No newline at end of file +} diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/WsSubscriptionsImpl.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/WsSubscriptionsImpl.kt index 5ba098305..ea2c44b1e 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/WsSubscriptionsImpl.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/WsSubscriptionsImpl.kt @@ -17,16 +17,15 @@ package io.emeraldpay.dshackle.upstream.ethereum import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcException import io.emeraldpay.dshackle.upstream.rpcclient.JsonRpcRequest -import java.util.concurrent.atomic.AtomicLong -import java.util.concurrent.atomic.AtomicReference -import org.apache.commons.lang3.StringUtils import org.slf4j.LoggerFactory import reactor.core.publisher.Flux import reactor.core.publisher.Mono +import java.util.concurrent.atomic.AtomicLong +import java.util.concurrent.atomic.AtomicReference class WsSubscriptionsImpl( val conn: WsConnectionImpl, -): WsSubscriptions { +) : WsSubscriptions { companion object { private val log = LoggerFactory.getLogger(WsSubscriptionsImpl::class.java) @@ -52,4 +51,4 @@ class WsSubscriptionsImpl( } } } -} \ No newline at end of file +} diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/AggregatedPendingTxes.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/AggregatedPendingTxes.kt index ad28fcff5..af259bc05 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/AggregatedPendingTxes.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/AggregatedPendingTxes.kt @@ -18,13 +18,13 @@ package io.emeraldpay.dshackle.upstream.ethereum.subscribe import io.emeraldpay.dshackle.commons.ExpiringSet import io.emeraldpay.etherjar.domain.TransactionId import io.emeraldpay.etherjar.hex.HexDataComparator -import java.time.Duration import org.slf4j.LoggerFactory import reactor.core.publisher.Flux +import java.time.Duration class AggregatedPendingTxes( private val sources: List -): PendingTxesSource { +) : PendingTxesSource { companion object { private val log = LoggerFactory.getLogger(AggregatedPendingTxes::class.java) @@ -41,5 +41,4 @@ class AggregatedPendingTxes( sources.map(PendingTxesSource::connect) ).filter(track::add) } - -} \ No newline at end of file +} diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/ConnectBlockUpdates.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/ConnectBlockUpdates.kt index a5a9c642d..a48b9bc80 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/ConnectBlockUpdates.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/ConnectBlockUpdates.kt @@ -34,7 +34,7 @@ import kotlin.concurrent.write class ConnectBlockUpdates( private val upstream: EthereumMultistream -): SubscriptionConnect { +) : SubscriptionConnect { companion object { private val log = LoggerFactory.getLogger(ConnectBlockUpdates::class.java) diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/ConnectNewHeads.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/ConnectNewHeads.kt index 3c28b6566..375dca31f 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/ConnectNewHeads.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/ConnectNewHeads.kt @@ -30,7 +30,7 @@ import kotlin.concurrent.withLock */ class ConnectNewHeads( private val upstream: EthereumMultistream -): SubscriptionConnect { +) : SubscriptionConnect { companion object { private val log = LoggerFactory.getLogger(ConnectNewHeads::class.java) diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/ConnectSyncing.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/ConnectSyncing.kt index c086d12f0..b6e741fad 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/ConnectSyncing.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/ConnectSyncing.kt @@ -26,7 +26,7 @@ import kotlin.concurrent.withLock class ConnectSyncing( private val upstream: EthereumMultistream -): SubscriptionConnect { +) : SubscriptionConnect { companion object { private val log = LoggerFactory.getLogger(ConnectSyncing::class.java) diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/DefaultPendingTxesSource.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/DefaultPendingTxesSource.kt index d7fdc1ac2..f38575046 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/DefaultPendingTxesSource.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/DefaultPendingTxesSource.kt @@ -19,10 +19,10 @@ import io.emeraldpay.dshackle.commons.DurableFlux import io.emeraldpay.dshackle.commons.SharedFluxHolder import io.emeraldpay.dshackle.upstream.SubscriptionConnect import io.emeraldpay.etherjar.domain.TransactionId -import java.time.Duration import reactor.core.publisher.Flux +import java.time.Duration -abstract class DefaultPendingTxesSource: SubscriptionConnect, PendingTxesSource { +abstract class DefaultPendingTxesSource : SubscriptionConnect, PendingTxesSource { private val connectionSource = DurableFlux .newBuilder() @@ -38,4 +38,4 @@ abstract class DefaultPendingTxesSource: SubscriptionConnect, Pen } abstract fun createConnection(): Flux -} \ No newline at end of file +} diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/DshacklePendingTxesSource.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/DshacklePendingTxesSource.kt index 46af5671b..4eada9422 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/DshacklePendingTxesSource.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/DshacklePendingTxesSource.kt @@ -27,7 +27,7 @@ import reactor.core.publisher.Flux class DshacklePendingTxesSource( private val blockchain: Chain, private val conn: ReactorBlockchainGrpc.ReactorBlockchainStub, -): PendingTxesSource, DefaultPendingTxesSource() { +) : PendingTxesSource, DefaultPendingTxesSource() { private val request = NativeSubscribeRequest.newBuilder() .setChainValue(blockchain.id) @@ -56,4 +56,4 @@ class DshacklePendingTxesSource( it == EthereumSubscriptionApi.METHOD_PENDING_TXES } } -} \ No newline at end of file +} diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/EthereumDshackleSubscriptions.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/EthereumDshackleSubscriptions.kt index b968fceb7..f0e9061fc 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/EthereumDshackleSubscriptions.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/EthereumDshackleSubscriptions.kt @@ -49,4 +49,4 @@ class EthereumDshackleSubscriptions( override fun getPendingTxes(): PendingTxesSource? { return pendingTxes } -} \ No newline at end of file +} diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/EthereumWsSubscriptions.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/EthereumWsSubscriptions.kt index b57919848..b60ed3abe 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/EthereumWsSubscriptions.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/EthereumWsSubscriptions.kt @@ -24,7 +24,7 @@ import org.slf4j.LoggerFactory class EthereumWsSubscriptions( private val conn: WsSubscriptions -): UpstreamSubscriptions, EthereumUpstreamSubscriptions { +) : UpstreamSubscriptions, EthereumUpstreamSubscriptions { companion object { private val log = LoggerFactory.getLogger(EthereumWsSubscriptions::class.java) @@ -42,5 +42,4 @@ class EthereumWsSubscriptions( override fun getPendingTxes(): PendingTxesSource? { return pendingTxes } - -} \ No newline at end of file +} diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/NoPendingTxes.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/NoPendingTxes.kt index 909efb76b..145b8f326 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/NoPendingTxes.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/NoPendingTxes.kt @@ -19,7 +19,7 @@ import io.emeraldpay.etherjar.domain.TransactionId import org.slf4j.LoggerFactory import reactor.core.publisher.Flux -class NoPendingTxes: PendingTxesSource { +class NoPendingTxes : PendingTxesSource { companion object { private val log = LoggerFactory.getLogger(NoPendingTxes::class.java) @@ -30,5 +30,4 @@ class NoPendingTxes: PendingTxesSource { override fun connect(): Flux { return Flux.empty() } - -} \ No newline at end of file +} diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/PendingTxesSource.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/PendingTxesSource.kt index 22bd4dba3..9af538d5a 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/PendingTxesSource.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/PendingTxesSource.kt @@ -22,4 +22,4 @@ import io.emeraldpay.etherjar.domain.TransactionId * A source to subscribe to newPendingTransactions. * When using a Websocket RPC on a node, it produces hashes of new transactions received. */ -interface PendingTxesSource: SubscriptionConnect \ No newline at end of file +interface PendingTxesSource : SubscriptionConnect diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/WebsocketPendingTxes.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/WebsocketPendingTxes.kt index 474239a6e..3e59fc185 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/WebsocketPendingTxes.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/WebsocketPendingTxes.kt @@ -19,10 +19,10 @@ import io.emeraldpay.dshackle.upstream.SubscriptionConnect import io.emeraldpay.dshackle.upstream.ethereum.EthereumSubscriptionApi import io.emeraldpay.dshackle.upstream.ethereum.WsSubscriptions import io.emeraldpay.etherjar.domain.TransactionId -import java.time.Duration import org.slf4j.LoggerFactory import reactor.core.publisher.Flux import reactor.core.publisher.Mono +import java.time.Duration class WebsocketPendingTxes( private val wsSubscriptions: WsSubscriptions @@ -44,5 +44,4 @@ class WebsocketPendingTxes( .doOnError { t -> log.warn("Invalid pending transaction", t) } .onErrorResume { Mono.empty() } } - -} \ No newline at end of file +} diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/json/TransactionIdSerializer.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/json/TransactionIdSerializer.kt index 96adf5985..55d8ae18f 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/json/TransactionIdSerializer.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/ethereum/subscribe/json/TransactionIdSerializer.kt @@ -21,7 +21,7 @@ import com.fasterxml.jackson.databind.SerializerProvider import io.emeraldpay.etherjar.domain.TransactionId import org.slf4j.LoggerFactory -class TransactionIdSerializer: JsonSerializer() { +class TransactionIdSerializer : JsonSerializer() { companion object { private val log = LoggerFactory.getLogger(TransactionIdSerializer::class.java) @@ -34,5 +34,4 @@ class TransactionIdSerializer: JsonSerializer() { } gen.writeString(value.toHex()) } - -} \ No newline at end of file +} diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/grpc/EthereumGrpcUpstream.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/grpc/EthereumGrpcUpstream.kt index be043b899..ec7463af8 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/grpc/EthereumGrpcUpstream.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/grpc/EthereumGrpcUpstream.kt @@ -17,7 +17,6 @@ package io.emeraldpay.dshackle.upstream.grpc import io.emeraldpay.api.proto.BlockchainOuterClass -import io.emeraldpay.api.proto.ReactorBlockchainGrpc import io.emeraldpay.api.proto.ReactorBlockchainGrpc.ReactorBlockchainStub import io.emeraldpay.dshackle.Defaults import io.emeraldpay.dshackle.config.UpstreamsConfig @@ -32,7 +31,6 @@ import io.emeraldpay.dshackle.upstream.OptionalHead import io.emeraldpay.dshackle.upstream.Selector import io.emeraldpay.dshackle.upstream.Upstream import io.emeraldpay.dshackle.upstream.UpstreamAvailability -import io.emeraldpay.dshackle.upstream.UpstreamSubscriptions import io.emeraldpay.dshackle.upstream.calls.CallMethods import io.emeraldpay.dshackle.upstream.ethereum.EthereumUpstream import io.emeraldpay.dshackle.upstream.ethereum.EthereumUpstreamSubscriptions diff --git a/src/main/kotlin/io/emeraldpay/dshackle/upstream/rpcclient/JsonRpcWsMessage.kt b/src/main/kotlin/io/emeraldpay/dshackle/upstream/rpcclient/JsonRpcWsMessage.kt index e701ffbef..9c74d39ab 100644 --- a/src/main/kotlin/io/emeraldpay/dshackle/upstream/rpcclient/JsonRpcWsMessage.kt +++ b/src/main/kotlin/io/emeraldpay/dshackle/upstream/rpcclient/JsonRpcWsMessage.kt @@ -19,5 +19,4 @@ class JsonRpcWsMessage( val result: ByteArray?, val error: JsonRpcError?, val subscriptionId: String, -) { -} \ No newline at end of file +)