From 3ff27dee684d4e107573cf4d707c67e90ed768d7 Mon Sep 17 00:00:00 2001 From: Dominik Zajkowski Date: Tue, 1 Jun 2021 09:22:06 +0200 Subject: [PATCH 01/15] [ETCM-841] Change protocol version to a full name --- src/main/resources/conf/base.conf | 2 +- src/main/resources/conf/pottery.conf | 2 +- .../conf/testnet-internal-nomad.conf | 2 +- .../sync/regular/BlockBroadcast.scala | 2 +- .../ethereum/jsonrpc/EthInfoService.scala | 19 ++++++++++--------- .../io/iohk/ethereum/network/PeerActor.scala | 7 +++---- .../ethereum/network/PeerManagerActor.scala | 8 ++++---- .../network/handshaker/EtcHandshaker.scala | 4 ++-- .../handshaker/EtcHelloExchangeState.scala | 2 +- .../EtcNodeStatus63ExchangeState.scala | 2 +- .../EtcNodeStatus64ExchangeState.scala | 2 +- .../iohk/ethereum/network/p2p/Message.scala | 10 +++------- .../network/p2p/MessageDecoders.scala | 7 +++---- .../network/p2p/messages/Capability.scala | 14 +++++++++++--- .../network/p2p/messages/package.scala | 10 ++++------ .../ethereum/network/rlpx/MessageCodec.scala | 4 ++-- .../network/rlpx/RLPxConnectionHandler.scala | 10 +++++----- .../ethereum/nodebuilder/NodeBuilder.scala | 7 ++++--- .../scala/io/iohk/ethereum/utils/Config.scala | 2 +- 19 files changed, 59 insertions(+), 57 deletions(-) diff --git a/src/main/resources/conf/base.conf b/src/main/resources/conf/base.conf index 15ffa0a158..a0e2a11e44 100644 --- a/src/main/resources/conf/base.conf +++ b/src/main/resources/conf/base.conf @@ -37,7 +37,7 @@ mantis { # Ethereum protocol version # Supported versions: # 63, 64 (experimental version which enables usage of messages with checkpointing information. In the future after ETCM-355, ETCM-356, it will be 66 probably) - protocol-version = 63 + protocol-version = "eth/63" server-address { # Listening interface for Ethereum protocol connections diff --git a/src/main/resources/conf/pottery.conf b/src/main/resources/conf/pottery.conf index ce4df4a6e5..f6b4e7fa4f 100644 --- a/src/main/resources/conf/pottery.conf +++ b/src/main/resources/conf/pottery.conf @@ -6,7 +6,7 @@ mantis { } network { - protocol-version = 64 + protocol-version = "etc/64" discovery { discovery-enabled = true diff --git a/src/main/resources/conf/testnet-internal-nomad.conf b/src/main/resources/conf/testnet-internal-nomad.conf index 539065ef62..7c5cef589d 100644 --- a/src/main/resources/conf/testnet-internal-nomad.conf +++ b/src/main/resources/conf/testnet-internal-nomad.conf @@ -6,7 +6,7 @@ mantis { } network { - protocol-version = 64 + protocol-version = "etc/64" discovery { discovery-enabled = true diff --git a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockBroadcast.scala b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockBroadcast.scala index ed98cab56d..4cb514935b 100644 --- a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockBroadcast.scala +++ b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockBroadcast.scala @@ -40,7 +40,7 @@ class BlockBroadcast(val etcPeerManager: ActorRef) { private def broadcastNewBlock(blockToBroadcast: BlockToBroadcast, peers: Map[PeerId, PeerWithInfo]): Unit = obtainRandomPeerSubset(peers.values.map(_.peer).toSet).foreach { peer => val message: MessageSerializable = - if (peers(peer.id).peerInfo.remoteStatus.protocolVersion == ProtocolVersions.PV64) blockToBroadcast.as64 + if (peers(peer.id).peerInfo.remoteStatus.protocolVersion.toByte == ProtocolVersions.PV64.version) blockToBroadcast.as64 else blockToBroadcast.as63 etcPeerManager ! EtcPeerManagerActor.SendMessage(message, peer.id) } diff --git a/src/main/scala/io/iohk/ethereum/jsonrpc/EthInfoService.scala b/src/main/scala/io/iohk/ethereum/jsonrpc/EthInfoService.scala index 455c4978ee..e76b9cee52 100644 --- a/src/main/scala/io/iohk/ethereum/jsonrpc/EthInfoService.scala +++ b/src/main/scala/io/iohk/ethereum/jsonrpc/EthInfoService.scala @@ -11,6 +11,7 @@ import io.iohk.ethereum.domain.{BlockHeader, _} import io.iohk.ethereum.jsonrpc.AkkaTaskOps._ import io.iohk.ethereum.keystore.KeyStore import io.iohk.ethereum.ledger.{InMemoryWorldStateProxy, Ledger, StxLedger} +import io.iohk.ethereum.network.p2p.messages.Capability import io.iohk.ethereum.rlp import io.iohk.ethereum.rlp.RLPImplicitConversions._ import io.iohk.ethereum.rlp.RLPImplicits._ @@ -69,20 +70,20 @@ object EthInfoService { } class EthInfoService( - val blockchain: Blockchain, - blockchainConfig: BlockchainConfig, - val ledger: Ledger, - stxLedger: StxLedger, - keyStore: KeyStore, - syncingController: ActorRef, - protocolVersion: Int, - askTimeout: Timeout + val blockchain: Blockchain, + blockchainConfig: BlockchainConfig, + val ledger: Ledger, + stxLedger: StxLedger, + keyStore: KeyStore, + syncingController: ActorRef, + protocolVersion: Capability, + askTimeout: Timeout ) extends ResolveBlock { import EthInfoService._ def protocolVersion(req: ProtocolVersionRequest): ServiceResponse[ProtocolVersionResponse] = - Task.now(Right(ProtocolVersionResponse(f"0x$protocolVersion%x"))) + Task.now(Right(ProtocolVersionResponse(f"0x${protocolVersion.version}%x"))) def chainId(req: ChainIdRequest): ServiceResponse[ChainIdResponse] = Task.now(Right(ChainIdResponse(blockchainConfig.chainId))) diff --git a/src/main/scala/io/iohk/ethereum/network/PeerActor.scala b/src/main/scala/io/iohk/ethereum/network/PeerActor.scala index e4a958aa12..644afca8fa 100644 --- a/src/main/scala/io/iohk/ethereum/network/PeerActor.scala +++ b/src/main/scala/io/iohk/ethereum/network/PeerActor.scala @@ -1,7 +1,6 @@ package io.iohk.ethereum.network import java.net.{InetSocketAddress, URI} - import akka.actor.SupervisorStrategy.Escalate import akka.actor._ import akka.util.ByteString @@ -12,8 +11,8 @@ import io.iohk.ethereum.network.PeerManagerActor.PeerConfiguration import io.iohk.ethereum.network.handshaker.Handshaker import io.iohk.ethereum.network.handshaker.Handshaker.HandshakeComplete.{HandshakeFailure, HandshakeSuccess} import io.iohk.ethereum.network.handshaker.Handshaker.{HandshakeResult, NextMessage} -import io.iohk.ethereum.network.p2p.Message.Version import io.iohk.ethereum.network.p2p._ +import io.iohk.ethereum.network.p2p.messages.Capability import io.iohk.ethereum.network.p2p.messages.WireProtocol._ import io.iohk.ethereum.network.rlpx.RLPxConnectionHandler.RLPxConfiguration import io.iohk.ethereum.network.rlpx.{AuthHandshaker, RLPxConnectionHandler} @@ -303,7 +302,7 @@ object PeerActor { handshaker: Handshaker[R], authHandshaker: AuthHandshaker, messageDecoder: MessageDecoder, - bestProtocolVersion: Version + bestProtocolVersion: Capability ): Props = Props( new PeerActor( @@ -322,7 +321,7 @@ object PeerActor { authHandshaker: AuthHandshaker, messageDecoder: MessageDecoder, rlpxConfiguration: RLPxConfiguration, - bestProtocolVersion: Version + bestProtocolVersion: Capability ): ActorContext => ActorRef = { ctx => ctx.actorOf( RLPxConnectionHandler diff --git a/src/main/scala/io/iohk/ethereum/network/PeerManagerActor.scala b/src/main/scala/io/iohk/ethereum/network/PeerManagerActor.scala index 48dc795a91..86b230a38d 100644 --- a/src/main/scala/io/iohk/ethereum/network/PeerManagerActor.scala +++ b/src/main/scala/io/iohk/ethereum/network/PeerManagerActor.scala @@ -13,7 +13,7 @@ import io.iohk.ethereum.network.PeerManagerActor.PeerConfiguration import io.iohk.ethereum.network.discovery.{DiscoveryConfig, Node, PeerDiscoveryManager} import io.iohk.ethereum.network.handshaker.Handshaker import io.iohk.ethereum.network.handshaker.Handshaker.HandshakeResult -import io.iohk.ethereum.network.p2p.Message.Version +import io.iohk.ethereum.network.p2p.messages.Capability import io.iohk.ethereum.network.p2p.messages.WireProtocol.Disconnect import io.iohk.ethereum.network.p2p.{MessageDecoder, MessageSerializable} import io.iohk.ethereum.network.rlpx.AuthHandshaker @@ -21,9 +21,9 @@ import io.iohk.ethereum.network.rlpx.RLPxConnectionHandler.RLPxConfiguration import monix.eval.Task import monix.execution.{Scheduler => MonixScheduler} import org.bouncycastle.util.encoders.Hex + import java.net.{InetSocketAddress, URI} import java.util.Collections.newSetFromMap - import scala.collection.mutable import scala.concurrent.duration._ import scala.jdk.CollectionConverters._ @@ -424,7 +424,7 @@ object PeerManagerActor { messageDecoder: MessageDecoder, discoveryConfig: DiscoveryConfig, blacklist: Blacklist, - bestProtocolVersion: Version + bestProtocolVersion: Capability ): Props = { val factory: (ActorContext, InetSocketAddress, Boolean) => ActorRef = peerFactory( @@ -459,7 +459,7 @@ object PeerManagerActor { handshaker: Handshaker[R], authHandshaker: AuthHandshaker, messageDecoder: MessageDecoder, - bestProtocolVersion: Version + bestProtocolVersion: Capability ): (ActorContext, InetSocketAddress, Boolean) => ActorRef = { (ctx, address, incomingConnection) => val id: String = address.toString.filterNot(_ == '/') val props = PeerActor.props( diff --git a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHandshaker.scala b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHandshaker.scala index f63a0fca82..f1c1f318c9 100644 --- a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHandshaker.scala +++ b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHandshaker.scala @@ -1,12 +1,12 @@ package io.iohk.ethereum.network.handshaker import java.util.concurrent.atomic.AtomicReference - import io.iohk.ethereum.db.storage.AppStateStorage import io.iohk.ethereum.domain.Blockchain import io.iohk.ethereum.network.EtcPeerManagerActor.PeerInfo import io.iohk.ethereum.network.ForkResolver import io.iohk.ethereum.network.PeerManagerActor.PeerConfiguration +import io.iohk.ethereum.network.p2p.messages.Capability import io.iohk.ethereum.utils.NodeStatus case class EtcHandshaker private ( @@ -35,5 +35,5 @@ trait EtcHandshakerConfiguration { val appStateStorage: AppStateStorage val peerConfiguration: PeerConfiguration val forkResolverOpt: Option[ForkResolver] - val protocolVersion: Int + val protocolVersion: Capability } diff --git a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala index d33a14d4cc..140fc2c0a2 100644 --- a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala +++ b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala @@ -34,7 +34,7 @@ case class EtcHelloExchangeState(handshakerConfiguration: EtcHandshakerConfigura EtcNodeStatus63ExchangeState(handshakerConfiguration) else { log.debug( - s"Connected peer does not support eth ${ProtocolVersions.PV63.toByte} / ${ProtocolVersions.PV64.toByte} protocol. Disconnecting." + s"Connected peer does not support eth ${ProtocolVersions.PV63} / ${ProtocolVersions.PV64} protocol. Disconnecting." ) DisconnectedState(Disconnect.Reasons.IncompatibleP2pProtocolVersion) } diff --git a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcNodeStatus63ExchangeState.scala b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcNodeStatus63ExchangeState.scala index d1614655fb..7768b837b5 100644 --- a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcNodeStatus63ExchangeState.scala +++ b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcNodeStatus63ExchangeState.scala @@ -20,7 +20,7 @@ case class EtcNodeStatus63ExchangeState( val chainWeight = blockchain.getChainWeightByHash(bestBlockHeader.hash).get val status = CommonMessages.Status( - protocolVersion = ProtocolVersions.PV63, + protocolVersion = ProtocolVersions.PV63.version, networkId = peerConfiguration.networkId, totalDifficulty = chainWeight.totalDifficulty, bestHash = bestBlockHeader.hash, diff --git a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcNodeStatus64ExchangeState.scala b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcNodeStatus64ExchangeState.scala index 837c8b9e37..cb9b9ce55b 100644 --- a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcNodeStatus64ExchangeState.scala +++ b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcNodeStatus64ExchangeState.scala @@ -19,7 +19,7 @@ case class EtcNodeStatus64ExchangeState( val chainWeight = blockchain.getChainWeightByHash(bestBlockHeader.hash).get val status = PV64.Status( - protocolVersion = ProtocolVersions.PV64, + protocolVersion = ProtocolVersions.PV64.version, networkId = peerConfiguration.networkId, chainWeight = chainWeight, bestHash = bestBlockHeader.hash, diff --git a/src/main/scala/io/iohk/ethereum/network/p2p/Message.scala b/src/main/scala/io/iohk/ethereum/network/p2p/Message.scala index 2636e0f54b..d9f3f5893b 100644 --- a/src/main/scala/io/iohk/ethereum/network/p2p/Message.scala +++ b/src/main/scala/io/iohk/ethereum/network/p2p/Message.scala @@ -1,14 +1,10 @@ package io.iohk.ethereum.network.p2p import akka.util.ByteString -import io.iohk.ethereum.network.p2p.Message.Version +import io.iohk.ethereum.network.p2p.messages.Capability import scala.util.Try -object Message { - type Version = Int -} - trait Message { def code: Int def toShortString: String @@ -25,10 +21,10 @@ trait MessageSerializable extends Message { } trait MessageDecoder { self => - def fromBytes(`type`: Int, payload: Array[Byte], protocolVersion: Message.Version): Message + def fromBytes(`type`: Int, payload: Array[Byte], protocolVersion: Capability): Message def orElse(otherMessageDecoder: MessageDecoder): MessageDecoder = new MessageDecoder { - override def fromBytes(`type`: Int, payload: Array[Byte], protocolVersion: Version): Message = + override def fromBytes(`type`: Int, payload: Array[Byte], protocolVersion: Capability): Message = Try { self.fromBytes(`type`, payload, protocolVersion) } .getOrElse(otherMessageDecoder.fromBytes(`type`, payload, protocolVersion)) } diff --git a/src/main/scala/io/iohk/ethereum/network/p2p/MessageDecoders.scala b/src/main/scala/io/iohk/ethereum/network/p2p/MessageDecoders.scala index 6b528d5367..b7a57137c5 100644 --- a/src/main/scala/io/iohk/ethereum/network/p2p/MessageDecoders.scala +++ b/src/main/scala/io/iohk/ethereum/network/p2p/MessageDecoders.scala @@ -1,7 +1,6 @@ package io.iohk.ethereum.network.p2p -import io.iohk.ethereum.network.p2p.Message.Version -import io.iohk.ethereum.network.p2p.messages.Codes +import io.iohk.ethereum.network.p2p.messages.{Capability, Codes} import io.iohk.ethereum.network.p2p.messages.CommonMessages.SignedTransactions._ import io.iohk.ethereum.network.p2p.messages.PV61.BlockHashesFromNumber._ import io.iohk.ethereum.network.p2p.messages.PV62.BlockBodies._ @@ -22,7 +21,7 @@ import io.iohk.ethereum.network.p2p.messages.WireProtocol._ object NetworkMessageDecoder extends MessageDecoder { - override def fromBytes(msgCode: Int, payload: Array[Byte], protocolVersion: Version): Message = + override def fromBytes(msgCode: Int, payload: Array[Byte], protocolVersion: Capability): Message = msgCode match { case Disconnect.code => payload.toDisconnect case Ping.code => payload.toPing @@ -36,7 +35,7 @@ object NetworkMessageDecoder extends MessageDecoder { // scalastyle:off object EthereumMessageDecoder extends MessageDecoder { - override def fromBytes(msgCode: Int, payload: Array[Byte], protocolVersion: Version): Message = { + override def fromBytes(msgCode: Int, payload: Array[Byte], protocolVersion: Capability): Message = { protocolVersion match { case PV64 => handlePV64(msgCode, payload) case PV63 => handlePV63(msgCode, payload) diff --git a/src/main/scala/io/iohk/ethereum/network/p2p/messages/Capability.scala b/src/main/scala/io/iohk/ethereum/network/p2p/messages/Capability.scala index 6618e69c8b..8e38ca974a 100644 --- a/src/main/scala/io/iohk/ethereum/network/p2p/messages/Capability.scala +++ b/src/main/scala/io/iohk/ethereum/network/p2p/messages/Capability.scala @@ -7,6 +7,14 @@ import io.iohk.ethereum.rlp.{RLPEncodeable, RLPException, RLPList, RLPSerializab case class Capability(name: String, version: Byte) object Capability { + private val pattern = "(.*)/(\\d*)".r + + def from(protocolVersion: String): Capability = + protocolVersion match { + case pattern(name, version) => Capability(name, version.toByte) + case _ => throw new RuntimeException(s"Unable to parse capability $protocolVersion") + } + implicit class CapabilityEnc(val msg: Capability) extends RLPSerializable { override def toRLPEncodable: RLPEncodeable = RLPList(msg.name, msg.version) } @@ -23,9 +31,9 @@ object Capability { } object Capabilities { - val Eth63Capability: Capability = Capability("eth", ProtocolVersions.PV63.toByte) - val Etc64Capability: Capability = Capability("etc", ProtocolVersions.PV64.toByte) + val Eth63Capability: Capability = ProtocolVersions.PV63 + val Etc64Capability: Capability = ProtocolVersions.PV64 - val All: Seq[Capability] = Seq(Etc64Capability, Eth63Capability) + val All: Seq[Capability] = Seq(ProtocolVersions.PV64, ProtocolVersions.PV63) } } diff --git a/src/main/scala/io/iohk/ethereum/network/p2p/messages/package.scala b/src/main/scala/io/iohk/ethereum/network/p2p/messages/package.scala index 589ddff766..ebe83e5248 100644 --- a/src/main/scala/io/iohk/ethereum/network/p2p/messages/package.scala +++ b/src/main/scala/io/iohk/ethereum/network/p2p/messages/package.scala @@ -1,13 +1,11 @@ package io.iohk.ethereum.network.p2p -import io.iohk.ethereum.network.p2p.Message.Version - package object messages { object ProtocolVersions { - val PV61: Version = 61 - val PV62: Version = 62 - val PV63: Version = 63 - val PV64: Version = 64 + val PV61: Capability = Capability("eth", 61.toByte) + val PV62: Capability = Capability("eth", 62.toByte) + val PV63: Capability = Capability("eth", 63.toByte) + val PV64: Capability = Capability("etc", 64.toByte) val SubProtocolOffset = 0x10 } diff --git a/src/main/scala/io/iohk/ethereum/network/rlpx/MessageCodec.scala b/src/main/scala/io/iohk/ethereum/network/rlpx/MessageCodec.scala index ec3e9dc91e..bf81fdee56 100644 --- a/src/main/scala/io/iohk/ethereum/network/rlpx/MessageCodec.scala +++ b/src/main/scala/io/iohk/ethereum/network/rlpx/MessageCodec.scala @@ -1,16 +1,16 @@ package io.iohk.ethereum.network.rlpx import java.util.concurrent.atomic.AtomicInteger - import akka.util.ByteString import io.iohk.ethereum.network.handshaker.EtcHelloExchangeState +import io.iohk.ethereum.network.p2p.messages.Capability import io.iohk.ethereum.network.p2p.messages.WireProtocol.Hello import io.iohk.ethereum.network.p2p.{Message, MessageDecoder, MessageSerializable} import org.xerial.snappy.Snappy import scala.util.{Failure, Success, Try} -class MessageCodec(frameCodec: FrameCodec, messageDecoder: MessageDecoder, protocolVersion: Message.Version) { +class MessageCodec(frameCodec: FrameCodec, messageDecoder: MessageDecoder, protocolVersion: Capability) { val MaxFramePayloadSize: Int = Int.MaxValue // no framing diff --git a/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala b/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala index f288604c3e..6076955930 100644 --- a/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala +++ b/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala @@ -1,11 +1,11 @@ package io.iohk.ethereum.network.rlpx import java.net.{InetSocketAddress, URI} - import akka.actor._ import akka.io.Tcp._ import akka.io.{IO, Tcp} import akka.util.ByteString +import io.iohk.ethereum.network.p2p.messages.Capability import io.iohk.ethereum.network.p2p.{Message, MessageDecoder, MessageSerializable} import io.iohk.ethereum.network.rlpx.RLPxConnectionHandler.RLPxConfiguration import io.iohk.ethereum.utils.ByteUtils @@ -28,9 +28,9 @@ import scala.util.{Failure, Success, Try} */ class RLPxConnectionHandler( messageDecoder: MessageDecoder, - protocolVersion: Message.Version, + protocolVersion: Capability, authHandshaker: AuthHandshaker, - messageCodecFactory: (Secrets, MessageDecoder, Message.Version) => MessageCodec, + messageCodecFactory: (Secrets, MessageDecoder, Capability) => MessageCodec, rlpxConfiguration: RLPxConfiguration ) extends Actor with ActorLogging { @@ -279,7 +279,7 @@ class RLPxConnectionHandler( object RLPxConnectionHandler { def props( messageDecoder: MessageDecoder, - protocolVersion: Int, + protocolVersion: Capability, authHandshaker: AuthHandshaker, rlpxConfiguration: RLPxConfiguration ): Props = @@ -290,7 +290,7 @@ object RLPxConnectionHandler { def messageCodecFactory( secrets: Secrets, messageDecoder: MessageDecoder, - protocolVersion: Message.Version + protocolVersion: Capability ): MessageCodec = new MessageCodec(new FrameCodec(secrets), messageDecoder, protocolVersion) diff --git a/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala b/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala index db84439ffc..b6d16cc801 100644 --- a/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala +++ b/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala @@ -48,6 +48,7 @@ import scala.util.{Failure, Success, Try} import akka.util.ByteString import monix.execution.Scheduler import cats.implicits._ +import io.iohk.ethereum.network.p2p.messages.Capability import monix.eval.Task // scalastyle:off number.of.types @@ -183,7 +184,7 @@ trait HandshakerBuilder { override val peerConfiguration: PeerConfiguration = self.peerConfiguration override val blockchain: Blockchain = self.blockchain override val appStateStorage: AppStateStorage = self.storagesInstance.storages.appStateStorage - override val protocolVersion: Int = Config.Network.protocolVersion + override val protocolVersion: Capability = Capability.from(Config.Network.protocolVersion) } lazy val handshaker: Handshaker[PeerInfo] = EtcHandshaker(handshakerConfiguration) @@ -246,7 +247,7 @@ trait PeerManagerActorBuilder { EthereumMessageDecoder, discoveryConfig, blacklist, - Config.Network.protocolVersion + Capability.Capabilities.Eth63Capability // TODO replace with a list of capabilities ), "peer-manager" ) @@ -412,7 +413,7 @@ trait EthInfoServiceBuilder { stxLedger, keyStore, syncController, - Config.Network.protocolVersion, + Capability.from(Config.Network.protocolVersion), asyncConfig.askTimeout ) } diff --git a/src/main/scala/io/iohk/ethereum/utils/Config.scala b/src/main/scala/io/iohk/ethereum/utils/Config.scala index ad51873595..0861d7c5ce 100644 --- a/src/main/scala/io/iohk/ethereum/utils/Config.scala +++ b/src/main/scala/io/iohk/ethereum/utils/Config.scala @@ -40,7 +40,7 @@ object Config { object Network { private val networkConfig = config.getConfig("network") - val protocolVersion = networkConfig.getInt("protocol-version") + val protocolVersion = networkConfig.getString("protocol-version") val automaticPortForwarding = networkConfig.getBoolean("automatic-port-forwarding") From ad40a9fd73c137eeae6d9c81d7123f659a3774f7 Mon Sep 17 00:00:00 2001 From: Dominik Zajkowski Date: Tue, 1 Jun 2021 15:58:48 +0200 Subject: [PATCH 02/15] [ETCM-841] Remove general purpose protocol version, replace with capabilities list --- .../ethereum/sync/util/CommonFakePeer.scala | 6 ++-- .../txExecTest/util/DumpChainApp.scala | 6 ++-- src/main/resources/conf/base.conf | 11 ++---- src/main/resources/conf/chains/etc-chain.conf | 8 +++-- src/main/resources/conf/chains/eth-chain.conf | 2 ++ .../resources/conf/chains/mordor-chain.conf | 2 ++ .../resources/conf/chains/pottery-chain.conf | 2 ++ .../resources/conf/chains/ropsten-chain.conf | 2 ++ .../resources/conf/chains/test-chain.conf | 2 ++ .../chains/testnet-internal-gac-chain.conf | 2 ++ .../chains/testnet-internal-nomad-chain.conf | 2 ++ src/main/resources/conf/pottery.conf | 2 -- .../conf/testnet-internal-nomad.conf | 2 -- .../sync/regular/BlockBroadcast.scala | 3 +- .../ethereum/jsonrpc/EthInfoService.scala | 16 ++++----- .../network/handshaker/EtcHandshaker.scala | 2 +- .../handshaker/EtcHelloExchangeState.scala | 10 +++--- .../network/rlpx/RLPxConnectionHandler.scala | 4 +-- .../ethereum/nodebuilder/NodeBuilder.scala | 7 ++-- .../ethereum/utils/BlockchainConfig.scala | 9 +++-- .../scala/io/iohk/ethereum/utils/Config.scala | 2 -- .../iohk/ethereum/rpcTest/RpcApiTests.scala | 5 +-- .../blockchain/sync/BlockBroadcastSpec.scala | 4 +-- .../blockchain/sync/PeersClientSpec.scala | 2 +- .../sync/PivotBlockSelectorSpec.scala | 2 +- .../blockchain/sync/StateSyncSpec.scala | 2 +- .../blockchain/sync/TestSyncPeers.scala | 2 +- .../FastSyncBranchResolverActorSpec.scala | 2 +- .../sync/regular/RegularSyncFixtures.scala | 2 +- .../sync/regular/RegularSyncSpec.scala | 2 +- .../ethereum/jsonrpc/DebugServiceSpec.scala | 2 +- .../ethereum/jsonrpc/EthInfoServiceSpec.scala | 3 +- .../jsonrpc/JsonRpcControllerFixture.scala | 3 +- .../jsonrpc/JsonRpcControllerSpec.scala | 2 +- .../ethereum/network/EtcPeerManagerSpec.scala | 4 +-- .../network/PeerActorHandshakingSpec.scala | 2 +- .../network/PeerEventBusActorSpec.scala | 2 +- .../ethereum/network/PeerManagerSpec.scala | 2 +- .../handshaker/EtcHandshakerSpec.scala | 19 ++++++----- .../network/p2p/MessageCodecSpec.scala | 2 +- .../network/p2p/MessageDecodersSpec.scala | 6 ++-- .../ethereum/network/p2p/PeerActorSpec.scala | 34 +++++++++---------- .../messages/MessagesSerializationSpec.scala | 2 +- .../rlpx/RLPxConnectionHandlerSpec.scala | 6 ++-- 44 files changed, 113 insertions(+), 101 deletions(-) diff --git a/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala b/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala index 0bb4bc3afa..4fb1d0520b 100644 --- a/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala +++ b/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala @@ -3,7 +3,6 @@ package io.iohk.ethereum.sync.util import java.nio.file.Files import java.time.Clock import java.util.concurrent.atomic.AtomicReference - import akka.actor.{ActorRef, ActorSystem} import akka.testkit.TestProbe import akka.util.{ByteString, Timeout} @@ -25,6 +24,7 @@ import io.iohk.ethereum.network.discovery.PeerDiscoveryManager.DiscoveredNodesIn import io.iohk.ethereum.network.discovery.{DiscoveryConfig, Node} import io.iohk.ethereum.network.handshaker.{EtcHandshaker, EtcHandshakerConfiguration, Handshaker} import io.iohk.ethereum.network.p2p.EthereumMessageDecoder +import io.iohk.ethereum.network.p2p.messages.Capability import io.iohk.ethereum.network.rlpx.AuthHandshaker import io.iohk.ethereum.network.rlpx.RLPxConnectionHandler.RLPxConfiguration import io.iohk.ethereum.network.{ @@ -169,7 +169,7 @@ abstract class CommonFakePeer(peerName: String, fakePeerCustomConfig: FakePeerCu override val peerConfiguration: PeerConfiguration = peerConf override val blockchain: Blockchain = bl override val appStateStorage: AppStateStorage = storagesInstance.storages.appStateStorage - override val protocolVersion: Int = Config.Network.protocolVersion + override val capabilities: List[Capability] = blockchainConfig.capabilities } lazy val handshaker: Handshaker[PeerInfo] = EtcHandshaker(handshakerConfiguration) @@ -193,7 +193,7 @@ abstract class CommonFakePeer(peerName: String, fakePeerCustomConfig: FakePeerCu EthereumMessageDecoder, discoveryConfig, blacklist, - Config.Network.protocolVersion + blockchainConfig.capabilities.head ), "peer-manager" ) diff --git a/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala b/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala index c25bd4fb32..59b8359192 100644 --- a/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala +++ b/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala @@ -2,7 +2,6 @@ package io.iohk.ethereum.txExecTest.util import java.time.Clock import java.util.concurrent.atomic.AtomicReference - import akka.actor.ActorSystem import akka.util.ByteString import com.typesafe.config.ConfigFactory @@ -25,6 +24,7 @@ import io.iohk.ethereum.network.PeerStatisticsActor import io.iohk.ethereum.network.discovery.DiscoveryConfig import io.iohk.ethereum.network.handshaker.{EtcHandshaker, EtcHandshakerConfiguration, Handshaker} import io.iohk.ethereum.network.p2p.EthereumMessageDecoder +import io.iohk.ethereum.network.p2p.messages.Capability import io.iohk.ethereum.network.rlpx.RLPxConnectionHandler.RLPxConfiguration import io.iohk.ethereum.network.{ForkResolver, PeerEventBusActor, PeerManagerActor} import io.iohk.ethereum.nodebuilder.{AuthHandshakerBuilder, NodeKeyBuilder} @@ -93,7 +93,7 @@ object DumpChainApp extends App with NodeKeyBuilder with SecureRandomBuilder wit override val peerConfiguration: PeerConfiguration = peerConfig override val blockchain: Blockchain = DumpChainApp.blockchain override val appStateStorage: AppStateStorage = storagesInstance.storages.appStateStorage - override val protocolVersion: Int = Config.Network.protocolVersion + override val capabilities: List[Capability] = blockchainConfig.capabilities } lazy val handshaker: Handshaker[PeerInfo] = EtcHandshaker(handshakerConfiguration) @@ -116,7 +116,7 @@ object DumpChainApp extends App with NodeKeyBuilder with SecureRandomBuilder wit messageDecoder = EthereumMessageDecoder, discoveryConfig = discoveryConfig, blacklist = blacklist, - bestProtocolVersion = Config.Network.protocolVersion + bestProtocolVersion = blockchainConfig.capabilities.head ), "peer-manager" ) diff --git a/src/main/resources/conf/base.conf b/src/main/resources/conf/base.conf index a0e2a11e44..8e4a31f21c 100644 --- a/src/main/resources/conf/base.conf +++ b/src/main/resources/conf/base.conf @@ -34,11 +34,6 @@ mantis { } network { - # Ethereum protocol version - # Supported versions: - # 63, 64 (experimental version which enables usage of messages with checkpointing information. In the future after ETCM-355, ETCM-356, it will be 66 probably) - protocol-version = "eth/63" - server-address { # Listening interface for Ethereum protocol connections interface = "0.0.0.0" @@ -328,15 +323,15 @@ mantis { mordor {include required("chains/mordor-chain.conf")} + pottery {include required("chains/pottery-chain.conf")} + ropsten {include required("chains/ropsten-chain.conf")} test {include required("chains/test-chain.conf")} - testnet-internal-nomad {include required("chains/testnet-internal-nomad-chain.conf")} - testnet-internal-gac {include required("chains/testnet-internal-gac-chain.conf")} - pottery {include required("chains/pottery-chain.conf")} + testnet-internal-nomad {include required("chains/testnet-internal-nomad-chain.conf")} } sync { diff --git a/src/main/resources/conf/chains/etc-chain.conf b/src/main/resources/conf/chains/etc-chain.conf index 242c7d063c..df654edd66 100644 --- a/src/main/resources/conf/chains/etc-chain.conf +++ b/src/main/resources/conf/chains/etc-chain.conf @@ -3,6 +3,11 @@ # 1 - mainnet, 3 - ropsten, 7 - mordor network-id = 1 + # The ID of the accepted chain + chain-id = "0x3d" + + capabilities = ["eth/63"] + # Possibility to set Proof of Work target time for testing purposes. # null means that the standard difficulty calculation rules are used pow-target-time = null @@ -126,9 +131,6 @@ # Starting nonce of an empty account. Some networks (like Morden) use different values. account-start-nonce = "0" - # The ID of the accepted chain - chain-id = "0x3d" - # Custom genesis JSON file path # null value indicates using default genesis definition that matches the main network custom-genesis-file = null diff --git a/src/main/resources/conf/chains/eth-chain.conf b/src/main/resources/conf/chains/eth-chain.conf index b7e0e3aee0..fe00e198c4 100644 --- a/src/main/resources/conf/chains/eth-chain.conf +++ b/src/main/resources/conf/chains/eth-chain.conf @@ -3,6 +3,8 @@ # 1 - mainnet, 3 - ropsten, 7 - mordor network-id = 1 + capabilities = ["eth/63"] + # Possibility to set Proof of Work target time for testing purposes. # null means that the standard difficulty calculation rules are used pow-target-time = null diff --git a/src/main/resources/conf/chains/mordor-chain.conf b/src/main/resources/conf/chains/mordor-chain.conf index b5e2971e79..f88b6c17ec 100644 --- a/src/main/resources/conf/chains/mordor-chain.conf +++ b/src/main/resources/conf/chains/mordor-chain.conf @@ -3,6 +3,8 @@ # 1 - mainnet, 3 - ropsten, 7 - mordor network-id = 7 + capabilities = ["eth/63"] + # Possibility to set Proof of Work target time for testing purposes. # null means that the standard difficulty calculation rules are used pow-target-time = null diff --git a/src/main/resources/conf/chains/pottery-chain.conf b/src/main/resources/conf/chains/pottery-chain.conf index 8e72f5603f..0db53bd8f9 100644 --- a/src/main/resources/conf/chains/pottery-chain.conf +++ b/src/main/resources/conf/chains/pottery-chain.conf @@ -2,6 +2,8 @@ include "mordor-chain.conf" # 1 - mainnet, 3 - ropsten, 7 - mordor, 9 - pottery network-id = 9 + capabilities = ["etc/64"] + ecip1098-block-number = "0" ecip1097-block-number = "0" diff --git a/src/main/resources/conf/chains/ropsten-chain.conf b/src/main/resources/conf/chains/ropsten-chain.conf index b713cc1691..6816ef5f9b 100644 --- a/src/main/resources/conf/chains/ropsten-chain.conf +++ b/src/main/resources/conf/chains/ropsten-chain.conf @@ -3,6 +3,8 @@ # 1 - mainnet, 3 - ropsten, 7 - mordor network-id = 3 + capabilities = ["eth/63"] + # Possibility to set Proof of Work target time for testing purposes. # null means that the standard difficulty calculation rules are used pow-target-time = null diff --git a/src/main/resources/conf/chains/test-chain.conf b/src/main/resources/conf/chains/test-chain.conf index 7a8fec32fd..b6166249a3 100644 --- a/src/main/resources/conf/chains/test-chain.conf +++ b/src/main/resources/conf/chains/test-chain.conf @@ -3,6 +3,8 @@ # 1 - mainnet, 7 - mordor network-id = 1 + capabilities = ["eth/63"] + # Possibility to set Proof of Work target time for testing purposes. # null means that the standard difficulty calculation rules are used pow-target-time = null diff --git a/src/main/resources/conf/chains/testnet-internal-gac-chain.conf b/src/main/resources/conf/chains/testnet-internal-gac-chain.conf index 497b50f869..5950bc4503 100644 --- a/src/main/resources/conf/chains/testnet-internal-gac-chain.conf +++ b/src/main/resources/conf/chains/testnet-internal-gac-chain.conf @@ -3,6 +3,8 @@ # 1 - mainnet, 3 - ropsten, 7 - mordor network-id = 42 + capabilities = ["eth/63"] + # Possibility to set Proof of Work target time for testing purposes. # null means that the standard difficulty calculation rules are used pow-target-time = null diff --git a/src/main/resources/conf/chains/testnet-internal-nomad-chain.conf b/src/main/resources/conf/chains/testnet-internal-nomad-chain.conf index 770b2ba677..02fa0fc1f7 100644 --- a/src/main/resources/conf/chains/testnet-internal-nomad-chain.conf +++ b/src/main/resources/conf/chains/testnet-internal-nomad-chain.conf @@ -3,6 +3,8 @@ # 1 - mainnet, 3 - ropsten, 7 - mordor network-id = 42 + capabilities = ["etc/64"] + # Possibility to set Proof of Work target time for testing purposes. # null means that the standard difficulty calculation rules are used pow-target-time = 15 seconds diff --git a/src/main/resources/conf/pottery.conf b/src/main/resources/conf/pottery.conf index f6b4e7fa4f..8df720f23f 100644 --- a/src/main/resources/conf/pottery.conf +++ b/src/main/resources/conf/pottery.conf @@ -6,8 +6,6 @@ mantis { } network { - protocol-version = "etc/64" - discovery { discovery-enabled = true } diff --git a/src/main/resources/conf/testnet-internal-nomad.conf b/src/main/resources/conf/testnet-internal-nomad.conf index 7c5cef589d..1e83ea16e2 100644 --- a/src/main/resources/conf/testnet-internal-nomad.conf +++ b/src/main/resources/conf/testnet-internal-nomad.conf @@ -6,8 +6,6 @@ mantis { } network { - protocol-version = "etc/64" - discovery { discovery-enabled = true } diff --git a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockBroadcast.scala b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockBroadcast.scala index 4cb514935b..e7400a7a31 100644 --- a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockBroadcast.scala +++ b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockBroadcast.scala @@ -40,7 +40,8 @@ class BlockBroadcast(val etcPeerManager: ActorRef) { private def broadcastNewBlock(blockToBroadcast: BlockToBroadcast, peers: Map[PeerId, PeerWithInfo]): Unit = obtainRandomPeerSubset(peers.values.map(_.peer).toSet).foreach { peer => val message: MessageSerializable = - if (peers(peer.id).peerInfo.remoteStatus.protocolVersion.toByte == ProtocolVersions.PV64.version) blockToBroadcast.as64 + if (peers(peer.id).peerInfo.remoteStatus.protocolVersion.toByte == ProtocolVersions.PV64.version) + blockToBroadcast.as64 else blockToBroadcast.as63 etcPeerManager ! EtcPeerManagerActor.SendMessage(message, peer.id) } diff --git a/src/main/scala/io/iohk/ethereum/jsonrpc/EthInfoService.scala b/src/main/scala/io/iohk/ethereum/jsonrpc/EthInfoService.scala index e76b9cee52..b17f424020 100644 --- a/src/main/scala/io/iohk/ethereum/jsonrpc/EthInfoService.scala +++ b/src/main/scala/io/iohk/ethereum/jsonrpc/EthInfoService.scala @@ -70,14 +70,14 @@ object EthInfoService { } class EthInfoService( - val blockchain: Blockchain, - blockchainConfig: BlockchainConfig, - val ledger: Ledger, - stxLedger: StxLedger, - keyStore: KeyStore, - syncingController: ActorRef, - protocolVersion: Capability, - askTimeout: Timeout + val blockchain: Blockchain, + blockchainConfig: BlockchainConfig, + val ledger: Ledger, + stxLedger: StxLedger, + keyStore: KeyStore, + syncingController: ActorRef, + protocolVersion: Capability, + askTimeout: Timeout ) extends ResolveBlock { import EthInfoService._ diff --git a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHandshaker.scala b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHandshaker.scala index f1c1f318c9..0da7c37db0 100644 --- a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHandshaker.scala +++ b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHandshaker.scala @@ -35,5 +35,5 @@ trait EtcHandshakerConfiguration { val appStateStorage: AppStateStorage val peerConfiguration: PeerConfiguration val forkResolverOpt: Option[ForkResolver] - val protocolVersion: Capability + val capabilities: List[Capability] } diff --git a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala index 140fc2c0a2..b0757ffcb5 100644 --- a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala +++ b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala @@ -27,14 +27,15 @@ case class EtcHelloExchangeState(handshakerConfiguration: EtcHandshakerConfigura override def applyResponseMessage: PartialFunction[Message, HandshakerState[PeerInfo]] = { case hello: Hello => log.debug("Protocol handshake finished with peer ({})", hello) if ( - handshakerConfiguration.protocolVersion == ProtocolVersions.PV64 && hello.capabilities.contains(Etc64Capability) + handshakerConfiguration.capabilities + .contains(ProtocolVersions.PV64) && hello.capabilities.contains(Etc64Capability) ) EtcNodeStatus64ExchangeState(handshakerConfiguration) else if (hello.capabilities.contains(Eth63Capability)) EtcNodeStatus63ExchangeState(handshakerConfiguration) else { log.debug( - s"Connected peer does not support eth ${ProtocolVersions.PV63} / ${ProtocolVersions.PV64} protocol. Disconnecting." + s"Connected peer does not support ${ProtocolVersions.PV63} / ${ProtocolVersions.PV64} protocol. Disconnecting." ) DisconnectedState(Disconnect.Reasons.IncompatibleP2pProtocolVersion) } @@ -52,13 +53,10 @@ case class EtcHelloExchangeState(handshakerConfiguration: EtcHandshakerConfigura case ServerStatus.Listening(address) => address.getPort case ServerStatus.NotListening => 0 } - val capabilities = - if (handshakerConfiguration.protocolVersion == ProtocolVersions.PV64) Capabilities.All else Seq(Eth63Capability) - Hello( p2pVersion = EtcHelloExchangeState.P2pVersion, clientId = Config.clientId, - capabilities = capabilities, + capabilities = handshakerConfiguration.capabilities, listenPort = listenPort, nodeId = ByteString(nodeStatus.nodeId) ) diff --git a/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala b/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala index 6076955930..78a7b166b6 100644 --- a/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala +++ b/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala @@ -146,7 +146,7 @@ class RLPxConnectionHandler( def processHandshakeResult(result: AuthHandshakeResult, remainingData: ByteString): Unit = result match { case AuthHandshakeSuccess(secrets, remotePubKey) => - log.debug(s"Auth handshake succeeded for peer $peerId") + log.info(s"Auth handshake succeeded for peer $peerId") context.parent ! ConnectionEstablished(remotePubKey) val messageCodec = messageCodecFactory(secrets, messageDecoder, protocolVersion) val messagesSoFar = messageCodec.readMessages(remainingData) @@ -164,7 +164,7 @@ class RLPxConnectionHandler( context.parent ! MessageReceived(message) case Failure(ex) => - log.debug(s"Cannot decode message from $peerId, because of ${ex.getMessage}") + log.info(s"Cannot decode message from $peerId, because of ${ex.getMessage}") // break connection in case of failed decoding, to avoid attack which would send us garbage context stop self } diff --git a/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala b/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala index b6d16cc801..0e8d3e6461 100644 --- a/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala +++ b/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala @@ -175,7 +175,8 @@ trait HandshakerBuilder { with NodeStatusBuilder with StorageBuilder with PeerManagerActorBuilder - with ForkResolverBuilder => + with ForkResolverBuilder + with BlockchainConfigBuilder => private val handshakerConfiguration: EtcHandshakerConfiguration = new EtcHandshakerConfiguration { @@ -184,7 +185,7 @@ trait HandshakerBuilder { override val peerConfiguration: PeerConfiguration = self.peerConfiguration override val blockchain: Blockchain = self.blockchain override val appStateStorage: AppStateStorage = self.storagesInstance.storages.appStateStorage - override val protocolVersion: Capability = Capability.from(Config.Network.protocolVersion) + override val capabilities: List[Capability] = self.blockchainConfig.capabilities } lazy val handshaker: Handshaker[PeerInfo] = EtcHandshaker(handshakerConfiguration) @@ -413,7 +414,7 @@ trait EthInfoServiceBuilder { stxLedger, keyStore, syncController, - Capability.from(Config.Network.protocolVersion), + blockchainConfig.capabilities.head, asyncConfig.askTimeout ) } diff --git a/src/main/scala/io/iohk/ethereum/utils/BlockchainConfig.scala b/src/main/scala/io/iohk/ethereum/utils/BlockchainConfig.scala index c33fa0a6cc..ff7dd0674e 100644 --- a/src/main/scala/io/iohk/ethereum/utils/BlockchainConfig.scala +++ b/src/main/scala/io/iohk/ethereum/utils/BlockchainConfig.scala @@ -8,6 +8,7 @@ import io.iohk.ethereum.utils.NumericUtils._ import scala.jdk.CollectionConverters._ import scala.util.Try import com.typesafe.config.ConfigRenderOptions +import io.iohk.ethereum.network.p2p.messages.Capability case class BlockchainConfig( powTargetTime: Option[Long] = None, @@ -28,7 +29,8 @@ case class BlockchainConfig( ethCompatibleStorage: Boolean, bootstrapNodes: Set[String], checkpointPubKeys: Set[ByteString] = Set.empty, - allowedMinersPublicKeys: Set[ByteString] = Set.empty + allowedMinersPublicKeys: Set[ByteString] = Set.empty, + capabilities: List[Capability] = List.empty ) { val minRequireSignatures: Int = (Math.floor(checkpointPubKeys.size / 2) + 1).toInt @@ -124,6 +126,8 @@ object BlockchainConfig { val allowedMinersPublicKeys = readPubKeySet(blockchainConfig, "allowed-miners") val ecip1099BlockNumber: BigInt = BigInt(blockchainConfig.getString("ecip1099-block-number")) + val capabilities: List[Capability] = + blockchainConfig.getStringList("capabilities").asScala.toList.map(Capability.from) BlockchainConfig( powTargetTime = powTargetTime, @@ -163,7 +167,8 @@ object BlockchainConfig { ethCompatibleStorage = ethCompatibleStorage, bootstrapNodes = bootstrapNodes, checkpointPubKeys = checkpointPubKeys, - allowedMinersPublicKeys = allowedMinersPublicKeys + allowedMinersPublicKeys = allowedMinersPublicKeys, + capabilities = capabilities ) } // scalastyle:on method.length diff --git a/src/main/scala/io/iohk/ethereum/utils/Config.scala b/src/main/scala/io/iohk/ethereum/utils/Config.scala index 0861d7c5ce..69880b21f8 100644 --- a/src/main/scala/io/iohk/ethereum/utils/Config.scala +++ b/src/main/scala/io/iohk/ethereum/utils/Config.scala @@ -40,8 +40,6 @@ object Config { object Network { private val networkConfig = config.getConfig("network") - val protocolVersion = networkConfig.getString("protocol-version") - val automaticPortForwarding = networkConfig.getBoolean("automatic-port-forwarding") object Server { diff --git a/src/rpcTest/scala/io/iohk/ethereum/rpcTest/RpcApiTests.scala b/src/rpcTest/scala/io/iohk/ethereum/rpcTest/RpcApiTests.scala index 0ab833d7d5..0578c3f0dc 100644 --- a/src/rpcTest/scala/io/iohk/ethereum/rpcTest/RpcApiTests.scala +++ b/src/rpcTest/scala/io/iohk/ethereum/rpcTest/RpcApiTests.scala @@ -37,7 +37,7 @@ class RpcApiTests extends AnyFlatSpec with Matchers with Logger { (it should "return correct protocol version").taggedAs(MainNet, PrivNet) in new ScenarioSetup { val response = service.ethProtocolVersion().send() - hexToBigInt(response.getProtocolVersion) shouldEqual protocolVersion + hexToBigInt(response.getProtocolVersion) shouldEqual capabilities.head.version.toInt } (it should "return correct client version").taggedAs(MainNet, PrivNet) in new ScenarioSetup { @@ -1149,7 +1149,8 @@ abstract class ScenarioSetup { // Some data from mantis config (this data is not exposed to built version so it is safe to load it here val config = ConfigFactory.load("application.conf").getConfig("mantis") val clientVersion: String = io.iohk.ethereum.utils.Config.clientVersion - val protocolVersion = config.getConfig("network").getInt("protocol-version") + val networkName: String = io.iohk.ethereum.utils.Config.blockchains.network + val capabilities = io.iohk.ethereum.utils.Config.blockchains.blockchains(networkName).capabilities // val service = Admin.build(new HttpService(testConfig.mantisUrl)) diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/BlockBroadcastSpec.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/BlockBroadcastSpec.scala index 40db475098..41bf3937ca 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/BlockBroadcastSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/BlockBroadcastSpec.scala @@ -49,7 +49,7 @@ class BlockBroadcastSpec val blockHeader: BlockHeader = baseBlockHeader.copy(number = initialPeerInfo.maxBlockNumber - 3) val newBlockNewHashes = NewBlockHashes(Seq(PV62.BlockHash(blockHeader.hash, blockHeader.number))) val peerInfo = initialPeerInfo - .copy(remoteStatus = peerStatus.copy(protocolVersion = ProtocolVersions.PV63)) + .copy(remoteStatus = peerStatus.copy(protocolVersion = ProtocolVersions.PV63.version)) .withChainWeight(ChainWeight.totalDifficultyOnly(initialPeerInfo.chainWeight.totalDifficulty)) val newBlock = CommonMessages.NewBlock(Block(blockHeader, BlockBody(Nil, Nil)), peerInfo.chainWeight.totalDifficulty + 2) @@ -164,7 +164,7 @@ class BlockBroadcastSpec val baseBlockHeader = Fixtures.Blocks.Block3125369.header val peerStatus = RemoteStatus( - protocolVersion = ProtocolVersions.PV64, + protocolVersion = ProtocolVersions.PV64.version, networkId = 1, chainWeight = ChainWeight(10, 10000), bestHash = Fixtures.Blocks.Block3125369.header.hash, diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/PeersClientSpec.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/PeersClientSpec.scala index ff8a069b42..ded702ce40 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/PeersClientSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/PeersClientSpec.scala @@ -76,7 +76,7 @@ class PeersClientSpec extends AnyFlatSpec with Matchers with ScalaCheckPropertyC val peer3 = Peer(PeerId("peer3"), new InetSocketAddress("127.0.0.1", 3), TestProbe().ref, false) private val peerStatus = RemoteStatus( - protocolVersion = ProtocolVersions.PV63, + protocolVersion = ProtocolVersions.PV63.version, networkId = 1, chainWeight = ChainWeight(0, 0), bestHash = ByteString.empty, diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/PivotBlockSelectorSpec.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/PivotBlockSelectorSpec.scala index 462ce2b4be..3a9c6fd664 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/PivotBlockSelectorSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/PivotBlockSelectorSpec.scala @@ -571,7 +571,7 @@ class PivotBlockSelectorSpec val peer1Status = RemoteStatus( - ProtocolVersions.PV64, + ProtocolVersions.PV64.version, 1, ChainWeight.totalDifficultyOnly(20), ByteString("peer1_bestHash"), diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncSpec.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncSpec.scala index 056d85f1c7..61c6ad1e64 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncSpec.scala @@ -148,7 +148,7 @@ class StateSyncSpec val syncInit = TestProbe() val peerStatus = RemoteStatus( - protocolVersion = ProtocolVersions.PV63, + protocolVersion = ProtocolVersions.PV63.version, networkId = 1, chainWeight = ChainWeight.totalDifficultyOnly(10000), bestHash = Fixtures.Blocks.Block3125369.header.hash, diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/TestSyncPeers.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/TestSyncPeers.scala index 8297004181..0b6a1579bf 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/TestSyncPeers.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/TestSyncPeers.scala @@ -21,7 +21,7 @@ trait TestSyncPeers { self: TestSyncConfig => val peer1Status = RemoteStatus( - ProtocolVersions.PV64, + ProtocolVersions.PV64.version, 1, ChainWeight.totalDifficultyOnly(20), ByteString("peer1_bestHash"), diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/fast/FastSyncBranchResolverActorSpec.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/fast/FastSyncBranchResolverActorSpec.scala index 90416cb9a2..ab1618bcd6 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/fast/FastSyncBranchResolverActorSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/fast/FastSyncBranchResolverActorSpec.scala @@ -246,7 +246,7 @@ class FastSyncBranchResolverActorSpec def peerId(number: Int): PeerId = PeerId(s"peer_$number") def getPeer(id: PeerId): Peer = Peer(id, new InetSocketAddress("127.0.0.1", 0), TestProbe(id.value).ref, incomingConnection = false) - def getPeerInfo(peer: Peer, protocolVersion: Int = ProtocolVersions.PV64): PeerInfo = { + def getPeerInfo(peer: Peer, protocolVersion: Int = ProtocolVersions.PV64.version): PeerInfo = { val status = RemoteStatus( protocolVersion, diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncFixtures.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncFixtures.scala index 7b13f3c2a0..7673169ce2 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncFixtures.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncFixtures.scala @@ -102,7 +102,7 @@ trait RegularSyncFixtures { self: Matchers with AsyncMockFactory => def getPeer(id: PeerId): Peer = Peer(id, new InetSocketAddress("127.0.0.1", 0), TestProbe(id.value).ref, incomingConnection = false) - def getPeerInfo(peer: Peer, protocolVersion: Int = ProtocolVersions.PV64): PeerInfo = { + def getPeerInfo(peer: Peer, protocolVersion: Int = ProtocolVersions.PV64.version): PeerInfo = { val status = RemoteStatus( protocolVersion, diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncSpec.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncSpec.scala index e1f4f44e9b..1ae44a3ba5 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncSpec.scala @@ -681,7 +681,7 @@ class RegularSyncSpec val peerWithPV63: (Peer, PeerInfo) = { val id = peerId(handshakedPeers.size) val peer = getPeer(id) - val peerInfo = getPeerInfo(peer, ProtocolVersions.PV63) + val peerInfo = getPeerInfo(peer, ProtocolVersions.PV63.version) (peer, peerInfo) } diff --git a/src/test/scala/io/iohk/ethereum/jsonrpc/DebugServiceSpec.scala b/src/test/scala/io/iohk/ethereum/jsonrpc/DebugServiceSpec.scala index d8426367fb..0bfc263c4f 100644 --- a/src/test/scala/io/iohk/ethereum/jsonrpc/DebugServiceSpec.scala +++ b/src/test/scala/io/iohk/ethereum/jsonrpc/DebugServiceSpec.scala @@ -64,7 +64,7 @@ class DebugServiceSpec val debugService = new DebugService(peerManager.ref, etcPeerManager.ref) val peerStatus = RemoteStatus( - protocolVersion = ProtocolVersions.PV63, + protocolVersion = ProtocolVersions.PV63.version, networkId = 1, chainWeight = ChainWeight.totalDifficultyOnly(10000), bestHash = Fixtures.Blocks.Block3125369.header.hash, diff --git a/src/test/scala/io/iohk/ethereum/jsonrpc/EthInfoServiceSpec.scala b/src/test/scala/io/iohk/ethereum/jsonrpc/EthInfoServiceSpec.scala index df9ea5806a..a7e25ce9c5 100644 --- a/src/test/scala/io/iohk/ethereum/jsonrpc/EthInfoServiceSpec.scala +++ b/src/test/scala/io/iohk/ethereum/jsonrpc/EthInfoServiceSpec.scala @@ -14,6 +14,7 @@ import io.iohk.ethereum.jsonrpc.EthInfoService.{ProtocolVersionRequest, _} import io.iohk.ethereum.keystore.KeyStore import io.iohk.ethereum.ledger.Ledger.TxResult import io.iohk.ethereum.ledger.{Ledger, StxLedger} +import io.iohk.ethereum.network.p2p.messages.Capability import io.iohk.ethereum.testing.ActorsTesting.simpleAutoPilot import monix.execution.Scheduler.Implicits.global import org.bouncycastle.util.encoders.Hex @@ -160,7 +161,7 @@ class EthServiceSpec stxLedger, keyStore, syncingController.ref, - currentProtocolVersion, + Capability("eth", currentProtocolVersion.toByte), Timeouts.shortTimeout ) diff --git a/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerFixture.scala b/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerFixture.scala index 4621886a21..ebb4488af4 100644 --- a/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerFixture.scala +++ b/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerFixture.scala @@ -15,6 +15,7 @@ import io.iohk.ethereum.domain.{Block, BlockBody, SignedTransaction, UInt256} import io.iohk.ethereum.jsonrpc.server.controllers.JsonRpcBaseController.JsonRpcConfig import io.iohk.ethereum.keystore.KeyStore import io.iohk.ethereum.ledger.{BloomFilter, Ledger, StxLedger} +import io.iohk.ethereum.network.p2p.messages.Capability import io.iohk.ethereum.nodebuilder.ApisBuilder import io.iohk.ethereum.utils.{Config, FilterConfig} import io.iohk.ethereum.{Fixtures, ObjectGenerators, Timeouts} @@ -79,7 +80,7 @@ class JsonRpcControllerFixture(implicit system: ActorSystem) stxLedger, keyStore, syncingController.ref, - currentProtocolVersion, + Capability("eth", currentProtocolVersion.toByte), Timeouts.shortTimeout ) diff --git a/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerSpec.scala b/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerSpec.scala index d902355b95..1cbbe0696a 100644 --- a/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerSpec.scala @@ -117,7 +117,7 @@ class JsonRpcControllerSpec it should "debug_listPeersInfo" in new JsonRpcControllerFixture { val peerStatus = RemoteStatus( - protocolVersion = ProtocolVersions.PV63, + protocolVersion = ProtocolVersions.PV63.version, networkId = 1, chainWeight = ChainWeight.totalDifficultyOnly(10000), bestHash = Fixtures.Blocks.Block3125369.header.hash, diff --git a/src/test/scala/io/iohk/ethereum/network/EtcPeerManagerSpec.scala b/src/test/scala/io/iohk/ethereum/network/EtcPeerManagerSpec.scala index 621531712f..25f4ca5810 100644 --- a/src/test/scala/io/iohk/ethereum/network/EtcPeerManagerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/EtcPeerManagerSpec.scala @@ -285,7 +285,7 @@ class EtcPeerManagerSpec extends AnyFlatSpec with Matchers { val forkResolver = new ForkResolver.EtcForkResolver(blockchainConfig.daoForkConfig.get) val peerStatus = RemoteStatus( - protocolVersion = ProtocolVersions.PV63, + protocolVersion = ProtocolVersions.PV63.version, networkId = 1, chainWeight = ChainWeight.totalDifficultyOnly(10000), bestHash = Fixtures.Blocks.Block3125369.header.hash, @@ -301,7 +301,7 @@ class EtcPeerManagerSpec extends AnyFlatSpec with Matchers { ) val initialPeerInfoPV64 = PeerInfo( - remoteStatus = peerStatus.copy(protocolVersion = ProtocolVersions.PV64), + remoteStatus = peerStatus.copy(protocolVersion = ProtocolVersions.PV64.version), chainWeight = peerStatus.chainWeight, forkAccepted = false, maxBlockNumber = Fixtures.Blocks.Block3125369.header.number, diff --git a/src/test/scala/io/iohk/ethereum/network/PeerActorHandshakingSpec.scala b/src/test/scala/io/iohk/ethereum/network/PeerActorHandshakingSpec.scala index 1e23cd1400..b16898824a 100644 --- a/src/test/scala/io/iohk/ethereum/network/PeerActorHandshakingSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/PeerActorHandshakingSpec.scala @@ -169,7 +169,7 @@ class PeerActorHandshakingSpec extends AnyFlatSpec with Matchers { object DefaultValues { val defaultStatusMsg = Status( - protocolVersion = ProtocolVersions.PV63, + protocolVersion = ProtocolVersions.PV63.version, networkId = 1, totalDifficulty = Fixtures.Blocks.Genesis.header.difficulty, bestHash = Fixtures.Blocks.Genesis.header.hash, diff --git a/src/test/scala/io/iohk/ethereum/network/PeerEventBusActorSpec.scala b/src/test/scala/io/iohk/ethereum/network/PeerEventBusActorSpec.scala index 36f3ae6f9c..836dcd1bb0 100644 --- a/src/test/scala/io/iohk/ethereum/network/PeerEventBusActorSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/PeerEventBusActorSpec.scala @@ -243,7 +243,7 @@ class PeerEventBusActorSpec extends AnyFlatSpec with Matchers { val peerEventBusActor = system.actorOf(PeerEventBusActor.props) val peerStatus = RemoteStatus( - protocolVersion = ProtocolVersions.PV63, + protocolVersion = ProtocolVersions.PV63.version, networkId = 1, chainWeight = ChainWeight.totalDifficultyOnly(10000), bestHash = Fixtures.Blocks.Block3125369.header.hash, diff --git a/src/test/scala/io/iohk/ethereum/network/PeerManagerSpec.scala b/src/test/scala/io/iohk/ethereum/network/PeerManagerSpec.scala index c5dfaa90a8..7426f582b9 100644 --- a/src/test/scala/io/iohk/ethereum/network/PeerManagerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/PeerManagerSpec.scala @@ -606,7 +606,7 @@ class PeerManagerSpec val blacklist: CacheBasedBlacklist = CacheBasedBlacklist(cache) val peerStatus = RemoteStatus( - protocolVersion = ProtocolVersions.PV63, + protocolVersion = ProtocolVersions.PV63.version, networkId = 1, chainWeight = ChainWeight.totalDifficultyOnly(10000), bestHash = Fixtures.Blocks.Block3125369.header.hash, diff --git a/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala b/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala index a7f11a712c..5a92fbb876 100644 --- a/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala @@ -230,7 +230,7 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { } it should "fail if the remote peer doesn't support PV63/PV64" in new RemotePeerPV63Setup { - val pv62Capability = Capability("eth", ProtocolVersions.PV62.toByte) + val pv62Capability = ProtocolVersions.PV62 val handshakerAfterHelloOpt = initHandshakerWithResolver.applyMessage(remoteHello.copy(capabilities = Seq(pv62Capability))) assert(handshakerAfterHelloOpt.isDefined) @@ -269,13 +269,14 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { ) lazy val nodeStatusHolder = new AtomicReference(nodeStatus) - class MockEtcHandshakerConfiguration(pv: Int = Config.Network.protocolVersion) extends EtcHandshakerConfiguration { + class MockEtcHandshakerConfiguration(pv: List[Capability] = blockchainConfig.capabilities) + extends EtcHandshakerConfiguration { override val forkResolverOpt: Option[ForkResolver] = None override val nodeStatusHolder: AtomicReference[NodeStatus] = TestSetup.this.nodeStatusHolder override val peerConfiguration: PeerConfiguration = Config.Network.peer override val blockchain: Blockchain = TestSetup.this.blockchain override val appStateStorage: AppStateStorage = TestSetup.this.storagesInstance.storages.appStateStorage - override val protocolVersion: Int = pv + override val capabilities: List[Capability] = pv } val etcHandshakerConfigurationWithResolver = new MockEtcHandshakerConfiguration { @@ -284,7 +285,9 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { ) } - val initHandshakerWithoutResolver = EtcHandshaker(new MockEtcHandshakerConfiguration(ProtocolVersions.PV64)) + val initHandshakerWithoutResolver = EtcHandshaker( + new MockEtcHandshakerConfiguration(List(ProtocolVersions.PV64, ProtocolVersions.PV63)) + ) val initHandshakerWithResolver = EtcHandshaker(etcHandshakerConfigurationWithResolver) val firstBlock = @@ -306,7 +309,7 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { trait LocalPeerPV63Setup extends LocalPeerSetup { val localStatusMsg = CommonMessages.Status( - protocolVersion = ProtocolVersions.PV63, + protocolVersion = ProtocolVersions.PV63.version, networkId = Config.Network.peer.networkId, totalDifficulty = genesisBlock.header.difficulty, bestHash = genesisBlock.header.hash, @@ -317,7 +320,7 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { trait LocalPeerPV64Setup extends LocalPeerSetup { val localStatusMsg = PV64.Status( - protocolVersion = ProtocolVersions.PV64, + protocolVersion = ProtocolVersions.PV64.version, networkId = Config.Network.peer.networkId, chainWeight = ChainWeight.zero.increase(genesisBlock.header), bestHash = genesisBlock.header.hash, @@ -345,7 +348,7 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { ) val remoteStatusMsg = CommonMessages.Status( - protocolVersion = ProtocolVersions.PV63, + protocolVersion = ProtocolVersions.PV63.version, networkId = Config.Network.peer.networkId, totalDifficulty = 0, bestHash = genesisBlock.header.hash, @@ -366,7 +369,7 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { val remoteStatusMsg = PV64.Status( - protocolVersion = ProtocolVersions.PV64, + protocolVersion = ProtocolVersions.PV64.version, networkId = Config.Network.peer.networkId, chainWeight = ChainWeight.zero, bestHash = genesisBlock.header.hash, diff --git a/src/test/scala/io/iohk/ethereum/network/p2p/MessageCodecSpec.scala b/src/test/scala/io/iohk/ethereum/network/p2p/MessageCodecSpec.scala index 0b37a54e4d..03ef6ad53d 100644 --- a/src/test/scala/io/iohk/ethereum/network/p2p/MessageCodecSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/p2p/MessageCodecSpec.scala @@ -85,7 +85,7 @@ class MessageCodecSpec extends AnyFlatSpec with Matchers { val helloV4 = helloV5.copy(p2pVersion = 4) val status = Status( - protocolVersion = ProtocolVersions.PV63, + protocolVersion = ProtocolVersions.PV63.version, networkId = Config.Network.peer.networkId, totalDifficulty = 1, bestHash = ByteString(1), diff --git a/src/test/scala/io/iohk/ethereum/network/p2p/MessageDecodersSpec.scala b/src/test/scala/io/iohk/ethereum/network/p2p/MessageDecodersSpec.scala index c7b307ddfa..9356d78bec 100644 --- a/src/test/scala/io/iohk/ethereum/network/p2p/MessageDecodersSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/p2p/MessageDecodersSpec.scala @@ -181,9 +181,9 @@ class MessageDecodersSpec extends AnyFlatSpec with Matchers with SecureRandomBui } it should "decode Status message for all supported versions of protocol" in { - val status63 = CommonMessages.Status(ProtocolVersions.PV63, 1, BigInt(100), exampleHash, exampleHash) + val status63 = CommonMessages.Status(ProtocolVersions.PV63.version, 1, BigInt(100), exampleHash, exampleHash) val status63Bytes: Array[Byte] = status63.toBytes - val status64 = PV64.Status(ProtocolVersions.PV63, 1, ChainWeight(1, BigInt(100)), exampleHash, exampleHash) + val status64 = PV64.Status(ProtocolVersions.PV63.version, 1, ChainWeight(1, BigInt(100)), exampleHash, exampleHash) // it's not 100 % true as Status message was different in PV61, but we are not supporting old message decode(Codes.StatusCode, status63Bytes, ProtocolVersions.PV61) shouldBe status63 @@ -221,7 +221,7 @@ class MessageDecodersSpec extends AnyFlatSpec with Matchers with SecureRandomBui it should "not decode message of not supported protocol" in { assertThrows[RuntimeException] { - decode(Codes.NewBlockHashesCode, NewBlockHashesPV61bytes, ProtocolVersions.PV61 - 1) + decode(Codes.NewBlockHashesCode, NewBlockHashesPV61bytes, ProtocolVersions.PV61.copy(version = 60)) } } } diff --git a/src/test/scala/io/iohk/ethereum/network/p2p/PeerActorSpec.scala b/src/test/scala/io/iohk/ethereum/network/p2p/PeerActorSpec.scala index 1162d0024e..8f369ce984 100644 --- a/src/test/scala/io/iohk/ethereum/network/p2p/PeerActorSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/p2p/PeerActorSpec.scala @@ -3,7 +3,6 @@ package io.iohk.ethereum.network.p2p import java.net.{InetSocketAddress, URI} import java.security.SecureRandom import java.util.concurrent.atomic.AtomicReference - import akka.actor.{ActorSystem, PoisonPill, Props, Terminated} import akka.testkit.{TestActorRef, TestKit, TestProbe} import akka.util.ByteString @@ -18,13 +17,12 @@ import io.iohk.ethereum.network.PeerActor.Status.Handshaked import io.iohk.ethereum.network.PeerActor.{GetStatus, StatusResponse} import io.iohk.ethereum.network.PeerManagerActor.{FastSyncHostConfiguration, PeerConfiguration} import io.iohk.ethereum.network.handshaker.{EtcHandshaker, EtcHandshakerConfiguration} -import io.iohk.ethereum.network.p2p.Message.Version import io.iohk.ethereum.network.p2p.messages.Capability.Capabilities._ import io.iohk.ethereum.network.p2p.messages.CommonMessages.Status import io.iohk.ethereum.network.p2p.messages.CommonMessages.Status.StatusEnc import io.iohk.ethereum.network.p2p.messages.PV62.GetBlockHeaders.GetBlockHeadersEnc import io.iohk.ethereum.network.p2p.messages.PV62._ -import io.iohk.ethereum.network.p2p.messages.{PV64, ProtocolVersions} +import io.iohk.ethereum.network.p2p.messages.{Capability, PV64, ProtocolVersions} import io.iohk.ethereum.network.p2p.messages.WireProtocol.Disconnect.{DisconnectEnc, Reasons} import io.iohk.ethereum.network.p2p.messages.WireProtocol.Hello.HelloEnc import io.iohk.ethereum.network.p2p.messages.WireProtocol.Pong.PongEnc @@ -81,7 +79,7 @@ class PeerActorSpec it should "try to reconnect on broken rlpx connection" in new NodeStatusSetup with HandshakerSetup { override implicit lazy val system = ActorSystem("PeerActorSpec_System") - override def protocol: Version = ProtocolVersions.PV63 + override def protocol: Capability = ProtocolVersions.PV63 val time = new VirtualTime @@ -137,7 +135,7 @@ class PeerActorSpec rlpxConnection.send(peer, RLPxConnectionHandler.MessageReceived(remoteHello)) val remoteStatus = Status( - protocolVersion = ProtocolVersions.PV63, + protocolVersion = ProtocolVersions.PV63.version, networkId = peerConf.networkId, totalDifficulty = daoForkBlockChainTotalDifficulty + 100000, // remote is after the fork bestHash = ByteString("blockhash"), @@ -175,7 +173,7 @@ class PeerActorSpec rlpxConnection.send(peer, RLPxConnectionHandler.MessageReceived(remoteHello)) val remoteStatus = Status( - protocolVersion = ProtocolVersions.PV63, + protocolVersion = ProtocolVersions.PV63.version, networkId = peerConf.networkId, totalDifficulty = daoForkBlockChainTotalDifficulty + 100000, // remote is after the fork bestHash = ByteString("blockhash"), @@ -190,7 +188,7 @@ class PeerActorSpec } it should "successfully connect to ETC peer with protocol 64" in new TestSetup { - override def protocol: Version = ProtocolVersions.PV64 + override def protocol: Capability = ProtocolVersions.PV64 val uri = new URI(s"enode://${Hex.toHexString(remoteNodeId.toArray[Byte])}@localhost:9000") val completeUri = new URI(s"enode://${Hex.toHexString(remoteNodeId.toArray[Byte])}@127.0.0.1:9000?discport=9000") peer ! PeerActor.ConnectTo(uri) @@ -205,7 +203,7 @@ class PeerActorSpec rlpxConnection.send(peer, RLPxConnectionHandler.MessageReceived(remoteHello)) val remoteStatus = PV64.Status( - protocolVersion = ProtocolVersions.PV64, + protocolVersion = ProtocolVersions.PV64.version, networkId = peerConf.networkId, chainWeight = ChainWeight.totalDifficultyOnly(daoForkBlockChainTotalDifficulty + 100000), // remote is after the fork @@ -244,7 +242,7 @@ class PeerActorSpec rlpxConnection.send(peer, RLPxConnectionHandler.MessageReceived(remoteHello)) val remoteStatus = Status( - protocolVersion = ProtocolVersions.PV63, + protocolVersion = ProtocolVersions.PV63.version, networkId = peerConf.networkId, totalDifficulty = daoForkBlockChainTotalDifficulty + 100000, // remote is after the fork bestHash = ByteString("blockhash"), @@ -287,7 +285,7 @@ class PeerActorSpec .commit() val remoteStatus = Status( - protocolVersion = ProtocolVersions.PV63, + protocolVersion = ProtocolVersions.PV63.version, networkId = peerConf.networkId, totalDifficulty = daoForkBlockChainTotalDifficulty + 100000, // remote is after the fork bestHash = ByteString("blockhash"), @@ -313,7 +311,7 @@ class PeerActorSpec rlpxConnection.send(peer, RLPxConnectionHandler.MessageReceived(remoteHello)) val remoteStatus = Status( - protocolVersion = ProtocolVersions.PV63, + protocolVersion = ProtocolVersions.PV63.version, networkId = peerConf.networkId, totalDifficulty = daoForkBlockChainTotalDifficulty + 100000, // remote is after the fork bestHash = ByteString("blockhash"), @@ -360,7 +358,7 @@ class PeerActorSpec rlpxConnection.send(peer, RLPxConnectionHandler.MessageReceived(remoteHello)) val remoteStatus = Status( - protocolVersion = ProtocolVersions.PV63, + protocolVersion = ProtocolVersions.PV63.version, networkId = peerConf.networkId, totalDifficulty = daoForkBlockChainTotalDifficulty + 100000, // remote is after the fork bestHash = ByteString("blockhash"), @@ -392,7 +390,7 @@ class PeerActorSpec rlpxConnection.send(peer, RLPxConnectionHandler.MessageReceived(remoteHello)) val remoteStatus = Status( - protocolVersion = ProtocolVersions.PV63, + protocolVersion = ProtocolVersions.PV63.version, networkId = peerConf.networkId, totalDifficulty = daoForkBlockChainTotalDifficulty + 100000, // remote is after the fork bestHash = ByteString("blockhash"), @@ -411,7 +409,7 @@ class PeerActorSpec it should "stay connected to pre fork peer" in new TestSetup { val remoteStatus = RemoteStatus( - protocolVersion = ProtocolVersions.PV63, + protocolVersion = ProtocolVersions.PV63.version, networkId = peerConf.networkId, chainWeight = ChainWeight.totalDifficultyOnly(daoForkBlockChainTotalDifficulty - 200000), // remote is before the fork @@ -454,7 +452,7 @@ class PeerActorSpec rlpxConnection.send(peer, RLPxConnectionHandler.MessageReceived(remoteHello)) val remoteStatus = Status( - protocolVersion = ProtocolVersions.PV63, + protocolVersion = ProtocolVersions.PV63.version, networkId = peerConf.networkId, totalDifficulty = daoForkBlockChainTotalDifficulty + 100000, // remote is after the fork bestHash = ByteString("blockhash"), @@ -556,7 +554,7 @@ class PeerActorSpec } trait HandshakerSetup extends NodeStatusSetup { self => - def protocol: Version + def protocol: Capability val handshakerConfiguration = new EtcHandshakerConfiguration { override val forkResolverOpt: Option[ForkResolver] = Some( @@ -566,14 +564,14 @@ class PeerActorSpec override val peerConfiguration: PeerConfiguration = self.peerConf override val blockchain: Blockchain = self.blockchain override val appStateStorage: AppStateStorage = self.storagesInstance.storages.appStateStorage - override val protocolVersion: Int = protocol + override val capabilities: List[Capability] = List(protocol) } val handshaker = EtcHandshaker(handshakerConfiguration) } trait TestSetup extends NodeStatusSetup with BlockUtils with HandshakerSetup { - override def protocol: Version = ProtocolVersions.PV63 + override def protocol: Capability = ProtocolVersions.PV63 val genesisHash = genesisBlock.hash diff --git a/src/test/scala/io/iohk/ethereum/network/p2p/messages/MessagesSerializationSpec.scala b/src/test/scala/io/iohk/ethereum/network/p2p/messages/MessagesSerializationSpec.scala index 0e838e4832..088e635a1f 100644 --- a/src/test/scala/io/iohk/ethereum/network/p2p/messages/MessagesSerializationSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/p2p/messages/MessagesSerializationSpec.scala @@ -158,7 +158,7 @@ class MessagesSerializationSpec extends AnyWordSpec with ScalaCheckPropertyCheck val messageDecoder = NetworkMessageDecoder orElse EthereumMessageDecoder - def verify[T](msg: T, encode: T => Array[Byte], code: Int, version: Int): Unit = + def verify[T](msg: T, encode: T => Array[Byte], code: Int, version: Capability): Unit = messageDecoder.fromBytes(code, encode(msg), version) shouldEqual msg } diff --git a/src/test/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandlerSpec.scala b/src/test/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandlerSpec.scala index 848e4f30c6..b44934944d 100644 --- a/src/test/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandlerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandlerSpec.scala @@ -1,15 +1,13 @@ package io.iohk.ethereum.network.rlpx import java.net.{InetSocketAddress, URI} - import akka.actor.{ActorRef, ActorSystem, Props} import akka.io.Tcp import akka.testkit.{TestActorRef, TestKit, TestProbe} import akka.util.ByteString import io.iohk.ethereum.{Timeouts, WithActorSystemShutDown} -import io.iohk.ethereum.network.p2p.Message.Version import io.iohk.ethereum.network.p2p.{MessageDecoder, MessageSerializable} -import io.iohk.ethereum.network.p2p.messages.ProtocolVersions +import io.iohk.ethereum.network.p2p.messages.{Capability, ProtocolVersions} import io.iohk.ethereum.network.p2p.messages.WireProtocol.Ping import io.iohk.ethereum.network.rlpx.RLPxConnectionHandler.RLPxConfiguration import io.iohk.ethereum.security.SecureRandomBuilder @@ -174,7 +172,7 @@ class RLPxConnectionHandlerSpec //Mock parameters for RLPxConnectionHandler val mockMessageDecoder = new MessageDecoder { - override def fromBytes(`type`: Int, payload: Array[Byte], protocolVersion: Version) = + override def fromBytes(`type`: Int, payload: Array[Byte], protocolVersion: Capability) = throw new Exception("Mock message decoder fails to decode all messages") } val protocolVersion = ProtocolVersions.PV63 From efd65565561685afafe2f0e0da33967a8c46f8e8 Mon Sep 17 00:00:00 2001 From: Dominik Zajkowski Date: Tue, 1 Jun 2021 16:39:09 +0200 Subject: [PATCH 03/15] [ETCM-841] Introduce a simple Capability.best selection algorithm --- src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala | 2 +- .../scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala | 2 +- .../io/iohk/ethereum/network/p2p/messages/Capability.scala | 4 ++++ src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala | 2 +- 4 files changed, 7 insertions(+), 3 deletions(-) diff --git a/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala b/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala index 4fb1d0520b..a1df0f2d3d 100644 --- a/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala +++ b/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala @@ -193,7 +193,7 @@ abstract class CommonFakePeer(peerName: String, fakePeerCustomConfig: FakePeerCu EthereumMessageDecoder, discoveryConfig, blacklist, - blockchainConfig.capabilities.head + Capability.best(blockchainConfig.capabilities) ), "peer-manager" ) diff --git a/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala b/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala index 59b8359192..d4a1ee679f 100644 --- a/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala +++ b/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala @@ -116,7 +116,7 @@ object DumpChainApp extends App with NodeKeyBuilder with SecureRandomBuilder wit messageDecoder = EthereumMessageDecoder, discoveryConfig = discoveryConfig, blacklist = blacklist, - bestProtocolVersion = blockchainConfig.capabilities.head + bestProtocolVersion = Capability.best(blockchainConfig.capabilities) ), "peer-manager" ) diff --git a/src/main/scala/io/iohk/ethereum/network/p2p/messages/Capability.scala b/src/main/scala/io/iohk/ethereum/network/p2p/messages/Capability.scala index 8e38ca974a..dc9e62cd5e 100644 --- a/src/main/scala/io/iohk/ethereum/network/p2p/messages/Capability.scala +++ b/src/main/scala/io/iohk/ethereum/network/p2p/messages/Capability.scala @@ -15,6 +15,10 @@ object Capability { case _ => throw new RuntimeException(s"Unable to parse capability $protocolVersion") } + //TODO consider how this scoring should be handled with snap and other extended protocols + def best(capabilities: List[Capability]): Capability = + capabilities.maxBy(_.version) + implicit class CapabilityEnc(val msg: Capability) extends RLPSerializable { override def toRLPEncodable: RLPEncodeable = RLPList(msg.name, msg.version) } diff --git a/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala b/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala index 0e8d3e6461..c342d7484f 100644 --- a/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala +++ b/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala @@ -414,7 +414,7 @@ trait EthInfoServiceBuilder { stxLedger, keyStore, syncController, - blockchainConfig.capabilities.head, + Capability.best(blockchainConfig.capabilities), asyncConfig.askTimeout ) } From 98ee816ac6d8d2c31266e9912c40e49b6fd4627d Mon Sep 17 00:00:00 2001 From: Dominik Zajkowski Date: Wed, 2 Jun 2021 16:27:46 +0200 Subject: [PATCH 04/15] [ETCM-841] Add simple protocol negotiation --- .../io/iohk/ethereum/network/PeerActor.scala | 8 +-- .../ethereum/network/PeerManagerActor.scala | 8 +-- .../network/p2p/messages/Capability.scala | 4 ++ .../ethereum/network/rlpx/MessageCodec.scala | 30 ++------ .../network/rlpx/RLPxConnectionHandler.scala | 72 +++++++++++++++---- .../ethereum/nodebuilder/NodeBuilder.scala | 2 +- 6 files changed, 77 insertions(+), 47 deletions(-) diff --git a/src/main/scala/io/iohk/ethereum/network/PeerActor.scala b/src/main/scala/io/iohk/ethereum/network/PeerActor.scala index 644afca8fa..f81eb6e9b1 100644 --- a/src/main/scala/io/iohk/ethereum/network/PeerActor.scala +++ b/src/main/scala/io/iohk/ethereum/network/PeerActor.scala @@ -302,12 +302,12 @@ object PeerActor { handshaker: Handshaker[R], authHandshaker: AuthHandshaker, messageDecoder: MessageDecoder, - bestProtocolVersion: Capability + capabilities: List[Capability] ): Props = Props( new PeerActor( peerAddress, - rlpxConnectionFactory(authHandshaker, messageDecoder, peerConfiguration.rlpxConfiguration, bestProtocolVersion), + rlpxConnectionFactory(authHandshaker, messageDecoder, peerConfiguration.rlpxConfiguration, capabilities), peerConfiguration, peerEventBus, knownNodesManager, @@ -321,11 +321,11 @@ object PeerActor { authHandshaker: AuthHandshaker, messageDecoder: MessageDecoder, rlpxConfiguration: RLPxConfiguration, - bestProtocolVersion: Capability + capabilities: List[Capability] ): ActorContext => ActorRef = { ctx => ctx.actorOf( RLPxConnectionHandler - .props(NetworkMessageDecoder orElse messageDecoder, bestProtocolVersion, authHandshaker, rlpxConfiguration), + .props(NetworkMessageDecoder orElse messageDecoder, capabilities, authHandshaker, rlpxConfiguration), "rlpx-connection" ) } diff --git a/src/main/scala/io/iohk/ethereum/network/PeerManagerActor.scala b/src/main/scala/io/iohk/ethereum/network/PeerManagerActor.scala index 86b230a38d..250a7f3f3c 100644 --- a/src/main/scala/io/iohk/ethereum/network/PeerManagerActor.scala +++ b/src/main/scala/io/iohk/ethereum/network/PeerManagerActor.scala @@ -424,7 +424,7 @@ object PeerManagerActor { messageDecoder: MessageDecoder, discoveryConfig: DiscoveryConfig, blacklist: Blacklist, - bestProtocolVersion: Capability + capabilities: List[Capability] ): Props = { val factory: (ActorContext, InetSocketAddress, Boolean) => ActorRef = peerFactory( @@ -434,7 +434,7 @@ object PeerManagerActor { handshaker, authHandshaker, messageDecoder, - bestProtocolVersion + capabilities ) Props( @@ -459,7 +459,7 @@ object PeerManagerActor { handshaker: Handshaker[R], authHandshaker: AuthHandshaker, messageDecoder: MessageDecoder, - bestProtocolVersion: Capability + capabilities: List[Capability] ): (ActorContext, InetSocketAddress, Boolean) => ActorRef = { (ctx, address, incomingConnection) => val id: String = address.toString.filterNot(_ == '/') val props = PeerActor.props( @@ -471,7 +471,7 @@ object PeerManagerActor { handshaker, authHandshaker, messageDecoder, - bestProtocolVersion + capabilities ) ctx.actorOf(props, id) } diff --git a/src/main/scala/io/iohk/ethereum/network/p2p/messages/Capability.scala b/src/main/scala/io/iohk/ethereum/network/p2p/messages/Capability.scala index dc9e62cd5e..76ec2894fb 100644 --- a/src/main/scala/io/iohk/ethereum/network/p2p/messages/Capability.scala +++ b/src/main/scala/io/iohk/ethereum/network/p2p/messages/Capability.scala @@ -7,6 +7,10 @@ import io.iohk.ethereum.rlp.{RLPEncodeable, RLPException, RLPList, RLPSerializab case class Capability(name: String, version: Byte) object Capability { + def negotiate(c1: List[Capability], c2: List[Capability]): Option[Capability] = { + c1.intersect(c2).maxByOption(_.version) // FIXME ignores branches and other protocols + } + private val pattern = "(.*)/(\\d*)".r def from(protocolVersion: String): Capability = diff --git a/src/main/scala/io/iohk/ethereum/network/rlpx/MessageCodec.scala b/src/main/scala/io/iohk/ethereum/network/rlpx/MessageCodec.scala index bf81fdee56..f20528826c 100644 --- a/src/main/scala/io/iohk/ethereum/network/rlpx/MessageCodec.scala +++ b/src/main/scala/io/iohk/ethereum/network/rlpx/MessageCodec.scala @@ -10,7 +10,7 @@ import org.xerial.snappy.Snappy import scala.util.{Failure, Success, Try} -class MessageCodec(frameCodec: FrameCodec, messageDecoder: MessageDecoder, protocolVersion: Capability) { +class MessageCodec(frameCodec: FrameCodec, messageDecoder: MessageDecoder, protocolVersion: Capability, val remotePeer2PeerVersion: Long) { val MaxFramePayloadSize: Int = Int.MaxValue // no framing @@ -19,37 +19,24 @@ class MessageCodec(frameCodec: FrameCodec, messageDecoder: MessageDecoder, proto // 16Mb in base 2 val maxDecompressedLength = 16777216 - // MessageCodec is only used from actor context so it can be var - @volatile - private var remotePeerP2pVersion: Option[Long] = None - - private def setRemoteVersionBasedOnHelloMessage(m: Message): Unit = { - if (remotePeerP2pVersion.isEmpty) { - m match { - case hello: Hello => - remotePeerP2pVersion = Some(hello.p2pVersion) - case _ => - } - } - } - // TODO: ETCM-402 - messageDecoder should use negotiated protocol version def readMessages(data: ByteString): Seq[Try[Message]] = { val frames = frameCodec.readFrames(data) + readFrames(frames) + } + def readFrames(frames: Seq[Frame]): Seq[Try[Message]] = { frames map { frame => val frameData = frame.payload.toArray val payloadTry = - if (remotePeerP2pVersion.exists(version => version >= EtcHelloExchangeState.P2pVersion)) { + if (remotePeer2PeerVersion >= EtcHelloExchangeState.P2pVersion) { decompressData(frameData) } else { Success(frameData) } payloadTry.map { payload => - val m = messageDecoder.fromBytes(frame.`type`, payload, protocolVersion) - setRemoteVersionBasedOnHelloMessage(m) - m + messageDecoder.fromBytes(frame.`type`, payload, protocolVersion) } } } @@ -70,10 +57,7 @@ class MessageCodec(frameCodec: FrameCodec, messageDecoder: MessageDecoder, proto val frames = (0 until numFrames) map { frameNo => val framedPayload = encoded.drop(frameNo * MaxFramePayloadSize).take(MaxFramePayloadSize) val payload = - if ( - remotePeerP2pVersion - .exists(version => version >= EtcHelloExchangeState.P2pVersion) && serializable.code != Hello.code - ) { + if (remotePeer2PeerVersion >= EtcHelloExchangeState.P2pVersion && serializable.code != Hello.code) { Snappy.compress(framedPayload) } else { framedPayload diff --git a/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala b/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala index 78a7b166b6..61a9b79a70 100644 --- a/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala +++ b/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala @@ -5,8 +5,10 @@ import akka.actor._ import akka.io.Tcp._ import akka.io.{IO, Tcp} import akka.util.ByteString +import cats.data.NonEmptyList import io.iohk.ethereum.network.p2p.messages.Capability -import io.iohk.ethereum.network.p2p.{Message, MessageDecoder, MessageSerializable} +import io.iohk.ethereum.network.p2p.messages.WireProtocol.Hello +import io.iohk.ethereum.network.p2p.{Message, MessageDecoder, MessageSerializable, NetworkMessageDecoder} import io.iohk.ethereum.network.rlpx.RLPxConnectionHandler.RLPxConfiguration import io.iohk.ethereum.utils.ByteUtils import org.bouncycastle.util.encoders.Hex @@ -28,9 +30,9 @@ import scala.util.{Failure, Success, Try} */ class RLPxConnectionHandler( messageDecoder: MessageDecoder, - protocolVersion: Capability, + capabilities: List[Capability], authHandshaker: AuthHandshaker, - messageCodecFactory: (Secrets, MessageDecoder, Capability) => MessageCodec, + messageCodecFactory: (FrameCodec, MessageDecoder, Capability, Long) => MessageCodec, rlpxConfiguration: RLPxConfiguration ) extends Actor with ActorLogging { @@ -76,6 +78,7 @@ class RLPxConnectionHandler( def waitingForAuthHandshakeInit(handshaker: AuthHandshaker, timeout: Cancellable): Receive = handleTimeout orElse handleConnectionClosed orElse { case Received(data) => timeout.cancel() + // FIXME EIP8 is 6 years old, time to drop it val maybePreEIP8Result = Try { val (responsePacket, result) = handshaker.handleInitialMessage(data.take(InitiatePacketLength)) val remainingData = data.drop(InitiatePacketLength) @@ -115,7 +118,9 @@ class RLPxConnectionHandler( (result, remainingData) } maybePreEIP8Result orElse maybePostEIP8Result match { - case Success((result, remainingData)) => processHandshakeResult(result, remainingData) + case Success((result, remainingData)) => + processHandshakeResult(result, remainingData) + case Failure(ex) => log.debug( s"[Stopping Connection] Response AuthHandshaker message handling failed for peer $peerId due to ${ex.getMessage}" @@ -148,10 +153,34 @@ class RLPxConnectionHandler( case AuthHandshakeSuccess(secrets, remotePubKey) => log.info(s"Auth handshake succeeded for peer $peerId") context.parent ! ConnectionEstablished(remotePubKey) - val messageCodec = messageCodecFactory(secrets, messageDecoder, protocolVersion) - val messagesSoFar = messageCodec.readMessages(remainingData) - messagesSoFar foreach processMessage - context become handshaked(messageCodec) + //expect Hello + val frameCodec = new FrameCodec(secrets) + val frames = frameCodec.readFrames(remainingData) + frames.headOption.flatMap(extractHello) match { + case Some(h) => + context.parent ! MessageReceived(h) + val protocolVersion = Capability.negotiate(h.capabilities.toList, capabilities) + val p2pVersion = h.p2pVersion + protocolVersion match { + case Some(value) => + val messageCodec = messageCodecFactory(frameCodec, messageDecoder, value, p2pVersion) + val restFrames = frames.drop(1) + if (restFrames.nonEmpty) { + val messagesSoFar = messageCodec.readFrames(restFrames) // omit hello + messagesSoFar foreach processMessage + } + context become handshaked(messageCodec) + case None => + log.debug(s"[Stopping Connection] Unable to connect to $peerId, no common capability found ${h.capabilities} and $capabilities") + context.parent ! ConnectionFailed + context stop self + } + + case None => + log.debug(s"[Stopping Connection] Unable to connect to $peerId, 'Hello' not found") + context.parent ! ConnectionFailed + context stop self + } case AuthHandshakeError => log.debug(s"[Stopping Connection] Auth handshake failed for peer $peerId") @@ -159,6 +188,17 @@ class RLPxConnectionHandler( context stop self } + private def extractHello(frame: Frame): Option[Hello] = { + val frameData = frame.payload.toArray + if(frame.`type` == Hello.code) { + val m = NetworkMessageDecoder.fromBytes(frame.`type`, frameData, Capability.Capabilities.Eth63Capability) + Some(m.asInstanceOf[Hello]) + } else { + log.error("Unable to find 'Hello'") + None + } + } + def processMessage(messageTry: Try[Message]): Unit = messageTry match { case Success(message) => context.parent ! MessageReceived(message) @@ -183,7 +223,7 @@ class RLPxConnectionHandler( messagesNotSent: Queue[MessageSerializable] = Queue.empty, cancellableAckTimeout: Option[CancellableAckTimeout] = None, seqNumber: Int = 0 - ): Receive = + ): Receive = { handleWriteFailed orElse handleConnectionClosed orElse { case sm: SendMessage => if (cancellableAckTimeout.isEmpty) @@ -215,6 +255,7 @@ class RLPxConnectionHandler( log.debug(s"[Stopping Connection] Write to $peerId failed") context stop self } + } /** * Sends an encoded message through the TCP connection, an Ack will be received when the message was @@ -233,7 +274,7 @@ class RLPxConnectionHandler( ): Unit = { val out = messageCodec.encodeMessage(messageToSend) connection ! Write(out, Ack) - log.debug(s"Sent message: $messageToSend from $peerId") + log.debug(s"Sent message: ${messageToSend.underlyingMsg.toShortString} to $peerId") val timeout = system.scheduler.scheduleOnce(rlpxConfiguration.waitForTcpAckTimeout, self, AckTimeout(seqNumber)) context become handshaked( @@ -279,20 +320,21 @@ class RLPxConnectionHandler( object RLPxConnectionHandler { def props( messageDecoder: MessageDecoder, - protocolVersion: Capability, + capabilities: List[Capability], authHandshaker: AuthHandshaker, rlpxConfiguration: RLPxConfiguration ): Props = Props( - new RLPxConnectionHandler(messageDecoder, protocolVersion, authHandshaker, messageCodecFactory, rlpxConfiguration) + new RLPxConnectionHandler(messageDecoder, capabilities, authHandshaker, messageCodecFactory, rlpxConfiguration) ) def messageCodecFactory( - secrets: Secrets, + frameCodec: FrameCodec, messageDecoder: MessageDecoder, - protocolVersion: Capability + protocolVersion: Capability, + p2pVersion: Long ): MessageCodec = - new MessageCodec(new FrameCodec(secrets), messageDecoder, protocolVersion) + new MessageCodec(frameCodec, messageDecoder, protocolVersion, p2pVersion) case class ConnectTo(uri: URI) diff --git a/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala b/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala index c342d7484f..785be10f71 100644 --- a/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala +++ b/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala @@ -248,7 +248,7 @@ trait PeerManagerActorBuilder { EthereumMessageDecoder, discoveryConfig, blacklist, - Capability.Capabilities.Eth63Capability // TODO replace with a list of capabilities + blockchainConfig.capabilities // TODO replace with a list of capabilities ), "peer-manager" ) From ff3aa2ff13fe648a4276c5ac655f7d4c0ef8550d Mon Sep 17 00:00:00 2001 From: Dominik Zajkowski Date: Wed, 2 Jun 2021 19:33:56 +0200 Subject: [PATCH 05/15] [ETCM-841] Introduce negotiated protocol version to EtcHelloExchangeState --- .../io/iohk/ethereum/network/PeerActor.scala | 10 ++++---- .../ethereum/network/PeerManagerActor.scala | 4 ++-- .../network/handshaker/EtcHandshaker.scala | 4 ++-- .../handshaker/EtcHelloExchangeState.scala | 23 +++++++------------ .../network/rlpx/RLPxConnectionHandler.scala | 7 +++--- .../ethereum/nodebuilder/NodeBuilder.scala | 3 ++- 6 files changed, 23 insertions(+), 28 deletions(-) diff --git a/src/main/scala/io/iohk/ethereum/network/PeerActor.scala b/src/main/scala/io/iohk/ethereum/network/PeerActor.scala index f81eb6e9b1..2f32e1ff02 100644 --- a/src/main/scala/io/iohk/ethereum/network/PeerActor.scala +++ b/src/main/scala/io/iohk/ethereum/network/PeerActor.scala @@ -34,7 +34,7 @@ class PeerActor[R <: HandshakeResult]( knownNodesManager: ActorRef, incomingConnection: Boolean, externalSchedulerOpt: Option[Scheduler] = None, - initHandshaker: Handshaker[R] + initHandshaker: Capability => Handshaker[R] ) extends Actor with ActorLogging with Stash { @@ -80,10 +80,10 @@ class PeerActor[R <: HandshakeResult]( def waitingForConnectionResult(rlpxConnection: RLPxConnection, numRetries: Int = 0): Receive = handleTerminated(rlpxConnection, numRetries, Connecting) orElse stashMessages orElse { - case RLPxConnectionHandler.ConnectionEstablished(remoteNodeId) => + case RLPxConnectionHandler.ConnectionEstablished(remoteNodeId, capability) => val newUri = rlpxConnection.uriOpt.map(outGoingUri => modifyOutGoingUri(remoteNodeId, rlpxConnection, outGoingUri)) - processHandshakerNextMessage(initHandshaker, remoteNodeId, rlpxConnection.copy(uriOpt = newUri), numRetries) + processHandshakerNextMessage(initHandshaker(capability), remoteNodeId, rlpxConnection.copy(uriOpt = newUri), numRetries) case RLPxConnectionHandler.ConnectionFailed => log.debug("Failed to establish RLPx connection") @@ -103,7 +103,7 @@ class PeerActor[R <: HandshakeResult]( case GetStatus => sender() ! StatusResponse(Connecting) } - def processingHandshaking( + private def processingHandshaking( handshaker: Handshaker[R], remoteNodeId: ByteString, rlpxConnection: RLPxConnection, @@ -299,7 +299,7 @@ object PeerActor { peerEventBus: ActorRef, knownNodesManager: ActorRef, incomingConnection: Boolean, - handshaker: Handshaker[R], + handshaker: Capability => Handshaker[R], authHandshaker: AuthHandshaker, messageDecoder: MessageDecoder, capabilities: List[Capability] diff --git a/src/main/scala/io/iohk/ethereum/network/PeerManagerActor.scala b/src/main/scala/io/iohk/ethereum/network/PeerManagerActor.scala index 250a7f3f3c..11941124e4 100644 --- a/src/main/scala/io/iohk/ethereum/network/PeerManagerActor.scala +++ b/src/main/scala/io/iohk/ethereum/network/PeerManagerActor.scala @@ -419,7 +419,7 @@ object PeerManagerActor { peerMessageBus: ActorRef, knownNodesManager: ActorRef, peerStatistics: ActorRef, - handshaker: Handshaker[R], + handshaker: Capability => Handshaker[R], authHandshaker: AuthHandshaker, messageDecoder: MessageDecoder, discoveryConfig: DiscoveryConfig, @@ -456,7 +456,7 @@ object PeerManagerActor { config: PeerConfiguration, eventBus: ActorRef, knownNodesManager: ActorRef, - handshaker: Handshaker[R], + handshaker: Capability => Handshaker[R], authHandshaker: AuthHandshaker, messageDecoder: MessageDecoder, capabilities: List[Capability] diff --git a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHandshaker.scala b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHandshaker.scala index 0da7c37db0..742da29bf5 100644 --- a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHandshaker.scala +++ b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHandshaker.scala @@ -22,8 +22,8 @@ case class EtcHandshaker private ( object EtcHandshaker { - def apply(handshakerConfiguration: EtcHandshakerConfiguration): EtcHandshaker = { - val initialState = EtcHelloExchangeState(handshakerConfiguration) + def apply(handshakerConfiguration: EtcHandshakerConfiguration, capability: Capability): EtcHandshaker = { + val initialState = EtcHelloExchangeState(handshakerConfiguration, capability) EtcHandshaker(initialState, handshakerConfiguration) } diff --git a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala index b0757ffcb5..84a3c3c902 100644 --- a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala +++ b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala @@ -6,11 +6,11 @@ import io.iohk.ethereum.network.handshaker.Handshaker.NextMessage import io.iohk.ethereum.network.p2p.Message import io.iohk.ethereum.network.p2p.messages.Capability.Capabilities import io.iohk.ethereum.network.p2p.messages.Capability.Capabilities._ -import io.iohk.ethereum.network.p2p.messages.ProtocolVersions +import io.iohk.ethereum.network.p2p.messages.{Capability, ProtocolVersions} import io.iohk.ethereum.network.p2p.messages.WireProtocol.{Disconnect, Hello} import io.iohk.ethereum.utils.{Config, Logger, ServerStatus} -case class EtcHelloExchangeState(handshakerConfiguration: EtcHandshakerConfiguration) +case class EtcHelloExchangeState(handshakerConfiguration: EtcHandshakerConfiguration, capability: Capability) extends InProgressState[PeerInfo] with Logger { @@ -26,20 +26,13 @@ case class EtcHelloExchangeState(handshakerConfiguration: EtcHandshakerConfigura override def applyResponseMessage: PartialFunction[Message, HandshakerState[PeerInfo]] = { case hello: Hello => log.debug("Protocol handshake finished with peer ({})", hello) - if ( - handshakerConfiguration.capabilities - .contains(ProtocolVersions.PV64) && hello.capabilities.contains(Etc64Capability) - ) - EtcNodeStatus64ExchangeState(handshakerConfiguration) - else if (hello.capabilities.contains(Eth63Capability)) - EtcNodeStatus63ExchangeState(handshakerConfiguration) - else { - log.debug( - s"Connected peer does not support ${ProtocolVersions.PV63} / ${ProtocolVersions.PV64} protocol. Disconnecting." - ) - DisconnectedState(Disconnect.Reasons.IncompatibleP2pProtocolVersion) + capability match { + case ProtocolVersions.PV64 => EtcNodeStatus64ExchangeState(handshakerConfiguration) + case ProtocolVersions.PV63 => EtcNodeStatus63ExchangeState(handshakerConfiguration) + case _ => + log.debug(s"Connected peer does not support ${ProtocolVersions.PV63} / ${ProtocolVersions.PV64} protocol. Disconnecting.") + DisconnectedState(Disconnect.Reasons.IncompatibleP2pProtocolVersion) } - } override def processTimeout: HandshakerState[PeerInfo] = { diff --git a/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala b/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala index 61a9b79a70..dba386f942 100644 --- a/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala +++ b/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala @@ -152,17 +152,18 @@ class RLPxConnectionHandler( result match { case AuthHandshakeSuccess(secrets, remotePubKey) => log.info(s"Auth handshake succeeded for peer $peerId") - context.parent ! ConnectionEstablished(remotePubKey) + //expect Hello val frameCodec = new FrameCodec(secrets) val frames = frameCodec.readFrames(remainingData) frames.headOption.flatMap(extractHello) match { case Some(h) => - context.parent ! MessageReceived(h) val protocolVersion = Capability.negotiate(h.capabilities.toList, capabilities) val p2pVersion = h.p2pVersion protocolVersion match { case Some(value) => + context.parent ! ConnectionEstablished(remotePubKey, value) + context.parent ! MessageReceived(h) val messageCodec = messageCodecFactory(frameCodec, messageDecoder, value, p2pVersion) val restFrames = frames.drop(1) if (restFrames.nonEmpty) { @@ -340,7 +341,7 @@ object RLPxConnectionHandler { case class HandleConnection(connection: ActorRef) - case class ConnectionEstablished(nodeId: ByteString) + case class ConnectionEstablished(nodeId: ByteString, negotiatedCapability: Capability) case object ConnectionFailed diff --git a/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala b/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala index 785be10f71..effc144c6c 100644 --- a/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala +++ b/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala @@ -188,7 +188,8 @@ trait HandshakerBuilder { override val capabilities: List[Capability] = self.blockchainConfig.capabilities } - lazy val handshaker: Handshaker[PeerInfo] = EtcHandshaker(handshakerConfiguration) + lazy val handshaker: Capability => Handshaker[PeerInfo] = (cap: Capability) => + EtcHandshaker(handshakerConfiguration, cap) } trait AuthHandshakerBuilder { From 0e66afb2facf2c9403580cd7f0bdaa0746dbf781 Mon Sep 17 00:00:00 2001 From: Dominik Zajkowski Date: Sun, 6 Jun 2021 12:32:12 +0200 Subject: [PATCH 06/15] [ETCM-841] Reorganise RLPxConnectionHandler 'processHandshakeResult' --- .../ethereum/jsonrpc/EthInfoService.scala | 4 +- .../network/rlpx/RLPxConnectionHandler.scala | 42 +++++++++---------- 2 files changed, 21 insertions(+), 25 deletions(-) diff --git a/src/main/scala/io/iohk/ethereum/jsonrpc/EthInfoService.scala b/src/main/scala/io/iohk/ethereum/jsonrpc/EthInfoService.scala index b17f424020..2a4838cacb 100644 --- a/src/main/scala/io/iohk/ethereum/jsonrpc/EthInfoService.scala +++ b/src/main/scala/io/iohk/ethereum/jsonrpc/EthInfoService.scala @@ -76,14 +76,14 @@ class EthInfoService( stxLedger: StxLedger, keyStore: KeyStore, syncingController: ActorRef, - protocolVersion: Capability, + capability: Capability, askTimeout: Timeout ) extends ResolveBlock { import EthInfoService._ def protocolVersion(req: ProtocolVersionRequest): ServiceResponse[ProtocolVersionResponse] = - Task.now(Right(ProtocolVersionResponse(f"0x${protocolVersion.version}%x"))) + Task.now(Right(ProtocolVersionResponse(f"0x${capability.version}%x"))) def chainId(req: ChainIdRequest): ServiceResponse[ChainIdResponse] = Task.now(Right(ChainIdResponse(blockchainConfig.chainId))) diff --git a/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala b/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala index dba386f942..24129393b1 100644 --- a/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala +++ b/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala @@ -152,33 +152,29 @@ class RLPxConnectionHandler( result match { case AuthHandshakeSuccess(secrets, remotePubKey) => log.info(s"Auth handshake succeeded for peer $peerId") - - //expect Hello val frameCodec = new FrameCodec(secrets) val frames = frameCodec.readFrames(remainingData) - frames.headOption.flatMap(extractHello) match { - case Some(h) => - val protocolVersion = Capability.negotiate(h.capabilities.toList, capabilities) - val p2pVersion = h.p2pVersion - protocolVersion match { - case Some(value) => - context.parent ! ConnectionEstablished(remotePubKey, value) - context.parent ! MessageReceived(h) - val messageCodec = messageCodecFactory(frameCodec, messageDecoder, value, p2pVersion) - val restFrames = frames.drop(1) - if (restFrames.nonEmpty) { - val messagesSoFar = messageCodec.readFrames(restFrames) // omit hello - messagesSoFar foreach processMessage - } - context become handshaked(messageCodec) - case None => - log.debug(s"[Stopping Connection] Unable to connect to $peerId, no common capability found ${h.capabilities} and $capabilities") - context.parent ! ConnectionFailed - context stop self + val messageCodecOpt = for { + frame <- frames.headOption + hello <- extractHello(frame) + protocolVersion <- Capability.negotiate(hello.capabilities.toList, capabilities) + p2pVersion = hello.p2pVersion + messageCodec = messageCodecFactory(frameCodec, messageDecoder, protocolVersion, p2pVersion) + _ = context.parent ! ConnectionEstablished(remotePubKey, protocolVersion) + _ = context.parent ! MessageReceived(hello) + restFrames = frames.drop(1) + _ = { + if (restFrames.nonEmpty) { + val messagesSoFar = messageCodec.readFrames(restFrames) // omit hello + messagesSoFar foreach processMessage } - + } + } yield messageCodec + messageCodecOpt match { + case Some(messageCodec) => + context become handshaked(messageCodec) case None => - log.debug(s"[Stopping Connection] Unable to connect to $peerId, 'Hello' not found") + log.debug(s"[Stopping Connection] Unable to connect to $peerId") context.parent ! ConnectionFailed context stop self } From f6cef5d46f49c468d6d169ab3bfcd8127fc4d220 Mon Sep 17 00:00:00 2001 From: Dominik Zajkowski Date: Sun, 6 Jun 2021 12:43:40 +0200 Subject: [PATCH 07/15] [ETCM-841] Rename CommonMessages to BaseETH6XMessages --- .../scala/io/iohk/ethereum/rlp/RLPSpeedSuite.scala | 2 +- .../iohk/ethereum/sync/util/RegularSyncItSpecUtils.scala | 2 +- .../ethereum/blockchain/sync/regular/BlockBroadcast.scala | 4 ++-- .../ethereum/blockchain/sync/regular/BlockFetcher.scala | 4 ++-- src/main/scala/io/iohk/ethereum/domain/Block.scala | 4 ++-- src/main/scala/io/iohk/ethereum/domain/BlockBody.scala | 4 ++-- .../scala/io/iohk/ethereum/domain/SignedTransaction.scala | 2 +- .../io/iohk/ethereum/faucet/jsonrpc/WalletService.scala | 2 +- .../scala/io/iohk/ethereum/jsonrpc/EthTxService.scala | 2 +- .../io/iohk/ethereum/jsonrpc/RawTransactionCodec.scala | 2 +- .../io/iohk/ethereum/network/EtcPeerManagerActor.scala | 8 ++++---- .../network/handshaker/EtcNodeStatus63ExchangeState.scala | 8 ++++---- .../io/iohk/ethereum/network/p2p/MessageDecoders.scala | 6 +++--- .../{CommonMessages.scala => BaseETH6XMessages.scala} | 2 +- .../io/iohk/ethereum/network/p2p/messages/PV64.scala | 4 ++-- .../transactions/PendingTransactionsManager.scala | 2 +- .../transactions/SignedTransactionsFilterActor.scala | 2 +- .../scala/io/iohk/ethereum/rpcTest/RpcApiTests.scala | 2 +- src/test/scala/io/iohk/ethereum/ObjectGenerators.scala | 2 +- .../ethereum/blockchain/sync/BlockBroadcastSpec.scala | 4 ++-- .../ethereum/blockchain/sync/PivotBlockSelectorSpec.scala | 2 +- .../blockchain/sync/regular/BlockFetcherSpec.scala | 2 +- .../blockchain/sync/regular/RegularSyncSpec.scala | 6 +++--- .../iohk/ethereum/db/storage/BlockBodiesStorageSpec.scala | 6 +++--- .../iohk/ethereum/faucet/jsonrpc/WalletServiceSpec.scala | 2 +- .../io/iohk/ethereum/network/EtcPeerManagerSpec.scala | 2 +- .../iohk/ethereum/network/PeerActorHandshakingSpec.scala | 2 +- .../scala/io/iohk/ethereum/network/PeerManagerSpec.scala | 2 +- .../ethereum/network/handshaker/EtcHandshakerSpec.scala | 8 ++++---- .../io/iohk/ethereum/network/p2p/MessageCodecSpec.scala | 2 +- .../iohk/ethereum/network/p2p/MessageDecodersSpec.scala | 4 ++-- .../io/iohk/ethereum/network/p2p/PeerActorSpec.scala | 4 ++-- .../network/p2p/messages/MessagesSerializationSpec.scala | 2 +- .../iohk/ethereum/network/p2p/messages/NewBlockSpec.scala | 2 +- .../transactions/PendingTransactionsManagerSpec.scala | 2 +- 35 files changed, 58 insertions(+), 58 deletions(-) rename src/main/scala/io/iohk/ethereum/network/p2p/messages/{CommonMessages.scala => BaseETH6XMessages.scala} (99%) diff --git a/src/benchmark/scala/io/iohk/ethereum/rlp/RLPSpeedSuite.scala b/src/benchmark/scala/io/iohk/ethereum/rlp/RLPSpeedSuite.scala index ca3a0dbcfc..1db6324656 100644 --- a/src/benchmark/scala/io/iohk/ethereum/rlp/RLPSpeedSuite.scala +++ b/src/benchmark/scala/io/iohk/ethereum/rlp/RLPSpeedSuite.scala @@ -4,7 +4,7 @@ import akka.util.ByteString import io.iohk.ethereum.ObjectGenerators import io.iohk.ethereum.domain.Block._ import io.iohk.ethereum.domain._ -import io.iohk.ethereum.network.p2p.messages.CommonMessages.SignedTransactions._ +import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.SignedTransactions._ import io.iohk.ethereum.utils.Logger import io.iohk.ethereum.utils.Hex import org.scalacheck.Gen diff --git a/src/it/scala/io/iohk/ethereum/sync/util/RegularSyncItSpecUtils.scala b/src/it/scala/io/iohk/ethereum/sync/util/RegularSyncItSpecUtils.scala index ddc8f2cdc1..a7f8bc11d0 100644 --- a/src/it/scala/io/iohk/ethereum/sync/util/RegularSyncItSpecUtils.scala +++ b/src/it/scala/io/iohk/ethereum/sync/util/RegularSyncItSpecUtils.scala @@ -25,7 +25,7 @@ import io.iohk.ethereum.crypto import io.iohk.ethereum.domain._ import io.iohk.ethereum.ledger._ import io.iohk.ethereum.network.PeerId -import io.iohk.ethereum.network.p2p.messages.CommonMessages.NewBlock +import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.NewBlock import io.iohk.ethereum.nodebuilder.VmSetup import io.iohk.ethereum.ommers.OmmersPool import io.iohk.ethereum.sync.util.SyncCommonItSpecUtils.FakePeerCustomConfig.defaultConfig diff --git a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockBroadcast.scala b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockBroadcast.scala index e7400a7a31..3d08225c46 100644 --- a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockBroadcast.scala +++ b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockBroadcast.scala @@ -7,7 +7,7 @@ import io.iohk.ethereum.domain.{Block, ChainWeight} import io.iohk.ethereum.network.EtcPeerManagerActor.PeerInfo import io.iohk.ethereum.network.p2p.MessageSerializable import io.iohk.ethereum.network.p2p.messages.PV62.BlockHash -import io.iohk.ethereum.network.p2p.messages.{CommonMessages, PV62, PV64, ProtocolVersions} +import io.iohk.ethereum.network.p2p.messages.{BaseETH6XMessages, PV62, PV64, ProtocolVersions} import io.iohk.ethereum.network.{EtcPeerManagerActor, Peer, PeerId} import scala.util.Random @@ -73,7 +73,7 @@ object BlockBroadcast { * (they are different versions of NewBlock msg) */ case class BlockToBroadcast(block: Block, chainWeight: ChainWeight) { - def as63: CommonMessages.NewBlock = CommonMessages.NewBlock(block, chainWeight.totalDifficulty) + def as63: BaseETH6XMessages.NewBlock = BaseETH6XMessages.NewBlock(block, chainWeight.totalDifficulty) def as64: PV64.NewBlock = PV64.NewBlock(block, chainWeight) } } diff --git a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockFetcher.scala b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockFetcher.scala index 9cde469138..734c5f3074 100644 --- a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockFetcher.scala +++ b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockFetcher.scala @@ -23,7 +23,7 @@ import io.iohk.ethereum.network.PeerEventBusActor.SubscriptionClassifier.Message import io.iohk.ethereum.network.PeerEventBusActor.{PeerSelector, Subscribe} import io.iohk.ethereum.network.{Peer, PeerEventBusActor, PeerId} import io.iohk.ethereum.network.p2p.Message -import io.iohk.ethereum.network.p2p.messages.{Codes, CommonMessages, PV64} +import io.iohk.ethereum.network.p2p.messages.{Codes, BaseETH6XMessages, PV64} import io.iohk.ethereum.network.p2p.messages.PV62._ import io.iohk.ethereum.network.p2p.messages.PV63.NodeData import io.iohk.ethereum.utils.ByteStringUtils @@ -214,7 +214,7 @@ class BlockFetcher( supervisor ! ProgressProtocol.GotNewBlock(newState.knownTop) fetchBlocks(newState) - case AdaptedMessageFromEventBus(CommonMessages.NewBlock(block, _), peerId) => + case AdaptedMessageFromEventBus(BaseETH6XMessages.NewBlock(block, _), peerId) => handleNewBlock(block, peerId, state) case AdaptedMessageFromEventBus(PV64.NewBlock(block, _), peerId) => diff --git a/src/main/scala/io/iohk/ethereum/domain/Block.scala b/src/main/scala/io/iohk/ethereum/domain/Block.scala index e5f2bce150..220315889c 100644 --- a/src/main/scala/io/iohk/ethereum/domain/Block.scala +++ b/src/main/scala/io/iohk/ethereum/domain/Block.scala @@ -29,7 +29,7 @@ case class Block(header: BlockHeader, body: BlockBody) { object Block { implicit class BlockEnc(val obj: Block) extends RLPSerializable { - import io.iohk.ethereum.network.p2p.messages.CommonMessages.SignedTransactions._ + import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.SignedTransactions._ override def toRLPEncodable: RLPEncodeable = RLPList( obj.header.toRLPEncodable, @@ -39,7 +39,7 @@ object Block { } implicit class BlockDec(val bytes: Array[Byte]) extends AnyVal { - import io.iohk.ethereum.network.p2p.messages.CommonMessages.SignedTransactions._ + import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.SignedTransactions._ def toBlock: Block = rawDecode(bytes) match { case RLPList(header: RLPList, stx: RLPList, uncles: RLPList) => Block( diff --git a/src/main/scala/io/iohk/ethereum/domain/BlockBody.scala b/src/main/scala/io/iohk/ethereum/domain/BlockBody.scala index fbafe00c82..544b242a53 100644 --- a/src/main/scala/io/iohk/ethereum/domain/BlockBody.scala +++ b/src/main/scala/io/iohk/ethereum/domain/BlockBody.scala @@ -32,7 +32,7 @@ object BlockBody { implicit class BlockBodyEnc(msg: BlockBody) extends RLPSerializable { override def toRLPEncodable: RLPEncodeable = { - import io.iohk.ethereum.network.p2p.messages.CommonMessages.SignedTransactions._ + import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.SignedTransactions._ blockBodyToRlpEncodable( msg, @@ -62,7 +62,7 @@ object BlockBody { implicit class BlockBodyRLPEncodableDec(val rlpEncodeable: RLPEncodeable) { def toBlockBody: BlockBody = { - import io.iohk.ethereum.network.p2p.messages.CommonMessages.SignedTransactions._ + import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.SignedTransactions._ rlpEncodableToBlockBody( rlpEncodeable, diff --git a/src/main/scala/io/iohk/ethereum/domain/SignedTransaction.scala b/src/main/scala/io/iohk/ethereum/domain/SignedTransaction.scala index eca2539e82..caa01222f7 100644 --- a/src/main/scala/io/iohk/ethereum/domain/SignedTransaction.scala +++ b/src/main/scala/io/iohk/ethereum/domain/SignedTransaction.scala @@ -5,7 +5,7 @@ import com.google.common.cache.{Cache, CacheBuilder} import io.iohk.ethereum.crypto import io.iohk.ethereum.crypto.{ECDSASignature, kec256} import io.iohk.ethereum.mpt.ByteArraySerializable -import io.iohk.ethereum.network.p2p.messages.CommonMessages.SignedTransactions._ +import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.SignedTransactions._ import io.iohk.ethereum.rlp.RLPImplicitConversions._ import io.iohk.ethereum.rlp.RLPImplicits._ import io.iohk.ethereum.rlp.{encode => rlpEncode, _} diff --git a/src/main/scala/io/iohk/ethereum/faucet/jsonrpc/WalletService.scala b/src/main/scala/io/iohk/ethereum/faucet/jsonrpc/WalletService.scala index d6e7de85c3..1ee118e4ba 100644 --- a/src/main/scala/io/iohk/ethereum/faucet/jsonrpc/WalletService.scala +++ b/src/main/scala/io/iohk/ethereum/faucet/jsonrpc/WalletService.scala @@ -7,7 +7,7 @@ import io.iohk.ethereum.faucet.FaucetConfig import io.iohk.ethereum.jsonrpc.client.RpcClient.RpcError import io.iohk.ethereum.keystore.KeyStore.KeyStoreError import io.iohk.ethereum.keystore.{KeyStore, Wallet} -import io.iohk.ethereum.network.p2p.messages.CommonMessages.SignedTransactions.SignedTransactionEnc +import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.SignedTransactions.SignedTransactionEnc import io.iohk.ethereum.rlp import io.iohk.ethereum.utils.{ByteStringUtils, Logger} import monix.eval.Task diff --git a/src/main/scala/io/iohk/ethereum/jsonrpc/EthTxService.scala b/src/main/scala/io/iohk/ethereum/jsonrpc/EthTxService.scala index 7d867fdd18..2be469e8bf 100644 --- a/src/main/scala/io/iohk/ethereum/jsonrpc/EthTxService.scala +++ b/src/main/scala/io/iohk/ethereum/jsonrpc/EthTxService.scala @@ -168,7 +168,7 @@ class EthTxService( } def sendRawTransaction(req: SendRawTransactionRequest): ServiceResponse[SendRawTransactionResponse] = { - import io.iohk.ethereum.network.p2p.messages.CommonMessages.SignedTransactions.SignedTransactionDec + import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.SignedTransactions.SignedTransactionDec Try(req.data.toArray.toSignedTransaction) match { case Success(signedTransaction) => diff --git a/src/main/scala/io/iohk/ethereum/jsonrpc/RawTransactionCodec.scala b/src/main/scala/io/iohk/ethereum/jsonrpc/RawTransactionCodec.scala index 825736e7b4..cbe839b17d 100644 --- a/src/main/scala/io/iohk/ethereum/jsonrpc/RawTransactionCodec.scala +++ b/src/main/scala/io/iohk/ethereum/jsonrpc/RawTransactionCodec.scala @@ -2,7 +2,7 @@ package io.iohk.ethereum.jsonrpc import akka.util.ByteString import io.iohk.ethereum.domain.SignedTransaction -import io.iohk.ethereum.network.p2p.messages.CommonMessages.SignedTransactions.SignedTransactionEnc +import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.SignedTransactions.SignedTransactionEnc import io.iohk.ethereum.rlp object RawTransactionCodec { diff --git a/src/main/scala/io/iohk/ethereum/network/EtcPeerManagerActor.scala b/src/main/scala/io/iohk/ethereum/network/EtcPeerManagerActor.scala index 3a348e0e9e..eaea5a2e34 100644 --- a/src/main/scala/io/iohk/ethereum/network/EtcPeerManagerActor.scala +++ b/src/main/scala/io/iohk/ethereum/network/EtcPeerManagerActor.scala @@ -13,7 +13,7 @@ import io.iohk.ethereum.network.handshaker.Handshaker.HandshakeResult import io.iohk.ethereum.network.p2p.messages.PV62.{BlockHeaders, GetBlockHeaders, NewBlockHashes} import io.iohk.ethereum.network.p2p.messages.PV64.NewBlock import io.iohk.ethereum.network.p2p.messages.WireProtocol.Disconnect -import io.iohk.ethereum.network.p2p.messages.{Codes, CommonMessages, PV64} +import io.iohk.ethereum.network.p2p.messages.{Codes, BaseETH6XMessages, PV64} import io.iohk.ethereum.network.p2p.{Message, MessageSerializable} import io.iohk.ethereum.utils.ByteStringUtils @@ -159,7 +159,7 @@ class EtcPeerManagerActor( */ private def updateChainWeight(message: Message)(initialPeerInfo: PeerInfo): PeerInfo = message match { - case newBlock: CommonMessages.NewBlock => + case newBlock: BaseETH6XMessages.NewBlock => initialPeerInfo.copy(chainWeight = ChainWeight.totalDifficultyOnly(newBlock.totalDifficulty)) case newBlock: PV64.NewBlock => initialPeerInfo.copy(chainWeight = newBlock.chainWeight) case _ => initialPeerInfo @@ -220,7 +220,7 @@ class EtcPeerManagerActor( message match { case m: BlockHeaders => update(m.headers.map(header => (header.number, header.hash))) - case m: CommonMessages.NewBlock => + case m: BaseETH6XMessages.NewBlock => update(Seq((m.block.header.number, m.block.header.hash))) case m: NewBlock => update(Seq((m.block.header.number, m.block.header.hash))) @@ -263,7 +263,7 @@ object EtcPeerManagerActor { RemoteStatus(status.protocolVersion, status.networkId, status.chainWeight, status.bestHash, status.genesisHash) } - def apply(status: CommonMessages.Status): RemoteStatus = { + def apply(status: BaseETH6XMessages.Status): RemoteStatus = { RemoteStatus( status.protocolVersion, status.networkId, diff --git a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcNodeStatus63ExchangeState.scala b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcNodeStatus63ExchangeState.scala index 7768b837b5..276cd2342c 100644 --- a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcNodeStatus63ExchangeState.scala +++ b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcNodeStatus63ExchangeState.scala @@ -1,17 +1,17 @@ package io.iohk.ethereum.network.handshaker import io.iohk.ethereum.network.EtcPeerManagerActor.{PeerInfo, RemoteStatus} -import io.iohk.ethereum.network.p2p.messages.{CommonMessages, ProtocolVersions} +import io.iohk.ethereum.network.p2p.messages.{BaseETH6XMessages, ProtocolVersions} import io.iohk.ethereum.network.p2p.{Message, MessageSerializable} case class EtcNodeStatus63ExchangeState( handshakerConfiguration: EtcHandshakerConfiguration -) extends EtcNodeStatusExchangeState[CommonMessages.Status] { +) extends EtcNodeStatusExchangeState[BaseETH6XMessages.Status] { import handshakerConfiguration._ def applyResponseMessage: PartialFunction[Message, HandshakerState[PeerInfo]] = { - case status: CommonMessages.Status => + case status: BaseETH6XMessages.Status => applyRemoteStatusMessage(RemoteStatus(status)) } @@ -19,7 +19,7 @@ case class EtcNodeStatus63ExchangeState( val bestBlockHeader = getBestBlockHeader() val chainWeight = blockchain.getChainWeightByHash(bestBlockHeader.hash).get - val status = CommonMessages.Status( + val status = BaseETH6XMessages.Status( protocolVersion = ProtocolVersions.PV63.version, networkId = peerConfiguration.networkId, totalDifficulty = chainWeight.totalDifficulty, diff --git a/src/main/scala/io/iohk/ethereum/network/p2p/MessageDecoders.scala b/src/main/scala/io/iohk/ethereum/network/p2p/MessageDecoders.scala index b7a57137c5..24b34b9c64 100644 --- a/src/main/scala/io/iohk/ethereum/network/p2p/MessageDecoders.scala +++ b/src/main/scala/io/iohk/ethereum/network/p2p/MessageDecoders.scala @@ -1,7 +1,7 @@ package io.iohk.ethereum.network.p2p import io.iohk.ethereum.network.p2p.messages.{Capability, Codes} -import io.iohk.ethereum.network.p2p.messages.CommonMessages.SignedTransactions._ +import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.SignedTransactions._ import io.iohk.ethereum.network.p2p.messages.PV61.BlockHashesFromNumber._ import io.iohk.ethereum.network.p2p.messages.PV62.BlockBodies._ import io.iohk.ethereum.network.p2p.messages.PV62.BlockHeaders._ @@ -48,10 +48,10 @@ object EthereumMessageDecoder extends MessageDecoder { private def handleCommonMessages(msgCode: Int, payload: Array[Byte]): Message = { msgCode match { case Codes.StatusCode => - import io.iohk.ethereum.network.p2p.messages.CommonMessages.Status._ + import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.Status._ payload.toStatus case Codes.NewBlockCode => - import io.iohk.ethereum.network.p2p.messages.CommonMessages.NewBlock._ + import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.NewBlock._ payload.toNewBlock case Codes.SignedTransactionsCode => payload.toSignedTransactions diff --git a/src/main/scala/io/iohk/ethereum/network/p2p/messages/CommonMessages.scala b/src/main/scala/io/iohk/ethereum/network/p2p/messages/BaseETH6XMessages.scala similarity index 99% rename from src/main/scala/io/iohk/ethereum/network/p2p/messages/CommonMessages.scala rename to src/main/scala/io/iohk/ethereum/network/p2p/messages/BaseETH6XMessages.scala index 6b312f21f5..0d45b7d4f3 100644 --- a/src/main/scala/io/iohk/ethereum/network/p2p/messages/CommonMessages.scala +++ b/src/main/scala/io/iohk/ethereum/network/p2p/messages/BaseETH6XMessages.scala @@ -11,7 +11,7 @@ import io.iohk.ethereum.utils.ByteStringUtils.ByteStringOps import io.iohk.ethereum.utils.Config import org.bouncycastle.util.encoders.Hex -object CommonMessages { +object BaseETH6XMessages { object Status { implicit class StatusEnc(val underlyingMsg: Status) extends MessageSerializableImplicit[Status](underlyingMsg) diff --git a/src/main/scala/io/iohk/ethereum/network/p2p/messages/PV64.scala b/src/main/scala/io/iohk/ethereum/network/p2p/messages/PV64.scala index 9df3b15fa2..1ef6a43efd 100644 --- a/src/main/scala/io/iohk/ethereum/network/p2p/messages/PV64.scala +++ b/src/main/scala/io/iohk/ethereum/network/p2p/messages/PV64.scala @@ -83,7 +83,7 @@ object PV64 { implicit class NewBlockEnc(val underlyingMsg: NewBlock) extends MessageSerializableImplicit[NewBlock](underlyingMsg) with RLPSerializable { - import io.iohk.ethereum.network.p2p.messages.CommonMessages.SignedTransactions._ + import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.SignedTransactions._ override def code: Int = Codes.NewBlockCode @@ -102,7 +102,7 @@ object PV64 { } implicit class NewBlockDec(val bytes: Array[Byte]) extends AnyVal { - import io.iohk.ethereum.network.p2p.messages.CommonMessages.SignedTransactions._ + import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.SignedTransactions._ def toNewBlock: NewBlock = rawDecode(bytes) match { case RLPList( diff --git a/src/main/scala/io/iohk/ethereum/transactions/PendingTransactionsManager.scala b/src/main/scala/io/iohk/ethereum/transactions/PendingTransactionsManager.scala index fb920b20d2..c5f188e8a4 100644 --- a/src/main/scala/io/iohk/ethereum/transactions/PendingTransactionsManager.scala +++ b/src/main/scala/io/iohk/ethereum/transactions/PendingTransactionsManager.scala @@ -7,7 +7,7 @@ import io.iohk.ethereum.domain.{SignedTransaction, SignedTransactionWithSender} import io.iohk.ethereum.metrics.MetricsContainer import io.iohk.ethereum.network.PeerEventBusActor.{PeerEvent, Subscribe, SubscriptionClassifier} import io.iohk.ethereum.network.PeerManagerActor.Peers -import io.iohk.ethereum.network.p2p.messages.CommonMessages.SignedTransactions +import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.SignedTransactions import io.iohk.ethereum.network.{EtcPeerManagerActor, Peer, PeerId, PeerManagerActor} import io.iohk.ethereum.transactions.SignedTransactionsFilterActor.ProperSignedTransactions import io.iohk.ethereum.utils.ByteStringUtils.ByteStringOps diff --git a/src/main/scala/io/iohk/ethereum/transactions/SignedTransactionsFilterActor.scala b/src/main/scala/io/iohk/ethereum/transactions/SignedTransactionsFilterActor.scala index 1dc03d7855..3602df113f 100644 --- a/src/main/scala/io/iohk/ethereum/transactions/SignedTransactionsFilterActor.scala +++ b/src/main/scala/io/iohk/ethereum/transactions/SignedTransactionsFilterActor.scala @@ -8,7 +8,7 @@ import io.iohk.ethereum.network.PeerEventBusActor.{PeerSelector, Subscribe} import io.iohk.ethereum.network.PeerEventBusActor.SubscriptionClassifier.MessageClassifier import io.iohk.ethereum.network.PeerId import io.iohk.ethereum.network.p2p.messages.Codes -import io.iohk.ethereum.network.p2p.messages.CommonMessages.SignedTransactions +import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.SignedTransactions import io.iohk.ethereum.transactions.SignedTransactionsFilterActor.ProperSignedTransactions class SignedTransactionsFilterActor(pendingTransactionsManager: ActorRef, peerEventBus: ActorRef) diff --git a/src/rpcTest/scala/io/iohk/ethereum/rpcTest/RpcApiTests.scala b/src/rpcTest/scala/io/iohk/ethereum/rpcTest/RpcApiTests.scala index 0578c3f0dc..a9fffe546f 100644 --- a/src/rpcTest/scala/io/iohk/ethereum/rpcTest/RpcApiTests.scala +++ b/src/rpcTest/scala/io/iohk/ethereum/rpcTest/RpcApiTests.scala @@ -16,7 +16,7 @@ import org.web3j.protocol.core.DefaultBlockParameter import org.web3j.protocol.core.methods.request.{EthFilter, Transaction} import org.web3j.protocol.core.methods.response.EthBlock.{TransactionHash, TransactionObject} import org.web3j.protocol.http.HttpService -import io.iohk.ethereum.network.p2p.messages.CommonMessages.SignedTransactions.SignedTransactionEnc +import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.SignedTransactions.SignedTransactionEnc import org.web3j.protocol.core.methods.response.EthLog.{Hash, LogObject} import io.iohk.ethereum.rpcTest.TestContracts._ import io.iohk.ethereum.rpcTest.TestData._ diff --git a/src/test/scala/io/iohk/ethereum/ObjectGenerators.scala b/src/test/scala/io/iohk/ethereum/ObjectGenerators.scala index c4b27eb6f5..415ab214d2 100644 --- a/src/test/scala/io/iohk/ethereum/ObjectGenerators.scala +++ b/src/test/scala/io/iohk/ethereum/ObjectGenerators.scala @@ -11,7 +11,7 @@ import io.iohk.ethereum.domain.BlockHeader.HeaderExtraFields._ import io.iohk.ethereum.domain._ import io.iohk.ethereum.mpt.HexPrefix.bytesToNibbles import io.iohk.ethereum.mpt.{BranchNode, ExtensionNode, HashNode, LeafNode, MptNode, MptTraversals} -import io.iohk.ethereum.network.p2p.messages.CommonMessages.NewBlock +import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.NewBlock import io.iohk.ethereum.network.p2p.messages.PV64 import org.bouncycastle.crypto.AsymmetricCipherKeyPair import org.scalacheck.{Arbitrary, Gen, Shrink} diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/BlockBroadcastSpec.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/BlockBroadcastSpec.scala index 41bf3937ca..25226e23bd 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/BlockBroadcastSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/BlockBroadcastSpec.scala @@ -11,7 +11,7 @@ import io.iohk.ethereum.domain.{Block, BlockBody, BlockHeader, ChainWeight} import io.iohk.ethereum.network.EtcPeerManagerActor.{PeerInfo, RemoteStatus} import io.iohk.ethereum.network.p2p.messages.PV62.NewBlockHashes import io.iohk.ethereum.network.p2p.messages.PV64.NewBlock -import io.iohk.ethereum.network.p2p.messages.{CommonMessages, PV62, ProtocolVersions} +import io.iohk.ethereum.network.p2p.messages.{BaseETH6XMessages, PV62, ProtocolVersions} import io.iohk.ethereum.network.{EtcPeerManagerActor, Peer, PeerId} import io.iohk.ethereum.{Fixtures, WithActorSystemShutDown} import org.scalatest.flatspec.AnyFlatSpecLike @@ -52,7 +52,7 @@ class BlockBroadcastSpec .copy(remoteStatus = peerStatus.copy(protocolVersion = ProtocolVersions.PV63.version)) .withChainWeight(ChainWeight.totalDifficultyOnly(initialPeerInfo.chainWeight.totalDifficulty)) val newBlock = - CommonMessages.NewBlock(Block(blockHeader, BlockBody(Nil, Nil)), peerInfo.chainWeight.totalDifficulty + 2) + BaseETH6XMessages.NewBlock(Block(blockHeader, BlockBody(Nil, Nil)), peerInfo.chainWeight.totalDifficulty + 2) //when blockBroadcast.broadcastBlock( diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/PivotBlockSelectorSpec.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/PivotBlockSelectorSpec.scala index 3a9c6fd664..7340e98f58 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/PivotBlockSelectorSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/PivotBlockSelectorSpec.scala @@ -13,7 +13,7 @@ import io.iohk.ethereum.network.PeerEventBusActor.PeerEvent.MessageFromPeer import io.iohk.ethereum.network.PeerEventBusActor.SubscriptionClassifier.{MessageClassifier, PeerDisconnectedClassifier} import io.iohk.ethereum.network.PeerEventBusActor.{PeerSelector, Subscribe, Unsubscribe} import io.iohk.ethereum.network.p2p.Message -import io.iohk.ethereum.network.p2p.messages.CommonMessages.NewBlock +import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.NewBlock import io.iohk.ethereum.network.p2p.messages.PV62._ import io.iohk.ethereum.network.p2p.messages.{Codes, ProtocolVersions} import io.iohk.ethereum.network.{EtcPeerManagerActor, Peer, PeerId} diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/BlockFetcherSpec.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/BlockFetcherSpec.scala index 05f86e1f1a..5494247733 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/BlockFetcherSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/BlockFetcherSpec.scala @@ -21,7 +21,7 @@ import io.iohk.ethereum.domain.{Block, HeadersSeq} import io.iohk.ethereum.network.PeerEventBusActor.SubscriptionClassifier.MessageClassifier import io.iohk.ethereum.network.PeerEventBusActor.{PeerSelector, Subscribe} import io.iohk.ethereum.network.p2p.messages.Codes -import io.iohk.ethereum.network.p2p.messages.CommonMessages.NewBlock +import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.NewBlock import io.iohk.ethereum.network.p2p.messages.PV62._ import io.iohk.ethereum.network.{Peer, PeerId} import io.iohk.ethereum.security.SecureRandomBuilder diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncSpec.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncSpec.scala index 1ae44a3ba5..1221d4cf63 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncSpec.scala @@ -23,7 +23,7 @@ import io.iohk.ethereum.network.EtcPeerManagerActor.{GetHandshakedPeers, Handsha import io.iohk.ethereum.network.PeerEventBusActor.PeerEvent.MessageFromPeer import io.iohk.ethereum.network.PeerEventBusActor.SubscriptionClassifier.MessageClassifier import io.iohk.ethereum.network.PeerEventBusActor.{PeerSelector, Subscribe} -import io.iohk.ethereum.network.p2p.messages.{Codes, CommonMessages, ProtocolVersions} +import io.iohk.ethereum.network.p2p.messages.{Codes, BaseETH6XMessages, ProtocolVersions} import io.iohk.ethereum.network.p2p.messages.PV62._ import io.iohk.ethereum.network.p2p.messages.PV63.{GetNodeData, NodeData} import io.iohk.ethereum.network.p2p.messages.PV64.NewBlock @@ -688,12 +688,12 @@ class RegularSyncSpec etcPeerManager.expectMsg(GetHandshakedPeers) etcPeerManager.reply(HandshakedPeers(Map(peerWithPV63._1 -> peerWithPV63._2))) - blockFetcher ! MessageFromPeer(CommonMessages.NewBlock(newBlock, newBlock.number), defaultPeer.id) + blockFetcher ! MessageFromPeer(BaseETH6XMessages.NewBlock(newBlock, newBlock.number), defaultPeer.id) etcPeerManager.fishForSpecificMessageMatching() { case EtcPeerManagerActor.SendMessage(message, _) => message.underlyingMsg match { - case CommonMessages.NewBlock(`newBlock`, _) => true + case BaseETH6XMessages.NewBlock(`newBlock`, _) => true case _ => false } case _ => false diff --git a/src/test/scala/io/iohk/ethereum/db/storage/BlockBodiesStorageSpec.scala b/src/test/scala/io/iohk/ethereum/db/storage/BlockBodiesStorageSpec.scala index 910a305d74..50fb2b175d 100644 --- a/src/test/scala/io/iohk/ethereum/db/storage/BlockBodiesStorageSpec.scala +++ b/src/test/scala/io/iohk/ethereum/db/storage/BlockBodiesStorageSpec.scala @@ -3,8 +3,8 @@ package io.iohk.ethereum.db.storage import io.iohk.ethereum.ObjectGenerators import io.iohk.ethereum.db.dataSource.EphemDataSource import io.iohk.ethereum.security.SecureRandomBuilder -import io.iohk.ethereum.network.p2p.messages.CommonMessages -import io.iohk.ethereum.network.p2p.messages.CommonMessages.NewBlock +import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages +import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.NewBlock import org.bouncycastle.util.encoders.Hex import org.scalacheck.Gen import org.scalatestplus.scalacheck.ScalaCheckPropertyChecks @@ -52,7 +52,7 @@ class BlockBodiesStorageSpec } } - def insertBlockBodiesMapping(newBlocks: Seq[CommonMessages.NewBlock]): BlockBodiesStorage = { + def insertBlockBodiesMapping(newBlocks: Seq[BaseETH6XMessages.NewBlock]): BlockBodiesStorage = { val storage = new BlockBodiesStorage(EphemDataSource()) val batchUpdates = newBlocks.foldLeft(storage.emptyBatchUpdate) { case (updates, NewBlock(block, _)) => diff --git a/src/test/scala/io/iohk/ethereum/faucet/jsonrpc/WalletServiceSpec.scala b/src/test/scala/io/iohk/ethereum/faucet/jsonrpc/WalletServiceSpec.scala index 2d8f44f83d..1fb1066253 100644 --- a/src/test/scala/io/iohk/ethereum/faucet/jsonrpc/WalletServiceSpec.scala +++ b/src/test/scala/io/iohk/ethereum/faucet/jsonrpc/WalletServiceSpec.scala @@ -9,7 +9,7 @@ import io.iohk.ethereum.faucet.{FaucetConfig, RpcClientConfig, SupervisorConfig} import io.iohk.ethereum.jsonrpc.client.RpcClient.ConnectionError import io.iohk.ethereum.keystore.KeyStore.DecryptionFailed import io.iohk.ethereum.keystore.{KeyStore, Wallet} -import io.iohk.ethereum.network.p2p.messages.CommonMessages.SignedTransactions.SignedTransactionEnc +import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.SignedTransactions.SignedTransactionEnc import io.iohk.ethereum.{crypto, rlp} import monix.eval.Task import monix.execution.Scheduler.Implicits.global diff --git a/src/test/scala/io/iohk/ethereum/network/EtcPeerManagerSpec.scala b/src/test/scala/io/iohk/ethereum/network/EtcPeerManagerSpec.scala index 25f4ca5810..9b8018b631 100644 --- a/src/test/scala/io/iohk/ethereum/network/EtcPeerManagerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/EtcPeerManagerSpec.scala @@ -14,7 +14,7 @@ import io.iohk.ethereum.network.PeerActor.DisconnectPeer import io.iohk.ethereum.network.PeerEventBusActor.PeerEvent.{MessageFromPeer, PeerDisconnected, PeerHandshakeSuccessful} import io.iohk.ethereum.network.PeerEventBusActor.SubscriptionClassifier._ import io.iohk.ethereum.network.PeerEventBusActor.{PeerSelector, Subscribe} -import io.iohk.ethereum.network.p2p.messages.CommonMessages.NewBlock +import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.NewBlock import io.iohk.ethereum.network.p2p.messages.PV62._ import io.iohk.ethereum.network.p2p.messages.WireProtocol.Disconnect import io.iohk.ethereum.network.p2p.messages.{Codes, PV64, ProtocolVersions} diff --git a/src/test/scala/io/iohk/ethereum/network/PeerActorHandshakingSpec.scala b/src/test/scala/io/iohk/ethereum/network/PeerActorHandshakingSpec.scala index b16898824a..f2089fe92a 100644 --- a/src/test/scala/io/iohk/ethereum/network/PeerActorHandshakingSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/PeerActorHandshakingSpec.scala @@ -15,7 +15,7 @@ import io.iohk.ethereum.network.handshaker.Handshaker.NextMessage import io.iohk.ethereum.network.handshaker._ import io.iohk.ethereum.network.p2p.Message import io.iohk.ethereum.network.p2p.messages.Capability.Capabilities._ -import io.iohk.ethereum.network.p2p.messages.CommonMessages.Status +import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.Status import io.iohk.ethereum.network.p2p.messages.ProtocolVersions import io.iohk.ethereum.network.p2p.messages.WireProtocol.{Disconnect, Hello, Pong} import io.iohk.ethereum.network.rlpx.RLPxConnectionHandler diff --git a/src/test/scala/io/iohk/ethereum/network/PeerManagerSpec.scala b/src/test/scala/io/iohk/ethereum/network/PeerManagerSpec.scala index 7426f582b9..c54c96b5a0 100644 --- a/src/test/scala/io/iohk/ethereum/network/PeerManagerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/PeerManagerSpec.scala @@ -15,7 +15,7 @@ import io.iohk.ethereum.network.PeerEventBusActor.SubscriptionClassifier.PeerHan import io.iohk.ethereum.network.PeerEventBusActor.{PeerEvent, Publish, Subscribe} import io.iohk.ethereum.network.PeerManagerActor.{GetPeers, PeerAddress, PeerConfiguration, Peers, SendMessage} import io.iohk.ethereum.network.discovery.{DiscoveryConfig, Node, PeerDiscoveryManager} -import io.iohk.ethereum.network.p2p.messages.CommonMessages.NewBlock +import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.NewBlock import io.iohk.ethereum.network.p2p.messages.ProtocolVersions import io.iohk.ethereum.network.p2p.messages.WireProtocol.Disconnect import io.iohk.ethereum.utils.Config diff --git a/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala b/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala index 5a92fbb876..c33484da1d 100644 --- a/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala @@ -13,12 +13,12 @@ import io.iohk.ethereum.network.ForkResolver import io.iohk.ethereum.network.PeerManagerActor.PeerConfiguration import io.iohk.ethereum.network.handshaker.Handshaker.HandshakeComplete.{HandshakeFailure, HandshakeSuccess} import io.iohk.ethereum.network.p2p.messages.Capability.Capabilities._ -import io.iohk.ethereum.network.p2p.messages.CommonMessages.Status.StatusEnc +import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.Status.StatusEnc import io.iohk.ethereum.network.p2p.messages.PV62.GetBlockHeaders.GetBlockHeadersEnc import io.iohk.ethereum.network.p2p.messages.PV62.{BlockHeaders, GetBlockHeaders} import io.iohk.ethereum.network.p2p.messages.WireProtocol.Hello.HelloEnc import io.iohk.ethereum.network.p2p.messages.WireProtocol.{Disconnect, Hello} -import io.iohk.ethereum.network.p2p.messages.{Capability, CommonMessages, PV64, ProtocolVersions} +import io.iohk.ethereum.network.p2p.messages.{Capability, BaseETH6XMessages, PV64, ProtocolVersions} import io.iohk.ethereum.utils._ import io.iohk.ethereum.security.SecureRandomBuilder import io.iohk.ethereum.utils.ByteStringUtils._ @@ -308,7 +308,7 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { } trait LocalPeerPV63Setup extends LocalPeerSetup { - val localStatusMsg = CommonMessages.Status( + val localStatusMsg = BaseETH6XMessages.Status( protocolVersion = ProtocolVersions.PV63.version, networkId = Config.Network.peer.networkId, totalDifficulty = genesisBlock.header.difficulty, @@ -347,7 +347,7 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { nodeId = ByteString(remoteNodeStatus.nodeId) ) - val remoteStatusMsg = CommonMessages.Status( + val remoteStatusMsg = BaseETH6XMessages.Status( protocolVersion = ProtocolVersions.PV63.version, networkId = Config.Network.peer.networkId, totalDifficulty = 0, diff --git a/src/test/scala/io/iohk/ethereum/network/p2p/MessageCodecSpec.scala b/src/test/scala/io/iohk/ethereum/network/p2p/MessageCodecSpec.scala index 03ef6ad53d..b811e2cb77 100644 --- a/src/test/scala/io/iohk/ethereum/network/p2p/MessageCodecSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/p2p/MessageCodecSpec.scala @@ -3,7 +3,7 @@ package io.iohk.ethereum.network.p2p import akka.util.ByteString import io.iohk.ethereum.network.handshaker.EtcHelloExchangeState import io.iohk.ethereum.network.p2p.messages.Capability.Capabilities._ -import io.iohk.ethereum.network.p2p.messages.CommonMessages.Status +import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.Status import io.iohk.ethereum.network.p2p.messages.ProtocolVersions import io.iohk.ethereum.network.p2p.messages.WireProtocol.Hello import io.iohk.ethereum.network.rlpx.{FrameCodec, MessageCodec} diff --git a/src/test/scala/io/iohk/ethereum/network/p2p/MessageDecodersSpec.scala b/src/test/scala/io/iohk/ethereum/network/p2p/MessageDecodersSpec.scala index 9356d78bec..1639f7a448 100644 --- a/src/test/scala/io/iohk/ethereum/network/p2p/MessageDecodersSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/p2p/MessageDecodersSpec.scala @@ -4,7 +4,7 @@ import akka.util.ByteString import io.iohk.ethereum.{Fixtures, ObjectGenerators} import io.iohk.ethereum.domain.ChainWeight import io.iohk.ethereum.network.p2p.messages.Capability.Capabilities._ -import io.iohk.ethereum.network.p2p.messages.CommonMessages.SignedTransactions +import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.SignedTransactions import io.iohk.ethereum.network.p2p.messages._ import io.iohk.ethereum.security.SecureRandomBuilder import org.bouncycastle.util.encoders.Hex @@ -181,7 +181,7 @@ class MessageDecodersSpec extends AnyFlatSpec with Matchers with SecureRandomBui } it should "decode Status message for all supported versions of protocol" in { - val status63 = CommonMessages.Status(ProtocolVersions.PV63.version, 1, BigInt(100), exampleHash, exampleHash) + val status63 = BaseETH6XMessages.Status(ProtocolVersions.PV63.version, 1, BigInt(100), exampleHash, exampleHash) val status63Bytes: Array[Byte] = status63.toBytes val status64 = PV64.Status(ProtocolVersions.PV63.version, 1, ChainWeight(1, BigInt(100)), exampleHash, exampleHash) diff --git a/src/test/scala/io/iohk/ethereum/network/p2p/PeerActorSpec.scala b/src/test/scala/io/iohk/ethereum/network/p2p/PeerActorSpec.scala index 8f369ce984..d25a71354c 100644 --- a/src/test/scala/io/iohk/ethereum/network/p2p/PeerActorSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/p2p/PeerActorSpec.scala @@ -18,8 +18,8 @@ import io.iohk.ethereum.network.PeerActor.{GetStatus, StatusResponse} import io.iohk.ethereum.network.PeerManagerActor.{FastSyncHostConfiguration, PeerConfiguration} import io.iohk.ethereum.network.handshaker.{EtcHandshaker, EtcHandshakerConfiguration} import io.iohk.ethereum.network.p2p.messages.Capability.Capabilities._ -import io.iohk.ethereum.network.p2p.messages.CommonMessages.Status -import io.iohk.ethereum.network.p2p.messages.CommonMessages.Status.StatusEnc +import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.Status +import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.Status.StatusEnc import io.iohk.ethereum.network.p2p.messages.PV62.GetBlockHeaders.GetBlockHeadersEnc import io.iohk.ethereum.network.p2p.messages.PV62._ import io.iohk.ethereum.network.p2p.messages.{Capability, PV64, ProtocolVersions} diff --git a/src/test/scala/io/iohk/ethereum/network/p2p/messages/MessagesSerializationSpec.scala b/src/test/scala/io/iohk/ethereum/network/p2p/messages/MessagesSerializationSpec.scala index 088e635a1f..d07d738bdd 100644 --- a/src/test/scala/io/iohk/ethereum/network/p2p/messages/MessagesSerializationSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/p2p/messages/MessagesSerializationSpec.scala @@ -3,7 +3,7 @@ package io.iohk.ethereum.network.p2p.messages import akka.util.ByteString import io.iohk.ethereum.Fixtures import io.iohk.ethereum.domain.ChainWeight -import io.iohk.ethereum.network.p2p.messages.CommonMessages._ +import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages._ import io.iohk.ethereum.network.p2p.messages.PV61.BlockHashesFromNumber import io.iohk.ethereum.network.p2p.messages.PV62._ import io.iohk.ethereum.network.p2p.messages.WireProtocol._ diff --git a/src/test/scala/io/iohk/ethereum/network/p2p/messages/NewBlockSpec.scala b/src/test/scala/io/iohk/ethereum/network/p2p/messages/NewBlockSpec.scala index 3237068257..2b17ee5994 100644 --- a/src/test/scala/io/iohk/ethereum/network/p2p/messages/NewBlockSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/p2p/messages/NewBlockSpec.scala @@ -3,7 +3,7 @@ package io.iohk.ethereum.network.p2p.messages import akka.util.ByteString import io.iohk.ethereum.ObjectGenerators import io.iohk.ethereum.domain.{Block, BlockBody, BlockHeader, ChainWeight} -import io.iohk.ethereum.network.p2p.messages.CommonMessages.NewBlock +import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.NewBlock import org.bouncycastle.util.encoders.Hex import NewBlock._ import io.iohk.ethereum.security.SecureRandomBuilder diff --git a/src/test/scala/io/iohk/ethereum/transactions/PendingTransactionsManagerSpec.scala b/src/test/scala/io/iohk/ethereum/transactions/PendingTransactionsManagerSpec.scala index b1503d21cc..e27392d51a 100644 --- a/src/test/scala/io/iohk/ethereum/transactions/PendingTransactionsManagerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/transactions/PendingTransactionsManagerSpec.scala @@ -12,7 +12,7 @@ import io.iohk.ethereum.network.PeerActor.Status.Handshaked import io.iohk.ethereum.network.PeerEventBusActor.PeerEvent import io.iohk.ethereum.network.PeerManagerActor.Peers import io.iohk.ethereum.network.handshaker.Handshaker.HandshakeResult -import io.iohk.ethereum.network.p2p.messages.CommonMessages.SignedTransactions +import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.SignedTransactions import io.iohk.ethereum.network.{EtcPeerManagerActor, Peer, PeerId, PeerManagerActor} import io.iohk.ethereum.transactions.PendingTransactionsManager._ import io.iohk.ethereum.{NormalPatience, Timeouts, crypto} From 964f8c6088024dea3f6af12c280cc91d3fde1628 Mon Sep 17 00:00:00 2001 From: Dominik Zajkowski Date: Wed, 9 Jun 2021 15:27:56 +0200 Subject: [PATCH 08/15] [ETCM-841] Align unit tests to changes in flow of protocol negotiation --- .../ethereum/sync/util/CommonFakePeer.scala | 4 +- .../txExecTest/util/DumpChainApp.scala | 4 +- .../ethereum/network/rlpx/FrameCodec.scala | 6 ++- .../ethereum/network/rlpx/MessageCodec.scala | 2 +- .../network/rlpx/RLPxConnectionHandler.scala | 48 +++++++++++-------- .../network/PeerActorHandshakingSpec.scala | 29 ++++++----- .../handshaker/EtcHandshakerSpec.scala | 16 +++++-- .../network/p2p/MessageCodecSpec.scala | 10 +++- .../ethereum/network/p2p/PeerActorSpec.scala | 30 ++++++------ .../rlpx/RLPxConnectionHandlerSpec.scala | 19 ++++++-- 10 files changed, 99 insertions(+), 69 deletions(-) diff --git a/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala b/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala index a1df0f2d3d..dd1e2f3e8b 100644 --- a/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala +++ b/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala @@ -172,7 +172,7 @@ abstract class CommonFakePeer(peerName: String, fakePeerCustomConfig: FakePeerCu override val capabilities: List[Capability] = blockchainConfig.capabilities } - lazy val handshaker: Handshaker[PeerInfo] = EtcHandshaker(handshakerConfiguration) + lazy val handshaker: Capability => Handshaker[PeerInfo] = c => EtcHandshaker(handshakerConfiguration, c) lazy val authHandshaker: AuthHandshaker = AuthHandshaker(nodeKey, secureRandom) @@ -193,7 +193,7 @@ abstract class CommonFakePeer(peerName: String, fakePeerCustomConfig: FakePeerCu EthereumMessageDecoder, discoveryConfig, blacklist, - Capability.best(blockchainConfig.capabilities) + blockchainConfig.capabilities ), "peer-manager" ) diff --git a/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala b/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala index d4a1ee679f..a5989e9c34 100644 --- a/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala +++ b/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala @@ -96,7 +96,7 @@ object DumpChainApp extends App with NodeKeyBuilder with SecureRandomBuilder wit override val capabilities: List[Capability] = blockchainConfig.capabilities } - lazy val handshaker: Handshaker[PeerInfo] = EtcHandshaker(handshakerConfiguration) + lazy val handshaker: Capability => Handshaker[PeerInfo] = c => EtcHandshaker(handshakerConfiguration, c) val peerMessageBus = actorSystem.actorOf(PeerEventBusActor.props) @@ -116,7 +116,7 @@ object DumpChainApp extends App with NodeKeyBuilder with SecureRandomBuilder wit messageDecoder = EthereumMessageDecoder, discoveryConfig = discoveryConfig, blacklist = blacklist, - bestProtocolVersion = Capability.best(blockchainConfig.capabilities) + capabilities = blockchainConfig.capabilities ), "peer-manager" ) diff --git a/src/main/scala/io/iohk/ethereum/network/rlpx/FrameCodec.scala b/src/main/scala/io/iohk/ethereum/network/rlpx/FrameCodec.scala index c3f7ae25f3..61c98bc492 100644 --- a/src/main/scala/io/iohk/ethereum/network/rlpx/FrameCodec.scala +++ b/src/main/scala/io/iohk/ethereum/network/rlpx/FrameCodec.scala @@ -25,13 +25,15 @@ class FrameCodec(private val secrets: Secrets) { private val allZerosIV = Array.fill[Byte](16)(0) - private val enc: StreamCipher = { + //needs to be lazy to enable mocking + private lazy val enc: StreamCipher = { val cipher = new SICBlockCipher(new AESEngine) cipher.init(true, new ParametersWithIV(new KeyParameter(secrets.aes), allZerosIV)) cipher } - private val dec: StreamCipher = { + //needs to be lazy to enable mocking + private lazy val dec: StreamCipher = { val cipher = new SICBlockCipher(new AESEngine) cipher.init(false, new ParametersWithIV(new KeyParameter(secrets.aes), allZerosIV)) cipher diff --git a/src/main/scala/io/iohk/ethereum/network/rlpx/MessageCodec.scala b/src/main/scala/io/iohk/ethereum/network/rlpx/MessageCodec.scala index f20528826c..ebd67d14e9 100644 --- a/src/main/scala/io/iohk/ethereum/network/rlpx/MessageCodec.scala +++ b/src/main/scala/io/iohk/ethereum/network/rlpx/MessageCodec.scala @@ -29,7 +29,7 @@ class MessageCodec(frameCodec: FrameCodec, messageDecoder: MessageDecoder, proto frames map { frame => val frameData = frame.payload.toArray val payloadTry = - if (remotePeer2PeerVersion >= EtcHelloExchangeState.P2pVersion) { + if (remotePeer2PeerVersion >= EtcHelloExchangeState.P2pVersion && frame.`type` != Hello.code) { decompressData(frameData) } else { Success(frameData) diff --git a/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala b/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala index 24129393b1..cffcbf3e34 100644 --- a/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala +++ b/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala @@ -9,7 +9,7 @@ import cats.data.NonEmptyList import io.iohk.ethereum.network.p2p.messages.Capability import io.iohk.ethereum.network.p2p.messages.WireProtocol.Hello import io.iohk.ethereum.network.p2p.{Message, MessageDecoder, MessageSerializable, NetworkMessageDecoder} -import io.iohk.ethereum.network.rlpx.RLPxConnectionHandler.RLPxConfiguration +import io.iohk.ethereum.network.rlpx.RLPxConnectionHandler.{HelloExtractor, RLPxConfiguration} import io.iohk.ethereum.utils.ByteUtils import org.bouncycastle.util.encoders.Hex @@ -33,7 +33,8 @@ class RLPxConnectionHandler( capabilities: List[Capability], authHandshaker: AuthHandshaker, messageCodecFactory: (FrameCodec, MessageDecoder, Capability, Long) => MessageCodec, - rlpxConfiguration: RLPxConfiguration + rlpxConfiguration: RLPxConfiguration, + extractor: Secrets => HelloExtractor ) extends Actor with ActorLogging { @@ -152,17 +153,15 @@ class RLPxConnectionHandler( result match { case AuthHandshakeSuccess(secrets, remotePubKey) => log.info(s"Auth handshake succeeded for peer $peerId") - val frameCodec = new FrameCodec(secrets) - val frames = frameCodec.readFrames(remainingData) + val e = extractor(secrets) val messageCodecOpt = for { - frame <- frames.headOption - hello <- extractHello(frame) + r <- e.readHello(remainingData) + (hello, restFrames) = r protocolVersion <- Capability.negotiate(hello.capabilities.toList, capabilities) p2pVersion = hello.p2pVersion - messageCodec = messageCodecFactory(frameCodec, messageDecoder, protocolVersion, p2pVersion) + messageCodec = messageCodecFactory(e.frameCodec, messageDecoder, protocolVersion, p2pVersion) _ = context.parent ! ConnectionEstablished(remotePubKey, protocolVersion) _ = context.parent ! MessageReceived(hello) - restFrames = frames.drop(1) _ = { if (restFrames.nonEmpty) { val messagesSoFar = messageCodec.readFrames(restFrames) // omit hello @@ -185,17 +184,6 @@ class RLPxConnectionHandler( context stop self } - private def extractHello(frame: Frame): Option[Hello] = { - val frameData = frame.payload.toArray - if(frame.`type` == Hello.code) { - val m = NetworkMessageDecoder.fromBytes(frame.`type`, frameData, Capability.Capabilities.Eth63Capability) - Some(m.asInstanceOf[Hello]) - } else { - log.error("Unable to find 'Hello'") - None - } - } - def processMessage(messageTry: Try[Message]): Unit = messageTry match { case Success(message) => context.parent ! MessageReceived(message) @@ -322,7 +310,9 @@ object RLPxConnectionHandler { rlpxConfiguration: RLPxConfiguration ): Props = Props( - new RLPxConnectionHandler(messageDecoder, capabilities, authHandshaker, messageCodecFactory, rlpxConfiguration) + new RLPxConnectionHandler( + messageDecoder, capabilities, authHandshaker, messageCodecFactory, rlpxConfiguration, HelloExtractor.apply + ) ) def messageCodecFactory( @@ -358,4 +348,22 @@ object RLPxConnectionHandler { val waitForTcpAckTimeout: FiniteDuration } + case class HelloExtractor(secrets: Secrets) { + lazy val frameCodec = new FrameCodec(secrets) + + def readHello(remainingData: ByteString): Option[(Hello, Seq[Frame])] = { + val frames = frameCodec.readFrames(remainingData) + frames.headOption.flatMap(extractHello).map(h => (h, frames.drop(1))) + } + } + + private def extractHello(frame: Frame): Option[Hello] = { + val frameData = frame.payload.toArray + if(frame.`type` == Hello.code) { + val m = NetworkMessageDecoder.fromBytes(frame.`type`, frameData, Capability.Capabilities.Eth63Capability) + Some(m.asInstanceOf[Hello]) + } else { + None + } + } } diff --git a/src/test/scala/io/iohk/ethereum/network/PeerActorHandshakingSpec.scala b/src/test/scala/io/iohk/ethereum/network/PeerActorHandshakingSpec.scala index f2089fe92a..691887e12b 100644 --- a/src/test/scala/io/iohk/ethereum/network/PeerActorHandshakingSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/PeerActorHandshakingSpec.scala @@ -1,7 +1,6 @@ package io.iohk.ethereum.network import java.net.{InetSocketAddress, URI} - import akka.actor.{ActorSystem, Props} import akka.testkit.{TestActorRef, TestProbe} import akka.util.ByteString @@ -16,7 +15,7 @@ import io.iohk.ethereum.network.handshaker._ import io.iohk.ethereum.network.p2p.Message import io.iohk.ethereum.network.p2p.messages.Capability.Capabilities._ import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.Status -import io.iohk.ethereum.network.p2p.messages.ProtocolVersions +import io.iohk.ethereum.network.p2p.messages.{Capability, ProtocolVersions} import io.iohk.ethereum.network.p2p.messages.WireProtocol.{Disconnect, Hello, Pong} import io.iohk.ethereum.network.rlpx.RLPxConnectionHandler import io.iohk.ethereum.utils.Config @@ -31,12 +30,12 @@ class PeerActorHandshakingSpec extends AnyFlatSpec with Matchers { import DefaultValues._ val peerActorHandshakeSucceeds = - peerActor(MockHandshakerAlwaysSucceeds(defaultStatus, defaultBlockNumber, defaultForkAccepted)) + peerActor(_ => MockHandshakerAlwaysSucceeds(defaultStatus, defaultBlockNumber, defaultForkAccepted)) //Establish probe rlpxconnection peerActorHandshakeSucceeds ! ConnectTo(uri) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.ConnectTo(uri)) - rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString())) + rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString(), ProtocolVersions.PV63)) //Test that the handshake succeeded val sender = TestProbe()(system) @@ -48,12 +47,12 @@ class PeerActorHandshakingSpec extends AnyFlatSpec with Matchers { import DefaultValues._ - val peerActorHandshakeFails = peerActor(MockHandshakerAlwaysFails(defaultReasonDisconnect)) + val peerActorHandshakeFails = peerActor(_ => MockHandshakerAlwaysFails(defaultReasonDisconnect)) //Establish probe rlpxconnection peerActorHandshakeFails ! ConnectTo(uri) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.ConnectTo(uri)) - rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString())) + rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString(), ProtocolVersions.PV63)) //Test that the handshake failed rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.SendMessage(Disconnect(defaultReasonDisconnect))) @@ -64,12 +63,12 @@ class PeerActorHandshakingSpec extends AnyFlatSpec with Matchers { import DefaultValues._ - val peerActorHandshakeRequiresHello = peerActor(MockHandshakerRequiresHello()) + val peerActorHandshakeRequiresHello = peerActor(_ => MockHandshakerRequiresHello()) //Establish probe rlpxconnection peerActorHandshakeRequiresHello ! ConnectTo(uri) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.ConnectTo(uri)) - rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString())) + rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString(), ProtocolVersions.PV63)) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.SendMessage(defaultHello)) peerActorHandshakeRequiresHello ! RLPxConnectionHandler.MessageReceived(defaultHello) @@ -84,12 +83,12 @@ class PeerActorHandshakingSpec extends AnyFlatSpec with Matchers { import DefaultValues._ - val peerActorHandshakeRequiresHello = peerActor(MockHandshakerRequiresHello()) + val peerActorHandshakeRequiresHello = peerActor(_ => MockHandshakerRequiresHello()) //Establish probe rlpxconnection peerActorHandshakeRequiresHello ! ConnectTo(uri) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.ConnectTo(uri)) - rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString())) + rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString(), ProtocolVersions.PV63)) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.SendMessage(defaultHello)) time.advance(defaultTimeout * 2) @@ -102,12 +101,12 @@ class PeerActorHandshakingSpec extends AnyFlatSpec with Matchers { import DefaultValues._ - val peerActorHandshakeRequiresHello = peerActor(MockHandshakerRequiresHello()) + val peerActorHandshakeRequiresHello = peerActor(_ => MockHandshakerRequiresHello()) //Establish probe rlpxconnection peerActorHandshakeRequiresHello ! ConnectTo(uri) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.ConnectTo(uri)) - rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString())) + rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString(), ProtocolVersions.PV63)) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.SendMessage(defaultHello)) peerActorHandshakeRequiresHello ! RLPxConnectionHandler.MessageReceived(defaultStatusMsg) @@ -120,12 +119,12 @@ class PeerActorHandshakingSpec extends AnyFlatSpec with Matchers { import DefaultValues._ - val peerActorHandshakeRequiresHello = peerActor(MockHandshakerRequiresHello()) + val peerActorHandshakeRequiresHello = peerActor(_ => MockHandshakerRequiresHello()) //Establish probe rlpxconnection peerActorHandshakeRequiresHello ! ConnectTo(uri) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.ConnectTo(uri)) - rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString())) + rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString(), ProtocolVersions.PV63)) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.SendMessage(defaultHello)) peerActorHandshakeRequiresHello ! RLPxConnectionHandler.MessageReceived(Pong()) //Ignored @@ -151,7 +150,7 @@ class PeerActorHandshakingSpec extends AnyFlatSpec with Matchers { val peerMessageBus = TestProbe() val knownNodesManager = TestProbe() - def peerActor(handshaker: Handshaker[PeerInfo]): TestActorRef[PeerActor[PeerInfo]] = TestActorRef( + def peerActor(handshaker: Capability => Handshaker[PeerInfo]): TestActorRef[PeerActor[PeerInfo]] = TestActorRef( Props( new PeerActor( new InetSocketAddress("127.0.0.1", 0), diff --git a/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala b/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala index c33484da1d..e3aa1a418b 100644 --- a/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala @@ -98,9 +98,9 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { bestHash = firstBlock.header.hash ) - initHandshakerWithoutResolver.nextMessage.map(_.messageToSend) shouldBe Right(localHello: HelloEnc) + initHandshakerWithoutResolverETC64.nextMessage.map(_.messageToSend) shouldBe Right(localHello: HelloEnc) - val handshakerAfterHelloOpt = initHandshakerWithoutResolver.applyMessage(remoteHello) + val handshakerAfterHelloOpt = initHandshakerWithoutResolverETC64.applyMessage(remoteHello) assert(handshakerAfterHelloOpt.isDefined) handshakerAfterHelloOpt.get.nextMessage.map(_.messageToSend.underlyingMsg) shouldBe Right(newLocalStatusMsg) @@ -232,7 +232,7 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { it should "fail if the remote peer doesn't support PV63/PV64" in new RemotePeerPV63Setup { val pv62Capability = ProtocolVersions.PV62 val handshakerAfterHelloOpt = - initHandshakerWithResolver.applyMessage(remoteHello.copy(capabilities = Seq(pv62Capability))) + initHandshakerWithResolverETH62.applyMessage(remoteHello.copy(capabilities = Seq(pv62Capability))) assert(handshakerAfterHelloOpt.isDefined) handshakerAfterHelloOpt.get.nextMessage.leftSide shouldBe Left( HandshakeFailure(Disconnect.Reasons.IncompatibleP2pProtocolVersion) @@ -286,9 +286,15 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { } val initHandshakerWithoutResolver = EtcHandshaker( - new MockEtcHandshakerConfiguration(List(ProtocolVersions.PV64, ProtocolVersions.PV63)) + new MockEtcHandshakerConfiguration(List(ProtocolVersions.PV64, ProtocolVersions.PV63)), ProtocolVersions.PV63 + ) + val initHandshakerWithoutResolverETC64 = EtcHandshaker( + new MockEtcHandshakerConfiguration(List(ProtocolVersions.PV64, ProtocolVersions.PV63)), ProtocolVersions.PV64 ) - val initHandshakerWithResolver = EtcHandshaker(etcHandshakerConfigurationWithResolver) + + val initHandshakerWithResolver = EtcHandshaker(etcHandshakerConfigurationWithResolver, ProtocolVersions.PV63) + + val initHandshakerWithResolverETH62 = EtcHandshaker(etcHandshakerConfigurationWithResolver, ProtocolVersions.PV62) val firstBlock = genesisBlock.copy(header = genesisBlock.header.copy(parentHash = genesisBlock.header.hash, number = 1)) diff --git a/src/test/scala/io/iohk/ethereum/network/p2p/MessageCodecSpec.scala b/src/test/scala/io/iohk/ethereum/network/p2p/MessageCodecSpec.scala index b811e2cb77..57d5767e80 100644 --- a/src/test/scala/io/iohk/ethereum/network/p2p/MessageCodecSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/p2p/MessageCodecSpec.scala @@ -2,6 +2,7 @@ package io.iohk.ethereum.network.p2p import akka.util.ByteString import io.iohk.ethereum.network.handshaker.EtcHelloExchangeState +import io.iohk.ethereum.network.handshaker.EtcHelloExchangeState.P2pVersion import io.iohk.ethereum.network.p2p.messages.Capability.Capabilities._ import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.Status import io.iohk.ethereum.network.p2p.messages.ProtocolVersions @@ -26,6 +27,9 @@ class MessageCodecSpec extends AnyFlatSpec with Matchers { } it should "compress messages when remote side advertises p2p version larger or equal 5" in new TestSetup { + override lazy val negotiatedRemoteP2PVersion: Long = 5L + override lazy val negotiatedLocalP2PVersion: Long = 4L + val remoteHello = remoteMessageCodec.encodeMessage(helloV5) val localReceivedRemoteHello = messageCodec.readMessages(remoteHello) @@ -73,6 +77,8 @@ class MessageCodecSpec extends AnyFlatSpec with Matchers { trait TestSetup extends SecureChannelSetup { val frameCodec = new FrameCodec(secrets) val remoteFrameCodec = new FrameCodec(remoteSecrets) + lazy val negotiatedRemoteP2PVersion: Long = 5L + lazy val negotiatedLocalP2PVersion: Long = 5L val helloV5 = Hello( p2pVersion = EtcHelloExchangeState.P2pVersion, @@ -94,8 +100,8 @@ class MessageCodecSpec extends AnyFlatSpec with Matchers { val decoder = NetworkMessageDecoder orElse EthereumMessageDecoder - val messageCodec = new MessageCodec(frameCodec, decoder, ProtocolVersions.PV63) - val remoteMessageCodec = new MessageCodec(remoteFrameCodec, decoder, ProtocolVersions.PV63) + val messageCodec = new MessageCodec(frameCodec, decoder, ProtocolVersions.PV63, negotiatedLocalP2PVersion) + val remoteMessageCodec = new MessageCodec(remoteFrameCodec, decoder, ProtocolVersions.PV63, negotiatedRemoteP2PVersion) } diff --git a/src/test/scala/io/iohk/ethereum/network/p2p/PeerActorSpec.scala b/src/test/scala/io/iohk/ethereum/network/p2p/PeerActorSpec.scala index d25a71354c..5b5547f8c4 100644 --- a/src/test/scala/io/iohk/ethereum/network/p2p/PeerActorSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/p2p/PeerActorSpec.scala @@ -56,7 +56,7 @@ class PeerActorSpec peer ! PeerActor.ConnectTo(new URI("encode://localhost:9000")) rlpxConnection.expectMsgClass(classOf[RLPxConnectionHandler.ConnectTo]) - rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId)) + rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId, protocol)) rlpxConnection.expectMsgPF() { case RLPxConnectionHandler.SendMessage(hello: HelloEnc) => () @@ -108,7 +108,7 @@ class PeerActorSpec peer ! PeerActor.ConnectTo(new URI("encode://localhost:9000")) rlpxConnection.expectMsgClass(classOf[RLPxConnectionHandler.ConnectTo]) - rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId)) + rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId, protocol)) rlpxConnection.expectMsgPF() { case RLPxConnectionHandler.SendMessage(hello: HelloEnc) => () @@ -127,7 +127,7 @@ class PeerActorSpec peer ! PeerActor.ConnectTo(uri) rlpxConnection.expectMsgClass(classOf[RLPxConnectionHandler.ConnectTo]) - rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId)) + rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId, protocol)) //Hello exchange val remoteHello = Hello(4, "test-client", Seq(Eth63Capability), 9000, ByteString("unused")) @@ -165,7 +165,7 @@ class PeerActorSpec peer ! PeerActor.ConnectTo(uri) rlpxConnection.expectMsgClass(classOf[RLPxConnectionHandler.ConnectTo]) - rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId)) + rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId, protocol)) //Hello exchange val remoteHello = Hello(4, "test-client", Seq(Eth63Capability), 9000, ByteString("unused")) @@ -195,7 +195,7 @@ class PeerActorSpec peer ! PeerActor.ConnectTo(uri) rlpxConnection.expectMsgClass(classOf[RLPxConnectionHandler.ConnectTo]) - rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId)) + rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId, protocol)) //Hello exchange val remoteHello = Hello(4, "test-client", Seq(Etc64Capability, Eth63Capability), 9000, ByteString("unused")) @@ -234,7 +234,7 @@ class PeerActorSpec peer ! PeerActor.ConnectTo(uri) rlpxConnection.expectMsgClass(classOf[RLPxConnectionHandler.ConnectTo]) - rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId)) + rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId, protocol)) //Hello exchange val remoteHello = Hello(4, "test-client", Seq(Eth63Capability), 9000, ByteString("unused")) @@ -269,7 +269,7 @@ class PeerActorSpec peer ! PeerActor.ConnectTo(new URI("encode://localhost:9000")) rlpxConnection.expectMsgClass(classOf[RLPxConnectionHandler.ConnectTo]) - rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId)) + rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId, protocol)) val remoteHello = Hello(4, "test-client", Seq(Eth63Capability), 9000, ByteString("unused")) rlpxConnection.expectMsgPF() { case RLPxConnectionHandler.SendMessage(_: HelloEnc) => () } @@ -304,7 +304,7 @@ class PeerActorSpec peer ! PeerActor.ConnectTo(new URI("encode://localhost:9000")) rlpxConnection.expectMsgClass(classOf[RLPxConnectionHandler.ConnectTo]) - rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId)) + rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId, protocol)) val remoteHello = Hello(4, "test-client", Seq(Eth63Capability), 9000, ByteString("unused")) rlpxConnection.expectMsgPF() { case RLPxConnectionHandler.SendMessage(_: HelloEnc) => () } @@ -333,7 +333,7 @@ class PeerActorSpec peer ! PeerActor.HandleConnection(connection.ref, new InetSocketAddress("localhost", 9000)) rlpxConnection.expectMsgClass(classOf[RLPxConnectionHandler.HandleConnection]) - rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId)) + rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId, protocol)) rlpxConnection.expectMsgPF() { case RLPxConnectionHandler.SendMessage(_: HelloEnc) => () } time.advance(5.seconds) rlpxConnection.expectMsg( @@ -351,7 +351,7 @@ class PeerActorSpec peer ! PeerActor.ConnectTo(new URI("encode://localhost:9000")) rlpxConnection.expectMsgClass(classOf[RLPxConnectionHandler.ConnectTo]) - rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId)) + rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId, protocol)) val remoteHello = Hello(4, "test-client", Seq(Eth63Capability), 9000, ByteString("unused")) rlpxConnection.expectMsgPF() { case RLPxConnectionHandler.SendMessage(_: HelloEnc) => () } @@ -383,7 +383,7 @@ class PeerActorSpec peer ! PeerActor.DisconnectPeer(Disconnect.Reasons.TooManyPeers) rlpxConnection.expectMsgClass(classOf[RLPxConnectionHandler.ConnectTo]) - rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId)) + rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId, protocol)) val remoteHello = Hello(4, "test-client", Seq(Eth63Capability), 9000, ByteString("unused")) rlpxConnection.expectMsgPF() { case RLPxConnectionHandler.SendMessage(_: HelloEnc) => () } @@ -427,7 +427,7 @@ class PeerActorSpec knownNodesManager.ref, false, None, - Mocks.MockHandshakerAlwaysSucceeds(remoteStatus, 0, false) + _ => Mocks.MockHandshakerAlwaysSucceeds(remoteStatus, 0, false) ) ) ) @@ -435,7 +435,7 @@ class PeerActorSpec peerActor ! PeerActor.ConnectTo(new URI("encode://localhost:9000")) rlpxConnection.expectMsgClass(classOf[RLPxConnectionHandler.ConnectTo]) - rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId)) + rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId, protocol)) rlpxConnection.send(peerActor, RLPxConnectionHandler.MessageReceived(Ping())) rlpxConnection.expectMsgPF() { case RLPxConnectionHandler.SendMessage(_: PongEnc) => () } @@ -445,7 +445,7 @@ class PeerActorSpec peer ! PeerActor.ConnectTo(new URI("encode://localhost:9000")) rlpxConnection.expectMsgClass(classOf[RLPxConnectionHandler.ConnectTo]) - rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId)) + rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId, protocol)) val remoteHello = Hello(4, "test-client", Seq(Eth63Capability), 9000, ByteString("unused")) rlpxConnection.expectMsgPF() { case RLPxConnectionHandler.SendMessage(_: HelloEnc) => () } @@ -567,7 +567,7 @@ class PeerActorSpec override val capabilities: List[Capability] = List(protocol) } - val handshaker = EtcHandshaker(handshakerConfiguration) + val handshaker = (c: Capability) => EtcHandshaker(handshakerConfiguration, c) } trait TestSetup extends NodeStatusSetup with BlockUtils with HandshakerSetup { diff --git a/src/test/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandlerSpec.scala b/src/test/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandlerSpec.scala index b44934944d..479f4d41ec 100644 --- a/src/test/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandlerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandlerSpec.scala @@ -5,11 +5,12 @@ import akka.actor.{ActorRef, ActorSystem, Props} import akka.io.Tcp import akka.testkit.{TestActorRef, TestKit, TestProbe} import akka.util.ByteString +import io.iohk.ethereum.network.p2p.messages.Capability.Capabilities import io.iohk.ethereum.{Timeouts, WithActorSystemShutDown} import io.iohk.ethereum.network.p2p.{MessageDecoder, MessageSerializable} import io.iohk.ethereum.network.p2p.messages.{Capability, ProtocolVersions} -import io.iohk.ethereum.network.p2p.messages.WireProtocol.Ping -import io.iohk.ethereum.network.rlpx.RLPxConnectionHandler.RLPxConfiguration +import io.iohk.ethereum.network.p2p.messages.WireProtocol.{Hello, Ping} +import io.iohk.ethereum.network.rlpx.RLPxConnectionHandler.{HelloExtractor, MessageReceived, RLPxConfiguration} import io.iohk.ethereum.security.SecureRandomBuilder import org.scalamock.scalatest.MockFactory @@ -96,6 +97,9 @@ class RLPxConnectionHandlerSpec rlpxConnection ! RLPxConnectionHandler.SendMessage(Ping()) connection.expectMsg(Tcp.Write(ByteString("ping encoded"), RLPxConnectionHandler.Ack)) + val expectedHello = rlpxConnectionParent.expectMsgType[MessageReceived] + expectedHello.message shouldBe a[Hello] + //The rlpx connection is closed after a timeout happens (after rlpxConfiguration.waitForTcpAckTimeout) and it is processed rlpxConnectionParent.expectTerminated( rlpxConnection, @@ -179,6 +183,7 @@ class RLPxConnectionHandlerSpec val mockHandshaker = mock[AuthHandshaker] val connection = TestProbe() val mockMessageCodec = mock[MessageCodec] + val mockHelloExtractor: HelloExtractor = mock[HelloExtractor] val uri = new URI( "enode://18a551bee469c2e02de660ab01dede06503c986f6b8520cb5a65ad122df88b17b285e3fef09a40a0d44f99e014f8616cf1ebc2e094f96c6e09e2f390f5d34857@47.90.36.129:30303" @@ -198,10 +203,11 @@ class RLPxConnectionHandlerSpec Props( new RLPxConnectionHandler( mockMessageDecoder, - protocolVersion, + protocolVersion:: Nil, mockHandshaker, - (_, _, _) => mockMessageCodec, - rlpxConfiguration + (_, _, _, _) => mockMessageCodec, + rlpxConfiguration, + _ => mockHelloExtractor ) { override def tcpActor: ActorRef = tcpActorProbe.ref } @@ -222,6 +228,9 @@ class RLPxConnectionHandlerSpec (mockHandshaker.handleInitialMessage _) .expects(data) .returning((response, AuthHandshakeSuccess(mock[Secrets], ByteString()))) + (mockHelloExtractor.readHello _) + .expects(ByteString.empty) + .returning(Some(Hello(5, "", Capabilities.Eth63Capability::Nil, 30303, ByteString("abc")), Seq.empty)) (mockMessageCodec.readMessages _) .expects(ByteString.empty) .returning(Nil) //For processing of messages after handshaking finishes From 61cf9812e75f910be727203f72ec5759f9b3dbe6 Mon Sep 17 00:00:00 2001 From: Dominik Zajkowski Date: Wed, 9 Jun 2021 16:00:06 +0200 Subject: [PATCH 09/15] [ETCM-841] Rename PV6X elements to ETX6X scheme --- .../txExecTest/util/DumpChainActor.scala | 8 +- .../txExecTest/util/FixtureProvider.scala | 2 +- .../blockchain/sync/BlockchainHostActor.scala | 6 +- .../blockchain/sync/PeersClient.scala | 4 +- .../sync/fast/DownloaderState.scala | 2 +- .../blockchain/sync/fast/FastSync.scala | 4 +- .../fast/FastSyncBranchResolverActor.scala | 2 +- .../sync/fast/PivotBlockSelector.scala | 2 +- .../sync/fast/SyncStateScheduler.scala | 2 +- .../sync/fast/SyncStateSchedulerActor.scala | 2 +- .../sync/regular/BlockBroadcast.scala | 10 +- .../sync/regular/BlockFetcher.scala | 8 +- .../sync/regular/BlockFetcherState.scala | 2 +- .../sync/regular/BodiesFetcher.scala | 2 +- .../sync/regular/HeadersFetcher.scala | 2 +- .../sync/regular/StateNodeFetcher.scala | 2 +- .../ethereum/db/storage/StateStorage.scala | 2 +- .../io/iohk/ethereum/domain/Account.scala | 2 +- .../io/iohk/ethereum/domain/Receipt.scala | 2 +- .../iohk/ethereum/jsonrpc/TestService.scala | 2 +- .../network/EtcPeerManagerActor.scala | 10 +- .../EtcForkBlockExchangeState.scala | 2 +- .../handshaker/EtcHelloExchangeState.scala | 6 +- .../EtcNodeStatus63ExchangeState.scala | 2 +- .../EtcNodeStatus64ExchangeState.scala | 10 +- .../network/p2p/MessageDecoders.scala | 46 +++--- .../network/p2p/messages/Capability.scala | 6 +- .../p2p/messages/{PV64.scala => ETC64.scala} | 10 +- .../p2p/messages/{PV61.scala => ETH61.scala} | 2 +- .../p2p/messages/{PV62.scala => ETH62.scala} | 2 +- .../p2p/messages/{PV63.scala => ETH63.scala} | 2 +- .../network/p2p/messages/package.scala | 10 +- .../io/iohk/ethereum/ObjectGenerators.scala | 6 +- .../blockchain/sync/BlockBroadcastSpec.scala | 20 +-- .../sync/BlockchainHostActorSpec.scala | 6 +- .../blockchain/sync/EtcPeerManagerFake.scala | 4 +- .../blockchain/sync/PeersClientSpec.scala | 2 +- .../sync/PivotBlockSelectorSpec.scala | 4 +- .../blockchain/sync/StateSyncSpec.scala | 6 +- .../blockchain/sync/SyncControllerSpec.scala | 12 +- .../sync/SyncStateDownloaderStateSpec.scala | 2 +- .../blockchain/sync/TestSyncPeers.scala | 2 +- .../FastSyncBranchResolverActorSpec.scala | 4 +- .../sync/regular/BlockFetcherSpec.scala | 2 +- .../sync/regular/RegularSyncFixtures.scala | 8 +- .../sync/regular/RegularSyncSpec.scala | 16 +- .../db/storage/NodeStorageSuite.scala | 2 +- .../ethereum/jsonrpc/DebugServiceSpec.scala | 2 +- .../jsonrpc/JsonRpcControllerSpec.scala | 2 +- .../ethereum/network/EtcPeerManagerSpec.scala | 36 ++--- .../network/PeerActorHandshakingSpec.scala | 14 +- .../network/PeerEventBusActorSpec.scala | 2 +- .../ethereum/network/PeerManagerSpec.scala | 2 +- .../ethereum/network/PeerStatisticsSpec.scala | 2 +- .../handshaker/EtcHandshakerSpec.scala | 70 ++++----- .../network/p2p/MessageCodecSpec.scala | 6 +- .../network/p2p/MessageDecodersSpec.scala | 144 +++++++++--------- .../ethereum/network/p2p/PeerActorSpec.scala | 36 ++--- .../messages/MessagesSerializationSpec.scala | 46 +++--- .../network/p2p/messages/NewBlockSpec.scala | 6 +- .../network/p2p/messages/NodeDataSpec.scala | 8 +- .../network/p2p/messages/ReceiptsSpec.scala | 6 +- .../rlpx/RLPxConnectionHandlerSpec.scala | 2 +- 63 files changed, 328 insertions(+), 328 deletions(-) rename src/main/scala/io/iohk/ethereum/network/p2p/messages/{PV64.scala => ETC64.scala} (95%) rename src/main/scala/io/iohk/ethereum/network/p2p/messages/{PV61.scala => ETH61.scala} (99%) rename src/main/scala/io/iohk/ethereum/network/p2p/messages/{PV62.scala => ETH62.scala} (99%) rename src/main/scala/io/iohk/ethereum/network/p2p/messages/{PV63.scala => ETH63.scala} (99%) diff --git a/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainActor.scala b/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainActor.scala index 2f6142bb2a..f4f29a981e 100644 --- a/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainActor.scala +++ b/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainActor.scala @@ -15,10 +15,10 @@ import io.iohk.ethereum.network.PeerEventBusActor.SubscriptionClassifier.Message import io.iohk.ethereum.network.PeerEventBusActor.{PeerSelector, Subscribe} import io.iohk.ethereum.network.PeerManagerActor.{GetPeers, Peers} import io.iohk.ethereum.network.p2p.messages.Codes -import io.iohk.ethereum.network.p2p.messages.PV62._ -import io.iohk.ethereum.network.p2p.messages.PV63.MptNodeEncoders._ -import io.iohk.ethereum.network.p2p.messages.PV63.ReceiptImplicits._ -import io.iohk.ethereum.network.p2p.messages.PV63._ +import io.iohk.ethereum.network.p2p.messages.ETH62._ +import io.iohk.ethereum.network.p2p.messages.ETH63.MptNodeEncoders._ +import io.iohk.ethereum.network.p2p.messages.ETH63.ReceiptImplicits._ +import io.iohk.ethereum.network.p2p.messages.ETH63._ import io.iohk.ethereum.network.{Peer, PeerManagerActor} import io.iohk.ethereum.txExecTest.util.DumpChainActor._ import org.bouncycastle.util.encoders.Hex diff --git a/src/it/scala/io/iohk/ethereum/txExecTest/util/FixtureProvider.scala b/src/it/scala/io/iohk/ethereum/txExecTest/util/FixtureProvider.scala index 07734114d2..7a4dfc5604 100644 --- a/src/it/scala/io/iohk/ethereum/txExecTest/util/FixtureProvider.scala +++ b/src/it/scala/io/iohk/ethereum/txExecTest/util/FixtureProvider.scala @@ -7,7 +7,7 @@ import io.iohk.ethereum.db.storage._ import io.iohk.ethereum.domain._ import io.iohk.ethereum.domain.BlockHeaderImplicits._ import io.iohk.ethereum.domain.BlockBody._ -import io.iohk.ethereum.network.p2p.messages.PV63._ +import io.iohk.ethereum.network.p2p.messages.ETH63._ import MptNodeEncoders._ import ReceiptImplicits._ import io.iohk.ethereum.db.cache.{AppCaches, LruCache} diff --git a/src/main/scala/io/iohk/ethereum/blockchain/sync/BlockchainHostActor.scala b/src/main/scala/io/iohk/ethereum/blockchain/sync/BlockchainHostActor.scala index eafb592d11..c6d891a3f5 100644 --- a/src/main/scala/io/iohk/ethereum/blockchain/sync/BlockchainHostActor.scala +++ b/src/main/scala/io/iohk/ethereum/blockchain/sync/BlockchainHostActor.scala @@ -8,9 +8,9 @@ import io.iohk.ethereum.network.PeerEventBusActor.SubscriptionClassifier.Message import io.iohk.ethereum.network.PeerEventBusActor.{PeerSelector, Subscribe} import io.iohk.ethereum.network.PeerManagerActor.PeerConfiguration import io.iohk.ethereum.network.p2p.{Message, MessageSerializable} -import io.iohk.ethereum.network.p2p.messages.PV62.{BlockBodies, BlockHeaders, GetBlockBodies, GetBlockHeaders} -import io.iohk.ethereum.network.p2p.messages.PV63.{GetNodeData, GetReceipts, NodeData, Receipts} -import io.iohk.ethereum.network.p2p.messages.PV63.MptNodeEncoders._ +import io.iohk.ethereum.network.p2p.messages.ETH62.{BlockBodies, BlockHeaders, GetBlockBodies, GetBlockHeaders} +import io.iohk.ethereum.network.p2p.messages.ETH63.{GetNodeData, GetReceipts, NodeData, Receipts} +import io.iohk.ethereum.network.p2p.messages.ETH63.MptNodeEncoders._ import io.iohk.ethereum.network.EtcPeerManagerActor import io.iohk.ethereum.network.p2p.messages.Codes diff --git a/src/main/scala/io/iohk/ethereum/blockchain/sync/PeersClient.scala b/src/main/scala/io/iohk/ethereum/blockchain/sync/PeersClient.scala index 41d7f2f29f..3cabe99eda 100644 --- a/src/main/scala/io/iohk/ethereum/blockchain/sync/PeersClient.scala +++ b/src/main/scala/io/iohk/ethereum/blockchain/sync/PeersClient.scala @@ -6,8 +6,8 @@ import io.iohk.ethereum.blockchain.sync.PeerListSupportNg.PeerWithInfo import io.iohk.ethereum.network.EtcPeerManagerActor.PeerInfo import io.iohk.ethereum.network.p2p.messages.Codes import io.iohk.ethereum.network.{Peer, PeerId} -import io.iohk.ethereum.network.p2p.messages.PV62._ -import io.iohk.ethereum.network.p2p.messages.PV63.{GetNodeData, NodeData} +import io.iohk.ethereum.network.p2p.messages.ETH62._ +import io.iohk.ethereum.network.p2p.messages.ETH63.{GetNodeData, NodeData} import io.iohk.ethereum.network.p2p.{Message, MessageSerializable} import io.iohk.ethereum.utils.Config.SyncConfig diff --git a/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/DownloaderState.scala b/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/DownloaderState.scala index de44369166..69debd0974 100644 --- a/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/DownloaderState.scala +++ b/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/DownloaderState.scala @@ -11,7 +11,7 @@ import io.iohk.ethereum.blockchain.sync.fast.SyncStateSchedulerActor.{ UsefulData } import io.iohk.ethereum.crypto.kec256 -import io.iohk.ethereum.network.p2p.messages.PV63.NodeData +import io.iohk.ethereum.network.p2p.messages.ETH63.NodeData import io.iohk.ethereum.network.{Peer, PeerId} import scala.annotation.tailrec diff --git a/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/FastSync.scala b/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/FastSync.scala index 5cc8f283d2..31aecf20ea 100644 --- a/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/FastSync.scala +++ b/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/FastSync.scala @@ -26,8 +26,8 @@ import io.iohk.ethereum.mpt.MerklePatriciaTrie import io.iohk.ethereum.network.EtcPeerManagerActor.PeerInfo import io.iohk.ethereum.network.Peer import io.iohk.ethereum.network.p2p.messages.Codes -import io.iohk.ethereum.network.p2p.messages.PV62._ -import io.iohk.ethereum.network.p2p.messages.PV63._ +import io.iohk.ethereum.network.p2p.messages.ETH62._ +import io.iohk.ethereum.network.p2p.messages.ETH63._ import io.iohk.ethereum.utils.ByteStringUtils import io.iohk.ethereum.utils.Config.SyncConfig import org.bouncycastle.util.encoders.Hex diff --git a/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/FastSyncBranchResolverActor.scala b/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/FastSyncBranchResolverActor.scala index 2d07e7b667..3f5b16717e 100644 --- a/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/FastSyncBranchResolverActor.scala +++ b/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/FastSyncBranchResolverActor.scala @@ -10,7 +10,7 @@ import io.iohk.ethereum.db.storage.AppStateStorage import io.iohk.ethereum.domain.{BlockHeader, Blockchain} import io.iohk.ethereum.network.Peer import io.iohk.ethereum.network.p2p.messages.Codes -import io.iohk.ethereum.network.p2p.messages.PV62.{BlockHeaders, GetBlockHeaders} +import io.iohk.ethereum.network.p2p.messages.ETH62.{BlockHeaders, GetBlockHeaders} import io.iohk.ethereum.utils.Config.SyncConfig import scala.concurrent.ExecutionContext.Implicits.global diff --git a/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/PivotBlockSelector.scala b/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/PivotBlockSelector.scala index 1afad53c46..cb2a1c3ee7 100644 --- a/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/PivotBlockSelector.scala +++ b/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/PivotBlockSelector.scala @@ -14,7 +14,7 @@ import io.iohk.ethereum.network.PeerEventBusActor.PeerEvent.MessageFromPeer import io.iohk.ethereum.network.PeerEventBusActor.SubscriptionClassifier.MessageClassifier import io.iohk.ethereum.network.PeerEventBusActor.{PeerSelector, Subscribe, Unsubscribe} import io.iohk.ethereum.network.p2p.messages.Codes -import io.iohk.ethereum.network.p2p.messages.PV62.{BlockHeaders, GetBlockHeaders} +import io.iohk.ethereum.network.p2p.messages.ETH62.{BlockHeaders, GetBlockHeaders} import io.iohk.ethereum.network.{EtcPeerManagerActor, Peer, PeerId} import io.iohk.ethereum.utils.Config.SyncConfig diff --git a/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/SyncStateScheduler.scala b/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/SyncStateScheduler.scala index 3ac599d431..90e6e8b4ab 100644 --- a/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/SyncStateScheduler.scala +++ b/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/SyncStateScheduler.scala @@ -8,7 +8,7 @@ import io.iohk.ethereum.blockchain.sync.fast.LoadableBloomFilter.BloomFilterLoad import io.iohk.ethereum.blockchain.sync.fast.SyncStateScheduler._ import io.iohk.ethereum.domain.{Account, Blockchain} import io.iohk.ethereum.mpt.{BranchNode, ExtensionNode, HashNode, LeafNode, MerklePatriciaTrie, MptNode} -import io.iohk.ethereum.network.p2p.messages.PV63.MptNodeEncoders.MptNodeDec +import io.iohk.ethereum.network.p2p.messages.ETH63.MptNodeEncoders.MptNodeDec import io.vavr.collection.PriorityQueue import monix.eval.Task diff --git a/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/SyncStateSchedulerActor.scala b/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/SyncStateSchedulerActor.scala index 1f7b1488b5..00947ff514 100644 --- a/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/SyncStateSchedulerActor.scala +++ b/src/main/scala/io/iohk/ethereum/blockchain/sync/fast/SyncStateSchedulerActor.scala @@ -19,7 +19,7 @@ import io.iohk.ethereum.blockchain.sync.fast.SyncStateSchedulerActor._ import io.iohk.ethereum.blockchain.sync.{Blacklist, PeerListSupportNg, PeerRequestHandler} import io.iohk.ethereum.network.Peer import io.iohk.ethereum.network.p2p.messages.Codes -import io.iohk.ethereum.network.p2p.messages.PV63.{GetNodeData, NodeData} +import io.iohk.ethereum.network.p2p.messages.ETH63.{GetNodeData, NodeData} import io.iohk.ethereum.utils.ByteStringUtils import io.iohk.ethereum.utils.Config.SyncConfig import monix.eval.Task diff --git a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockBroadcast.scala b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockBroadcast.scala index 3d08225c46..1159eb7e94 100644 --- a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockBroadcast.scala +++ b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockBroadcast.scala @@ -6,8 +6,8 @@ import io.iohk.ethereum.blockchain.sync.regular.BlockBroadcast.BlockToBroadcast import io.iohk.ethereum.domain.{Block, ChainWeight} import io.iohk.ethereum.network.EtcPeerManagerActor.PeerInfo import io.iohk.ethereum.network.p2p.MessageSerializable -import io.iohk.ethereum.network.p2p.messages.PV62.BlockHash -import io.iohk.ethereum.network.p2p.messages.{BaseETH6XMessages, PV62, PV64, ProtocolVersions} +import io.iohk.ethereum.network.p2p.messages.ETH62.BlockHash +import io.iohk.ethereum.network.p2p.messages.{BaseETH6XMessages, ETH62, ETC64, ProtocolVersions} import io.iohk.ethereum.network.{EtcPeerManagerActor, Peer, PeerId} import scala.util.Random @@ -40,7 +40,7 @@ class BlockBroadcast(val etcPeerManager: ActorRef) { private def broadcastNewBlock(blockToBroadcast: BlockToBroadcast, peers: Map[PeerId, PeerWithInfo]): Unit = obtainRandomPeerSubset(peers.values.map(_.peer).toSet).foreach { peer => val message: MessageSerializable = - if (peers(peer.id).peerInfo.remoteStatus.protocolVersion.toByte == ProtocolVersions.PV64.version) + if (peers(peer.id).peerInfo.remoteStatus.protocolVersion.toByte == ProtocolVersions.ETC64.version) blockToBroadcast.as64 else blockToBroadcast.as63 etcPeerManager ! EtcPeerManagerActor.SendMessage(message, peer.id) @@ -49,7 +49,7 @@ class BlockBroadcast(val etcPeerManager: ActorRef) { private def broadcastNewBlockHash(blockToBroadcast: BlockToBroadcast, peers: Set[Peer]): Unit = peers.foreach { peer => val newBlockHeader = blockToBroadcast.block.header - val newBlockHashMsg = PV62.NewBlockHashes(Seq(BlockHash(newBlockHeader.hash, newBlockHeader.number))) + val newBlockHashMsg = ETH62.NewBlockHashes(Seq(BlockHash(newBlockHeader.hash, newBlockHeader.number))) etcPeerManager ! EtcPeerManagerActor.SendMessage(newBlockHashMsg, peer.id) } @@ -74,6 +74,6 @@ object BlockBroadcast { */ case class BlockToBroadcast(block: Block, chainWeight: ChainWeight) { def as63: BaseETH6XMessages.NewBlock = BaseETH6XMessages.NewBlock(block, chainWeight.totalDifficulty) - def as64: PV64.NewBlock = PV64.NewBlock(block, chainWeight) + def as64: ETC64.NewBlock = ETC64.NewBlock(block, chainWeight) } } diff --git a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockFetcher.scala b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockFetcher.scala index 734c5f3074..47be28645e 100644 --- a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockFetcher.scala +++ b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockFetcher.scala @@ -23,9 +23,9 @@ import io.iohk.ethereum.network.PeerEventBusActor.SubscriptionClassifier.Message import io.iohk.ethereum.network.PeerEventBusActor.{PeerSelector, Subscribe} import io.iohk.ethereum.network.{Peer, PeerEventBusActor, PeerId} import io.iohk.ethereum.network.p2p.Message -import io.iohk.ethereum.network.p2p.messages.{Codes, BaseETH6XMessages, PV64} -import io.iohk.ethereum.network.p2p.messages.PV62._ -import io.iohk.ethereum.network.p2p.messages.PV63.NodeData +import io.iohk.ethereum.network.p2p.messages.{Codes, BaseETH6XMessages, ETC64} +import io.iohk.ethereum.network.p2p.messages.ETH62._ +import io.iohk.ethereum.network.p2p.messages.ETH63.NodeData import io.iohk.ethereum.utils.ByteStringUtils import io.iohk.ethereum.utils.Config.SyncConfig import io.iohk.ethereum.utils.FunctorOps._ @@ -217,7 +217,7 @@ class BlockFetcher( case AdaptedMessageFromEventBus(BaseETH6XMessages.NewBlock(block, _), peerId) => handleNewBlock(block, peerId, state) - case AdaptedMessageFromEventBus(PV64.NewBlock(block, _), peerId) => + case AdaptedMessageFromEventBus(ETC64.NewBlock(block, _), peerId) => handleNewBlock(block, peerId, state) case BlockImportFailed(blockNr, reason) => diff --git a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockFetcherState.scala b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockFetcherState.scala index 7599f9964c..127d67e4b2 100644 --- a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockFetcherState.scala +++ b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BlockFetcherState.scala @@ -9,7 +9,7 @@ import io.iohk.ethereum.blockchain.sync.regular.BlockFetcherState._ import io.iohk.ethereum.consensus.validators.BlockValidator import io.iohk.ethereum.domain.{Block, BlockBody, BlockHeader, HeadersSeq} import io.iohk.ethereum.network.PeerId -import io.iohk.ethereum.network.p2p.messages.PV62.BlockHash +import io.iohk.ethereum.network.p2p.messages.ETH62.BlockHash import scala.annotation.tailrec import scala.collection.immutable.Queue diff --git a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BodiesFetcher.scala b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BodiesFetcher.scala index 028ba8a177..c56e53b8b9 100644 --- a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BodiesFetcher.scala +++ b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/BodiesFetcher.scala @@ -9,7 +9,7 @@ import io.iohk.ethereum.blockchain.sync.regular.BlockFetcher.FetchCommand import io.iohk.ethereum.blockchain.sync.regular.BodiesFetcher.BodiesFetcherCommand import io.iohk.ethereum.network.Peer import io.iohk.ethereum.network.p2p.Message -import io.iohk.ethereum.network.p2p.messages.PV62.{BlockBodies, GetBlockBodies} +import io.iohk.ethereum.network.p2p.messages.ETH62.{BlockBodies, GetBlockBodies} import io.iohk.ethereum.utils.Config.SyncConfig import monix.execution.Scheduler diff --git a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/HeadersFetcher.scala b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/HeadersFetcher.scala index 2a5549ebc3..6b12072be2 100644 --- a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/HeadersFetcher.scala +++ b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/HeadersFetcher.scala @@ -7,7 +7,7 @@ import io.iohk.ethereum.blockchain.sync.regular.BlockFetcher.FetchCommand import io.iohk.ethereum.blockchain.sync.regular.HeadersFetcher.HeadersFetcherCommand import io.iohk.ethereum.network.Peer import io.iohk.ethereum.network.p2p.Message -import io.iohk.ethereum.network.p2p.messages.PV62.{BlockHeaders, GetBlockHeaders} +import io.iohk.ethereum.network.p2p.messages.ETH62.{BlockHeaders, GetBlockHeaders} import io.iohk.ethereum.utils.Config.SyncConfig import monix.eval.Task import monix.execution.Scheduler diff --git a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/StateNodeFetcher.scala b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/StateNodeFetcher.scala index 0d7a18af1d..78186cad94 100644 --- a/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/StateNodeFetcher.scala +++ b/src/main/scala/io/iohk/ethereum/blockchain/sync/regular/StateNodeFetcher.scala @@ -9,7 +9,7 @@ import io.iohk.ethereum.blockchain.sync.regular.BlockFetcher.{FetchCommand, Fetc import io.iohk.ethereum.crypto.kec256 import io.iohk.ethereum.network.Peer import io.iohk.ethereum.network.p2p.Message -import io.iohk.ethereum.network.p2p.messages.PV63.{GetNodeData, NodeData} +import io.iohk.ethereum.network.p2p.messages.ETH63.{GetNodeData, NodeData} import io.iohk.ethereum.utils.Config.SyncConfig import cats.syntax.either._ import io.iohk.ethereum.blockchain.sync.Blacklist.BlacklistReason diff --git a/src/main/scala/io/iohk/ethereum/db/storage/StateStorage.scala b/src/main/scala/io/iohk/ethereum/db/storage/StateStorage.scala index 2f550b8d82..7cbb3b0f13 100644 --- a/src/main/scala/io/iohk/ethereum/db/storage/StateStorage.scala +++ b/src/main/scala/io/iohk/ethereum/db/storage/StateStorage.scala @@ -8,7 +8,7 @@ import io.iohk.ethereum.db.storage.NodeStorage.{NodeEncoded, NodeHash} import io.iohk.ethereum.db.storage.StateStorage.{FlushSituation, GenesisDataLoad} import io.iohk.ethereum.db.storage.pruning.{ArchivePruning, PruningMode} import io.iohk.ethereum.mpt.MptNode -import io.iohk.ethereum.network.p2p.messages.PV63.MptNodeEncoders._ +import io.iohk.ethereum.network.p2p.messages.ETH63.MptNodeEncoders._ import io.iohk.ethereum.utils.Config.NodeCacheConfig import scala.concurrent.duration.FiniteDuration diff --git a/src/main/scala/io/iohk/ethereum/domain/Account.scala b/src/main/scala/io/iohk/ethereum/domain/Account.scala index 455cec90d1..2e479a038f 100644 --- a/src/main/scala/io/iohk/ethereum/domain/Account.scala +++ b/src/main/scala/io/iohk/ethereum/domain/Account.scala @@ -3,7 +3,7 @@ package io.iohk.ethereum.domain import akka.util.ByteString import io.iohk.ethereum.crypto.kec256 import io.iohk.ethereum.mpt.ByteArraySerializable -import io.iohk.ethereum.network.p2p.messages.PV63.AccountImplicits +import io.iohk.ethereum.network.p2p.messages.ETH63.AccountImplicits import io.iohk.ethereum.rlp import io.iohk.ethereum.rlp.RLPImplicits._ import org.bouncycastle.util.encoders.Hex diff --git a/src/main/scala/io/iohk/ethereum/domain/Receipt.scala b/src/main/scala/io/iohk/ethereum/domain/Receipt.scala index 956d583fef..0027b16b8a 100644 --- a/src/main/scala/io/iohk/ethereum/domain/Receipt.scala +++ b/src/main/scala/io/iohk/ethereum/domain/Receipt.scala @@ -7,7 +7,7 @@ import org.bouncycastle.util.encoders.Hex object Receipt { val byteArraySerializable: ByteArraySerializable[Receipt] = new ByteArraySerializable[Receipt] { - import io.iohk.ethereum.network.p2p.messages.PV63.ReceiptImplicits._ + import io.iohk.ethereum.network.p2p.messages.ETH63.ReceiptImplicits._ override def fromBytes(bytes: Array[Byte]): Receipt = bytes.toReceipt diff --git a/src/main/scala/io/iohk/ethereum/jsonrpc/TestService.scala b/src/main/scala/io/iohk/ethereum/jsonrpc/TestService.scala index b6aafb3ac4..5e97c14e87 100644 --- a/src/main/scala/io/iohk/ethereum/jsonrpc/TestService.scala +++ b/src/main/scala/io/iohk/ethereum/jsonrpc/TestService.scala @@ -403,7 +403,7 @@ class TestService( } def getLogHash(request: GetLogHashRequest): ServiceResponse[GetLogHashResponse] = { - import io.iohk.ethereum.network.p2p.messages.PV63.TxLogEntryImplicits.TxLogEntryEnc + import io.iohk.ethereum.network.p2p.messages.ETH63.TxLogEntryImplicits.TxLogEntryEnc val result = for { transactionLocation <- blockchain.getTransactionLocation(request.transactionHash) diff --git a/src/main/scala/io/iohk/ethereum/network/EtcPeerManagerActor.scala b/src/main/scala/io/iohk/ethereum/network/EtcPeerManagerActor.scala index eaea5a2e34..0e23807312 100644 --- a/src/main/scala/io/iohk/ethereum/network/EtcPeerManagerActor.scala +++ b/src/main/scala/io/iohk/ethereum/network/EtcPeerManagerActor.scala @@ -10,10 +10,10 @@ import io.iohk.ethereum.network.PeerEventBusActor.PeerEvent._ import io.iohk.ethereum.network.PeerEventBusActor.SubscriptionClassifier._ import io.iohk.ethereum.network.PeerEventBusActor.{PeerSelector, Subscribe, Unsubscribe} import io.iohk.ethereum.network.handshaker.Handshaker.HandshakeResult -import io.iohk.ethereum.network.p2p.messages.PV62.{BlockHeaders, GetBlockHeaders, NewBlockHashes} -import io.iohk.ethereum.network.p2p.messages.PV64.NewBlock +import io.iohk.ethereum.network.p2p.messages.ETH62.{BlockHeaders, GetBlockHeaders, NewBlockHashes} +import io.iohk.ethereum.network.p2p.messages.ETC64.NewBlock import io.iohk.ethereum.network.p2p.messages.WireProtocol.Disconnect -import io.iohk.ethereum.network.p2p.messages.{Codes, BaseETH6XMessages, PV64} +import io.iohk.ethereum.network.p2p.messages.{Codes, BaseETH6XMessages, ETC64} import io.iohk.ethereum.network.p2p.{Message, MessageSerializable} import io.iohk.ethereum.utils.ByteStringUtils @@ -161,7 +161,7 @@ class EtcPeerManagerActor( message match { case newBlock: BaseETH6XMessages.NewBlock => initialPeerInfo.copy(chainWeight = ChainWeight.totalDifficultyOnly(newBlock.totalDifficulty)) - case newBlock: PV64.NewBlock => initialPeerInfo.copy(chainWeight = newBlock.chainWeight) + case newBlock: ETC64.NewBlock => initialPeerInfo.copy(chainWeight = newBlock.chainWeight) case _ => initialPeerInfo } @@ -259,7 +259,7 @@ object EtcPeerManagerActor { } object RemoteStatus { - def apply(status: PV64.Status): RemoteStatus = { + def apply(status: ETC64.Status): RemoteStatus = { RemoteStatus(status.protocolVersion, status.networkId, status.chainWeight, status.bestHash, status.genesisHash) } diff --git a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcForkBlockExchangeState.scala b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcForkBlockExchangeState.scala index 74b6969dc6..67096355a8 100644 --- a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcForkBlockExchangeState.scala +++ b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcForkBlockExchangeState.scala @@ -3,7 +3,7 @@ package io.iohk.ethereum.network.handshaker import io.iohk.ethereum.network.EtcPeerManagerActor.{PeerInfo, RemoteStatus} import io.iohk.ethereum.network.ForkResolver import io.iohk.ethereum.network.handshaker.Handshaker.NextMessage -import io.iohk.ethereum.network.p2p.messages.PV62.{BlockHeaders, GetBlockHeaders} +import io.iohk.ethereum.network.p2p.messages.ETH62.{BlockHeaders, GetBlockHeaders} import io.iohk.ethereum.network.p2p.messages.WireProtocol.Disconnect import io.iohk.ethereum.network.p2p.{Message, MessageSerializable} import io.iohk.ethereum.utils.Logger diff --git a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala index 84a3c3c902..fc223aea5f 100644 --- a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala +++ b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala @@ -27,10 +27,10 @@ case class EtcHelloExchangeState(handshakerConfiguration: EtcHandshakerConfigura override def applyResponseMessage: PartialFunction[Message, HandshakerState[PeerInfo]] = { case hello: Hello => log.debug("Protocol handshake finished with peer ({})", hello) capability match { - case ProtocolVersions.PV64 => EtcNodeStatus64ExchangeState(handshakerConfiguration) - case ProtocolVersions.PV63 => EtcNodeStatus63ExchangeState(handshakerConfiguration) + case ProtocolVersions.ETC64 => EtcNodeStatus64ExchangeState(handshakerConfiguration) + case ProtocolVersions.ETH63 => EtcNodeStatus63ExchangeState(handshakerConfiguration) case _ => - log.debug(s"Connected peer does not support ${ProtocolVersions.PV63} / ${ProtocolVersions.PV64} protocol. Disconnecting.") + log.debug(s"Connected peer does not support ${ProtocolVersions.ETH63} / ${ProtocolVersions.ETC64} protocol. Disconnecting.") DisconnectedState(Disconnect.Reasons.IncompatibleP2pProtocolVersion) } } diff --git a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcNodeStatus63ExchangeState.scala b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcNodeStatus63ExchangeState.scala index 276cd2342c..010554ee60 100644 --- a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcNodeStatus63ExchangeState.scala +++ b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcNodeStatus63ExchangeState.scala @@ -20,7 +20,7 @@ case class EtcNodeStatus63ExchangeState( val chainWeight = blockchain.getChainWeightByHash(bestBlockHeader.hash).get val status = BaseETH6XMessages.Status( - protocolVersion = ProtocolVersions.PV63.version, + protocolVersion = ProtocolVersions.ETH63.version, networkId = peerConfiguration.networkId, totalDifficulty = chainWeight.totalDifficulty, bestHash = bestBlockHeader.hash, diff --git a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcNodeStatus64ExchangeState.scala b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcNodeStatus64ExchangeState.scala index cb9b9ce55b..4b70cd389b 100644 --- a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcNodeStatus64ExchangeState.scala +++ b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcNodeStatus64ExchangeState.scala @@ -1,16 +1,16 @@ package io.iohk.ethereum.network.handshaker import io.iohk.ethereum.network.EtcPeerManagerActor.{PeerInfo, RemoteStatus} -import io.iohk.ethereum.network.p2p.messages.{PV64, ProtocolVersions} +import io.iohk.ethereum.network.p2p.messages.{ETC64, ProtocolVersions} import io.iohk.ethereum.network.p2p.{Message, MessageSerializable} case class EtcNodeStatus64ExchangeState( handshakerConfiguration: EtcHandshakerConfiguration -) extends EtcNodeStatusExchangeState[PV64.Status] { +) extends EtcNodeStatusExchangeState[ETC64.Status] { import handshakerConfiguration._ - def applyResponseMessage: PartialFunction[Message, HandshakerState[PeerInfo]] = { case status: PV64.Status => + def applyResponseMessage: PartialFunction[Message, HandshakerState[PeerInfo]] = { case status: ETC64.Status => applyRemoteStatusMessage(RemoteStatus(status)) } @@ -18,8 +18,8 @@ case class EtcNodeStatus64ExchangeState( val bestBlockHeader = getBestBlockHeader() val chainWeight = blockchain.getChainWeightByHash(bestBlockHeader.hash).get - val status = PV64.Status( - protocolVersion = ProtocolVersions.PV64.version, + val status = ETC64.Status( + protocolVersion = ProtocolVersions.ETC64.version, networkId = peerConfiguration.networkId, chainWeight = chainWeight, bestHash = bestBlockHeader.hash, diff --git a/src/main/scala/io/iohk/ethereum/network/p2p/MessageDecoders.scala b/src/main/scala/io/iohk/ethereum/network/p2p/MessageDecoders.scala index 24b34b9c64..e6344304c8 100644 --- a/src/main/scala/io/iohk/ethereum/network/p2p/MessageDecoders.scala +++ b/src/main/scala/io/iohk/ethereum/network/p2p/MessageDecoders.scala @@ -2,16 +2,16 @@ package io.iohk.ethereum.network.p2p import io.iohk.ethereum.network.p2p.messages.{Capability, Codes} import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.SignedTransactions._ -import io.iohk.ethereum.network.p2p.messages.PV61.BlockHashesFromNumber._ -import io.iohk.ethereum.network.p2p.messages.PV62.BlockBodies._ -import io.iohk.ethereum.network.p2p.messages.PV62.BlockHeaders._ -import io.iohk.ethereum.network.p2p.messages.PV62.GetBlockBodies._ -import io.iohk.ethereum.network.p2p.messages.PV62.GetBlockHeaders._ -import io.iohk.ethereum.network.p2p.messages.PV62.NewBlockHashes._ -import io.iohk.ethereum.network.p2p.messages.PV63.GetNodeData._ -import io.iohk.ethereum.network.p2p.messages.PV63.GetReceipts._ -import io.iohk.ethereum.network.p2p.messages.PV63.NodeData._ -import io.iohk.ethereum.network.p2p.messages.PV63.Receipts._ +import io.iohk.ethereum.network.p2p.messages.ETH61.BlockHashesFromNumber._ +import io.iohk.ethereum.network.p2p.messages.ETH62.BlockBodies._ +import io.iohk.ethereum.network.p2p.messages.ETH62.BlockHeaders._ +import io.iohk.ethereum.network.p2p.messages.ETH62.GetBlockBodies._ +import io.iohk.ethereum.network.p2p.messages.ETH62.GetBlockHeaders._ +import io.iohk.ethereum.network.p2p.messages.ETH62.NewBlockHashes._ +import io.iohk.ethereum.network.p2p.messages.ETH63.GetNodeData._ +import io.iohk.ethereum.network.p2p.messages.ETH63.GetReceipts._ +import io.iohk.ethereum.network.p2p.messages.ETH63.NodeData._ +import io.iohk.ethereum.network.p2p.messages.ETH63.Receipts._ import io.iohk.ethereum.network.p2p.messages.ProtocolVersions._ import io.iohk.ethereum.network.p2p.messages.WireProtocol.Disconnect._ import io.iohk.ethereum.network.p2p.messages.WireProtocol.Hello._ @@ -37,10 +37,10 @@ object EthereumMessageDecoder extends MessageDecoder { override def fromBytes(msgCode: Int, payload: Array[Byte], protocolVersion: Capability): Message = { protocolVersion match { - case PV64 => handlePV64(msgCode, payload) - case PV63 => handlePV63(msgCode, payload) - case PV62 => handlePV62(msgCode, payload) - case PV61 => handlePV61(msgCode, payload) + case ETC64 => handleETC64(msgCode, payload) + case ETH63 => handleETH63(msgCode, payload) + case ETH62 => handleETH62(msgCode, payload) + case ETH61 => handleETH61(msgCode, payload) case pv => throw new RuntimeException("Unknown protocol version: " + pv) } } @@ -60,10 +60,10 @@ object EthereumMessageDecoder extends MessageDecoder { } } - private def handlePV61(msgCode: Int, payload: Array[Byte]): Message = { + private def handleETH61(msgCode: Int, payload: Array[Byte]): Message = { msgCode match { case Codes.NewBlockHashesCode => - import io.iohk.ethereum.network.p2p.messages.PV61.NewBlockHashes._ + import io.iohk.ethereum.network.p2p.messages.ETH61.NewBlockHashes._ payload.toNewBlockHashes case Codes.BlockHashesFromNumberCode => payload.toBlockHashesFromNumber @@ -71,7 +71,7 @@ object EthereumMessageDecoder extends MessageDecoder { } } - private def handlePV62(msgCode: Int, payload: Array[Byte]): Message = { + private def handleETH62(msgCode: Int, payload: Array[Byte]): Message = { msgCode match { case Codes.NewBlockHashesCode => payload.toNewBlockHashes case Codes.GetBlockHeadersCode => payload.toGetBlockHeaders @@ -82,25 +82,25 @@ object EthereumMessageDecoder extends MessageDecoder { } } - private def handlePV63(msgCode: Int, payload: Array[Byte]): Message = { + private def handleETH63(msgCode: Int, payload: Array[Byte]): Message = { msgCode match { case Codes.GetNodeDataCode => payload.toGetNodeData case Codes.NodeDataCode => payload.toNodeData case Codes.GetReceiptsCode => payload.toGetReceipts case Codes.ReceiptsCode => payload.toReceipts - case _ => handlePV62(msgCode, payload) + case _ => handleETH62(msgCode, payload) } } - private def handlePV64(msgCode: Int, payload: Array[Byte]): Message = { + private def handleETC64(msgCode: Int, payload: Array[Byte]): Message = { msgCode match { case Codes.StatusCode => - import io.iohk.ethereum.network.p2p.messages.PV64.Status._ + import io.iohk.ethereum.network.p2p.messages.ETC64.Status._ payload.toStatus case Codes.NewBlockCode => - import io.iohk.ethereum.network.p2p.messages.PV64.NewBlock._ + import io.iohk.ethereum.network.p2p.messages.ETC64.NewBlock._ payload.toNewBlock - case _ => handlePV63(msgCode, payload) + case _ => handleETH63(msgCode, payload) } } } diff --git a/src/main/scala/io/iohk/ethereum/network/p2p/messages/Capability.scala b/src/main/scala/io/iohk/ethereum/network/p2p/messages/Capability.scala index 76ec2894fb..6412df0180 100644 --- a/src/main/scala/io/iohk/ethereum/network/p2p/messages/Capability.scala +++ b/src/main/scala/io/iohk/ethereum/network/p2p/messages/Capability.scala @@ -39,9 +39,9 @@ object Capability { } object Capabilities { - val Eth63Capability: Capability = ProtocolVersions.PV63 - val Etc64Capability: Capability = ProtocolVersions.PV64 + val Eth63Capability: Capability = ProtocolVersions.ETH63 + val Etc64Capability: Capability = ProtocolVersions.ETC64 - val All: Seq[Capability] = Seq(ProtocolVersions.PV64, ProtocolVersions.PV63) + val All: Seq[Capability] = Seq(ProtocolVersions.ETC64, ProtocolVersions.ETH63) } } diff --git a/src/main/scala/io/iohk/ethereum/network/p2p/messages/PV64.scala b/src/main/scala/io/iohk/ethereum/network/p2p/messages/ETC64.scala similarity index 95% rename from src/main/scala/io/iohk/ethereum/network/p2p/messages/PV64.scala rename to src/main/scala/io/iohk/ethereum/network/p2p/messages/ETC64.scala index 1ef6a43efd..2cf8619f98 100644 --- a/src/main/scala/io/iohk/ethereum/network/p2p/messages/PV64.scala +++ b/src/main/scala/io/iohk/ethereum/network/p2p/messages/ETC64.scala @@ -10,10 +10,10 @@ import io.iohk.ethereum.rlp._ import org.bouncycastle.util.encoders.Hex /** - * This is temporal PV64 version, the real one will be implemented by ETCM-355 - * This one will be probably PV66 in the future + * This is temporary ETC64 version, the real one will be implemented by ETCM-355 + * This one will be probably ETC67 in the future */ -object PV64 { +object ETC64 { object Status { implicit class StatusEnc(val underlyingMsg: Status) extends MessageSerializableImplicit[Status](underlyingMsg) @@ -51,7 +51,7 @@ object PV64 { genesisHash ) - case _ => throw new RuntimeException("Cannot decode Status PV64 version") + case _ => throw new RuntimeException("Cannot decode Status ETC64 version") } } @@ -117,7 +117,7 @@ object PV64 { ), ChainWeight(lastCheckpointNumber, totalDifficulty) ) - case _ => throw new RuntimeException("Cannot decode NewBlock PV64 version") + case _ => throw new RuntimeException("Cannot decode NewBlock ETC64 version") } } } diff --git a/src/main/scala/io/iohk/ethereum/network/p2p/messages/PV61.scala b/src/main/scala/io/iohk/ethereum/network/p2p/messages/ETH61.scala similarity index 99% rename from src/main/scala/io/iohk/ethereum/network/p2p/messages/PV61.scala rename to src/main/scala/io/iohk/ethereum/network/p2p/messages/ETH61.scala index 2f38150be5..8b8369254c 100644 --- a/src/main/scala/io/iohk/ethereum/network/p2p/messages/PV61.scala +++ b/src/main/scala/io/iohk/ethereum/network/p2p/messages/ETH61.scala @@ -7,7 +7,7 @@ import io.iohk.ethereum.rlp.RLPImplicits._ import io.iohk.ethereum.rlp._ import org.bouncycastle.util.encoders.Hex -object PV61 { +object ETH61 { object NewBlockHashes { implicit class NewBlockHashesEnc(val underlyingMsg: NewBlockHashes) diff --git a/src/main/scala/io/iohk/ethereum/network/p2p/messages/PV62.scala b/src/main/scala/io/iohk/ethereum/network/p2p/messages/ETH62.scala similarity index 99% rename from src/main/scala/io/iohk/ethereum/network/p2p/messages/PV62.scala rename to src/main/scala/io/iohk/ethereum/network/p2p/messages/ETH62.scala index b1ea511745..3e147d3b26 100644 --- a/src/main/scala/io/iohk/ethereum/network/p2p/messages/PV62.scala +++ b/src/main/scala/io/iohk/ethereum/network/p2p/messages/ETH62.scala @@ -10,7 +10,7 @@ import io.iohk.ethereum.rlp.RLPImplicits._ import io.iohk.ethereum.rlp.{RLPList, _} import org.bouncycastle.util.encoders.Hex -object PV62 { +object ETH62 { object BlockHash { implicit class BlockHashEnc(blockHash: BlockHash) extends RLPSerializable { diff --git a/src/main/scala/io/iohk/ethereum/network/p2p/messages/PV63.scala b/src/main/scala/io/iohk/ethereum/network/p2p/messages/ETH63.scala similarity index 99% rename from src/main/scala/io/iohk/ethereum/network/p2p/messages/PV63.scala rename to src/main/scala/io/iohk/ethereum/network/p2p/messages/ETH63.scala index 258bdac122..f60ec0072b 100644 --- a/src/main/scala/io/iohk/ethereum/network/p2p/messages/PV63.scala +++ b/src/main/scala/io/iohk/ethereum/network/p2p/messages/ETH63.scala @@ -9,7 +9,7 @@ import io.iohk.ethereum.rlp.RLPImplicits._ import io.iohk.ethereum.rlp._ import org.bouncycastle.util.encoders.Hex -object PV63 { +object ETH63 { object GetNodeData { implicit class GetNodeDataEnc(val underlyingMsg: GetNodeData) diff --git a/src/main/scala/io/iohk/ethereum/network/p2p/messages/package.scala b/src/main/scala/io/iohk/ethereum/network/p2p/messages/package.scala index ebe83e5248..51cfceb9fd 100644 --- a/src/main/scala/io/iohk/ethereum/network/p2p/messages/package.scala +++ b/src/main/scala/io/iohk/ethereum/network/p2p/messages/package.scala @@ -2,10 +2,10 @@ package io.iohk.ethereum.network.p2p package object messages { object ProtocolVersions { - val PV61: Capability = Capability("eth", 61.toByte) - val PV62: Capability = Capability("eth", 62.toByte) - val PV63: Capability = Capability("eth", 63.toByte) - val PV64: Capability = Capability("etc", 64.toByte) + val ETH61: Capability = Capability("eth", 61.toByte) + val ETH62: Capability = Capability("eth", 62.toByte) + val ETH63: Capability = Capability("eth", 63.toByte) + val ETC64: Capability = Capability("etc", 64.toByte) val SubProtocolOffset = 0x10 } @@ -19,7 +19,7 @@ package object messages { val GetBlockBodiesCode: Int = ProtocolVersions.SubProtocolOffset + 0x05 val BlockBodiesCode: Int = ProtocolVersions.SubProtocolOffset + 0x06 val NewBlockCode: Int = ProtocolVersions.SubProtocolOffset + 0x07 - // This message is removed in PV62 and this code is reused in PV65 with different msg type + // This message is removed in ETH62 and this code is reused in ETH65 with different msg type val BlockHashesFromNumberCode: Int = ProtocolVersions.SubProtocolOffset + 0x08 val GetNodeDataCode: Int = ProtocolVersions.SubProtocolOffset + 0x0d val NodeDataCode: Int = ProtocolVersions.SubProtocolOffset + 0x0e diff --git a/src/test/scala/io/iohk/ethereum/ObjectGenerators.scala b/src/test/scala/io/iohk/ethereum/ObjectGenerators.scala index 415ab214d2..ecf20fb8c7 100644 --- a/src/test/scala/io/iohk/ethereum/ObjectGenerators.scala +++ b/src/test/scala/io/iohk/ethereum/ObjectGenerators.scala @@ -12,7 +12,7 @@ import io.iohk.ethereum.domain._ import io.iohk.ethereum.mpt.HexPrefix.bytesToNibbles import io.iohk.ethereum.mpt.{BranchNode, ExtensionNode, HashNode, LeafNode, MptNode, MptTraversals} import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.NewBlock -import io.iohk.ethereum.network.p2p.messages.PV64 +import io.iohk.ethereum.network.p2p.messages.ETC64 import org.bouncycastle.crypto.AsymmetricCipherKeyPair import org.scalacheck.{Arbitrary, Gen, Shrink} @@ -155,12 +155,12 @@ trait ObjectGenerators { td <- bigIntGen } yield NewBlock(Block(blockHeader, BlockBody(stxs, uncles)), td) - def newBlock64Gen(secureRandom: SecureRandom, chainId: Option[Byte]): Gen[PV64.NewBlock] = for { + def newBlock64Gen(secureRandom: SecureRandom, chainId: Option[Byte]): Gen[ETC64.NewBlock] = for { blockHeader <- blockHeaderGen stxs <- signedTxSeqGen(10, secureRandom, chainId) uncles <- seqBlockHeaderGen chainWeight <- chainWeightGen - } yield PV64.NewBlock(Block(blockHeader, BlockBody(stxs, uncles)), chainWeight) + } yield ETC64.NewBlock(Block(blockHeader, BlockBody(stxs, uncles)), chainWeight) def extraFieldsGen: Gen[HeaderExtraFields] = for { shouldCheckpoint <- Arbitrary.arbitrary[Option[Boolean]] diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/BlockBroadcastSpec.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/BlockBroadcastSpec.scala index 25226e23bd..9a680a4f2c 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/BlockBroadcastSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/BlockBroadcastSpec.scala @@ -9,9 +9,9 @@ import io.iohk.ethereum.blockchain.sync.regular.BlockBroadcast import io.iohk.ethereum.blockchain.sync.regular.BlockBroadcast.BlockToBroadcast import io.iohk.ethereum.domain.{Block, BlockBody, BlockHeader, ChainWeight} import io.iohk.ethereum.network.EtcPeerManagerActor.{PeerInfo, RemoteStatus} -import io.iohk.ethereum.network.p2p.messages.PV62.NewBlockHashes -import io.iohk.ethereum.network.p2p.messages.PV64.NewBlock -import io.iohk.ethereum.network.p2p.messages.{BaseETH6XMessages, PV62, ProtocolVersions} +import io.iohk.ethereum.network.p2p.messages.ETH62.NewBlockHashes +import io.iohk.ethereum.network.p2p.messages.ETC64.NewBlock +import io.iohk.ethereum.network.p2p.messages.{BaseETH6XMessages, ETH62, ProtocolVersions} import io.iohk.ethereum.network.{EtcPeerManagerActor, Peer, PeerId} import io.iohk.ethereum.{Fixtures, WithActorSystemShutDown} import org.scalatest.flatspec.AnyFlatSpecLike @@ -27,7 +27,7 @@ class BlockBroadcastSpec //given //Block that should be sent as it's total difficulty is higher than known by peer val blockHeader: BlockHeader = baseBlockHeader.copy(number = initialPeerInfo.maxBlockNumber - 3) - val newBlockNewHashes = NewBlockHashes(Seq(PV62.BlockHash(blockHeader.hash, blockHeader.number))) + val newBlockNewHashes = NewBlockHashes(Seq(ETH62.BlockHash(blockHeader.hash, blockHeader.number))) val newBlock = NewBlock(Block(blockHeader, BlockBody(Nil, Nil)), initialPeerInfo.chainWeight.increaseTotalDifficulty(2)) @@ -43,13 +43,13 @@ class BlockBroadcastSpec etcPeerManagerProbe.expectNoMessage() } - it should "send a new block when it is not known by the peer (known by comparing chain weights) (PV63)" in new TestSetup { + it should "send a new block when it is not known by the peer (known by comparing chain weights) (ETH63)" in new TestSetup { //given //Block that should be sent as it's total difficulty is higher than known by peer val blockHeader: BlockHeader = baseBlockHeader.copy(number = initialPeerInfo.maxBlockNumber - 3) - val newBlockNewHashes = NewBlockHashes(Seq(PV62.BlockHash(blockHeader.hash, blockHeader.number))) + val newBlockNewHashes = NewBlockHashes(Seq(ETH62.BlockHash(blockHeader.hash, blockHeader.number))) val peerInfo = initialPeerInfo - .copy(remoteStatus = peerStatus.copy(protocolVersion = ProtocolVersions.PV63.version)) + .copy(remoteStatus = peerStatus.copy(protocolVersion = ProtocolVersions.ETH63.version)) .withChainWeight(ChainWeight.totalDifficultyOnly(initialPeerInfo.chainWeight.totalDifficulty)) val newBlock = BaseETH6XMessages.NewBlock(Block(blockHeader, BlockBody(Nil, Nil)), peerInfo.chainWeight.totalDifficulty + 2) @@ -86,7 +86,7 @@ class BlockBroadcastSpec it should "send a new block when it is not known by the peer (known by comparing max block number)" in new TestSetup { //given val blockHeader: BlockHeader = baseBlockHeader.copy(number = initialPeerInfo.maxBlockNumber + 4) - val newBlockNewHashes = NewBlockHashes(Seq(PV62.BlockHash(blockHeader.hash, blockHeader.number))) + val newBlockNewHashes = NewBlockHashes(Seq(ETH62.BlockHash(blockHeader.hash, blockHeader.number))) val newBlock = NewBlock(Block(blockHeader, BlockBody(Nil, Nil)), initialPeerInfo.chainWeight.increaseTotalDifficulty(-2)) @@ -122,7 +122,7 @@ class BlockBroadcastSpec it should "send block hashes to all peers while the blocks only to sqrt of them" in new TestSetup { //given val firstHeader: BlockHeader = baseBlockHeader.copy(number = initialPeerInfo.maxBlockNumber + 4) - val firstBlockNewHashes = NewBlockHashes(Seq(PV62.BlockHash(firstHeader.hash, firstHeader.number))) + val firstBlockNewHashes = NewBlockHashes(Seq(ETH62.BlockHash(firstHeader.hash, firstHeader.number))) val firstBlock = NewBlock(Block(firstHeader, BlockBody(Nil, Nil)), initialPeerInfo.chainWeight.increaseTotalDifficulty(-2)) @@ -164,7 +164,7 @@ class BlockBroadcastSpec val baseBlockHeader = Fixtures.Blocks.Block3125369.header val peerStatus = RemoteStatus( - protocolVersion = ProtocolVersions.PV64.version, + protocolVersion = ProtocolVersions.ETC64.version, networkId = 1, chainWeight = ChainWeight(10, 10000), bestHash = Fixtures.Blocks.Block3125369.header.hash, diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/BlockchainHostActorSpec.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/BlockchainHostActorSpec.scala index e06bac6f85..1aeadd815d 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/BlockchainHostActorSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/BlockchainHostActorSpec.scala @@ -10,9 +10,9 @@ import io.iohk.ethereum.network.PeerEventBusActor.SubscriptionClassifier.Message import io.iohk.ethereum.network.PeerEventBusActor.{PeerSelector, Subscribe} import io.iohk.ethereum.network.PeerManagerActor.{FastSyncHostConfiguration, PeerConfiguration} import io.iohk.ethereum.network.p2p.messages.Codes -import io.iohk.ethereum.network.p2p.messages.PV62._ -import io.iohk.ethereum.network.p2p.messages.PV63._ -import io.iohk.ethereum.network.p2p.messages.PV63.MptNodeEncoders._ +import io.iohk.ethereum.network.p2p.messages.ETH62._ +import io.iohk.ethereum.network.p2p.messages.ETH63._ +import io.iohk.ethereum.network.p2p.messages.ETH63.MptNodeEncoders._ import io.iohk.ethereum.network.rlpx.RLPxConnectionHandler.RLPxConfiguration import io.iohk.ethereum.network.{EtcPeerManagerActor, PeerId} import io.iohk.ethereum.{Fixtures, Timeouts, crypto} diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/EtcPeerManagerFake.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/EtcPeerManagerFake.scala index c8253e58bb..c99e2fa472 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/EtcPeerManagerFake.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/EtcPeerManagerFake.scala @@ -9,8 +9,8 @@ import io.iohk.ethereum.domain.{Block, BlockHeader} import io.iohk.ethereum.network.{EtcPeerManagerActor, Peer, PeerId} import io.iohk.ethereum.network.EtcPeerManagerActor.{PeerInfo, SendMessage} import io.iohk.ethereum.network.PeerEventBusActor.PeerEvent.MessageFromPeer -import io.iohk.ethereum.network.p2p.messages.PV62.{BlockBodies, BlockHeaders, GetBlockBodies, GetBlockHeaders} -import io.iohk.ethereum.network.p2p.messages.PV63.{GetNodeData, GetReceipts, NodeData, Receipts} +import io.iohk.ethereum.network.p2p.messages.ETH62.{BlockBodies, BlockHeaders, GetBlockBodies, GetBlockHeaders} +import io.iohk.ethereum.network.p2p.messages.ETH63.{GetNodeData, GetReceipts, NodeData, Receipts} import io.iohk.ethereum.utils.Config.SyncConfig import monix.eval.Task import monix.execution.Scheduler diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/PeersClientSpec.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/PeersClientSpec.scala index ded702ce40..ebe509fe96 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/PeersClientSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/PeersClientSpec.scala @@ -76,7 +76,7 @@ class PeersClientSpec extends AnyFlatSpec with Matchers with ScalaCheckPropertyC val peer3 = Peer(PeerId("peer3"), new InetSocketAddress("127.0.0.1", 3), TestProbe().ref, false) private val peerStatus = RemoteStatus( - protocolVersion = ProtocolVersions.PV63.version, + protocolVersion = ProtocolVersions.ETH63.version, networkId = 1, chainWeight = ChainWeight(0, 0), bestHash = ByteString.empty, diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/PivotBlockSelectorSpec.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/PivotBlockSelectorSpec.scala index 7340e98f58..3bdcc90961 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/PivotBlockSelectorSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/PivotBlockSelectorSpec.scala @@ -14,7 +14,7 @@ import io.iohk.ethereum.network.PeerEventBusActor.SubscriptionClassifier.{Messag import io.iohk.ethereum.network.PeerEventBusActor.{PeerSelector, Subscribe, Unsubscribe} import io.iohk.ethereum.network.p2p.Message import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.NewBlock -import io.iohk.ethereum.network.p2p.messages.PV62._ +import io.iohk.ethereum.network.p2p.messages.ETH62._ import io.iohk.ethereum.network.p2p.messages.{Codes, ProtocolVersions} import io.iohk.ethereum.network.{EtcPeerManagerActor, Peer, PeerId} import io.iohk.ethereum.utils.Config.SyncConfig @@ -571,7 +571,7 @@ class PivotBlockSelectorSpec val peer1Status = RemoteStatus( - ProtocolVersions.PV64.version, + ProtocolVersions.ETC64.version, 1, ChainWeight.totalDifficultyOnly(20), ByteString("peer1_bestHash"), diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncSpec.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncSpec.scala index 61c6ad1e64..26d59a0608 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncSpec.scala @@ -21,8 +21,8 @@ import io.iohk.ethereum.db.dataSource.RocksDbDataSource.IterationError import io.iohk.ethereum.domain.{Address, BlockchainImpl, ChainWeight} import io.iohk.ethereum.network.EtcPeerManagerActor._ import io.iohk.ethereum.network.PeerEventBusActor.PeerEvent.MessageFromPeer -import io.iohk.ethereum.network.p2p.messages.PV63.GetNodeData.GetNodeDataEnc -import io.iohk.ethereum.network.p2p.messages.PV63.NodeData +import io.iohk.ethereum.network.p2p.messages.ETH63.GetNodeData.GetNodeDataEnc +import io.iohk.ethereum.network.p2p.messages.ETH63.NodeData import io.iohk.ethereum.network.p2p.messages.ProtocolVersions import io.iohk.ethereum.network.{Peer, PeerId} import io.iohk.ethereum.utils.Config @@ -148,7 +148,7 @@ class StateSyncSpec val syncInit = TestProbe() val peerStatus = RemoteStatus( - protocolVersion = ProtocolVersions.PV63.version, + protocolVersion = ProtocolVersions.ETH63.version, networkId = 1, chainWeight = ChainWeight.totalDifficultyOnly(10000), bestHash = Fixtures.Blocks.Block3125369.header.hash, diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/SyncControllerSpec.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/SyncControllerSpec.scala index 380359e8f4..04eea355e7 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/SyncControllerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/SyncControllerSpec.scala @@ -15,12 +15,12 @@ import io.iohk.ethereum.ledger.Ledger.VMImpl import io.iohk.ethereum.network.EtcPeerManagerActor import io.iohk.ethereum.network.EtcPeerManagerActor.{HandshakedPeers, SendMessage} import io.iohk.ethereum.network.PeerEventBusActor.PeerEvent.MessageFromPeer -import io.iohk.ethereum.network.p2p.messages.PV62.GetBlockBodies.GetBlockBodiesEnc -import io.iohk.ethereum.network.p2p.messages.PV62.GetBlockHeaders.GetBlockHeadersEnc -import io.iohk.ethereum.network.p2p.messages.PV62._ -import io.iohk.ethereum.network.p2p.messages.PV63.GetNodeData.GetNodeDataEnc -import io.iohk.ethereum.network.p2p.messages.PV63.GetReceipts.GetReceiptsEnc -import io.iohk.ethereum.network.p2p.messages.PV63.{NodeData, Receipts} +import io.iohk.ethereum.network.p2p.messages.ETH62.GetBlockBodies.GetBlockBodiesEnc +import io.iohk.ethereum.network.p2p.messages.ETH62.GetBlockHeaders.GetBlockHeadersEnc +import io.iohk.ethereum.network.p2p.messages.ETH62._ +import io.iohk.ethereum.network.p2p.messages.ETH63.GetNodeData.GetNodeDataEnc +import io.iohk.ethereum.network.p2p.messages.ETH63.GetReceipts.GetReceiptsEnc +import io.iohk.ethereum.network.p2p.messages.ETH63.{NodeData, Receipts} import io.iohk.ethereum.utils.Config.SyncConfig import io.iohk.ethereum.{Fixtures, Mocks, NormalPatience} import org.bouncycastle.util.encoders.Hex diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/SyncStateDownloaderStateSpec.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/SyncStateDownloaderStateSpec.scala index 561bcd8f06..54bd618f9c 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/SyncStateDownloaderStateSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/SyncStateDownloaderStateSpec.scala @@ -16,7 +16,7 @@ import io.iohk.ethereum.blockchain.sync.fast.SyncStateSchedulerActor.{ import io.iohk.ethereum.blockchain.sync.fast.DownloaderState import io.iohk.ethereum.crypto.kec256 import io.iohk.ethereum.network.{Peer, PeerId} -import io.iohk.ethereum.network.p2p.messages.PV63.NodeData +import io.iohk.ethereum.network.p2p.messages.ETH63.NodeData import org.scalatest.BeforeAndAfterAll import org.scalatest.flatspec.AnyFlatSpecLike import org.scalatest.matchers.must.Matchers diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/TestSyncPeers.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/TestSyncPeers.scala index 0b6a1579bf..1f657f808c 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/TestSyncPeers.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/TestSyncPeers.scala @@ -21,7 +21,7 @@ trait TestSyncPeers { self: TestSyncConfig => val peer1Status = RemoteStatus( - ProtocolVersions.PV64.version, + ProtocolVersions.ETC64.version, 1, ChainWeight.totalDifficultyOnly(20), ByteString("peer1_bestHash"), diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/fast/FastSyncBranchResolverActorSpec.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/fast/FastSyncBranchResolverActorSpec.scala index ab1618bcd6..3020bceaff 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/fast/FastSyncBranchResolverActorSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/fast/FastSyncBranchResolverActorSpec.scala @@ -14,7 +14,7 @@ import io.iohk.ethereum.blockchain.sync.fast.FastSyncBranchResolverActor.{Branch import io.iohk.ethereum.domain.{Block, BlockHeader, ChainWeight} import io.iohk.ethereum.network.EtcPeerManagerActor._ import io.iohk.ethereum.network.PeerEventBusActor.PeerEvent.MessageFromPeer -import io.iohk.ethereum.network.p2p.messages.PV62.{BlockHeaders, GetBlockHeaders} +import io.iohk.ethereum.network.p2p.messages.ETH62.{BlockHeaders, GetBlockHeaders} import io.iohk.ethereum.network.p2p.messages.ProtocolVersions import io.iohk.ethereum.network.{EtcPeerManagerActor, Peer, PeerId} import io.iohk.ethereum.utils.Logger @@ -246,7 +246,7 @@ class FastSyncBranchResolverActorSpec def peerId(number: Int): PeerId = PeerId(s"peer_$number") def getPeer(id: PeerId): Peer = Peer(id, new InetSocketAddress("127.0.0.1", 0), TestProbe(id.value).ref, incomingConnection = false) - def getPeerInfo(peer: Peer, protocolVersion: Int = ProtocolVersions.PV64.version): PeerInfo = { + def getPeerInfo(peer: Peer, protocolVersion: Int = ProtocolVersions.ETC64.version): PeerInfo = { val status = RemoteStatus( protocolVersion, diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/BlockFetcherSpec.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/BlockFetcherSpec.scala index 5494247733..b13eb3eb57 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/BlockFetcherSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/BlockFetcherSpec.scala @@ -22,7 +22,7 @@ import io.iohk.ethereum.network.PeerEventBusActor.SubscriptionClassifier.Message import io.iohk.ethereum.network.PeerEventBusActor.{PeerSelector, Subscribe} import io.iohk.ethereum.network.p2p.messages.Codes import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.NewBlock -import io.iohk.ethereum.network.p2p.messages.PV62._ +import io.iohk.ethereum.network.p2p.messages.ETH62._ import io.iohk.ethereum.network.{Peer, PeerId} import io.iohk.ethereum.security.SecureRandomBuilder import io.iohk.ethereum.{BlockHelpers, Timeouts} diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncFixtures.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncFixtures.scala index 7673169ce2..961e11cb96 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncFixtures.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncFixtures.scala @@ -20,9 +20,9 @@ import io.iohk.ethereum.network.EtcPeerManagerActor.{PeerInfo, RemoteStatus} import io.iohk.ethereum.network.PeerEventBusActor.PeerEvent.MessageFromPeer import io.iohk.ethereum.network.PeerEventBusActor.Subscribe import io.iohk.ethereum.network.p2p.Message -import io.iohk.ethereum.network.p2p.messages.PV62._ -import io.iohk.ethereum.network.p2p.messages.PV63.{GetNodeData, NodeData} -import io.iohk.ethereum.network.p2p.messages.PV64.NewBlock +import io.iohk.ethereum.network.p2p.messages.ETH62._ +import io.iohk.ethereum.network.p2p.messages.ETH63.{GetNodeData, NodeData} +import io.iohk.ethereum.network.p2p.messages.ETC64.NewBlock import io.iohk.ethereum.network.p2p.messages.ProtocolVersions import io.iohk.ethereum.network.{Peer, PeerId} import io.iohk.ethereum.utils.Config.SyncConfig @@ -102,7 +102,7 @@ trait RegularSyncFixtures { self: Matchers with AsyncMockFactory => def getPeer(id: PeerId): Peer = Peer(id, new InetSocketAddress("127.0.0.1", 0), TestProbe(id.value).ref, incomingConnection = false) - def getPeerInfo(peer: Peer, protocolVersion: Int = ProtocolVersions.PV64.version): PeerInfo = { + def getPeerInfo(peer: Peer, protocolVersion: Int = ProtocolVersions.ETC64.version): PeerInfo = { val status = RemoteStatus( protocolVersion, diff --git a/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncSpec.scala b/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncSpec.scala index 1221d4cf63..1fd2d196d8 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/regular/RegularSyncSpec.scala @@ -24,9 +24,9 @@ import io.iohk.ethereum.network.PeerEventBusActor.PeerEvent.MessageFromPeer import io.iohk.ethereum.network.PeerEventBusActor.SubscriptionClassifier.MessageClassifier import io.iohk.ethereum.network.PeerEventBusActor.{PeerSelector, Subscribe} import io.iohk.ethereum.network.p2p.messages.{Codes, BaseETH6XMessages, ProtocolVersions} -import io.iohk.ethereum.network.p2p.messages.PV62._ -import io.iohk.ethereum.network.p2p.messages.PV63.{GetNodeData, NodeData} -import io.iohk.ethereum.network.p2p.messages.PV64.NewBlock +import io.iohk.ethereum.network.p2p.messages.ETH62._ +import io.iohk.ethereum.network.p2p.messages.ETH63.{GetNodeData, NodeData} +import io.iohk.ethereum.network.p2p.messages.ETC64.NewBlock import io.iohk.ethereum.network.{EtcPeerManagerActor, Peer, PeerEventBusActor} import io.iohk.ethereum.utils.Config.SyncConfig import io.iohk.ethereum.{BlockHelpers, ObjectGenerators, ResourceFixtures, WordSpecBase} @@ -674,19 +674,19 @@ class RegularSyncSpec } "broadcasting blocks" should { - "send a NewBlock message without latest checkpoint number when client not support PV64" in sync( + "send a NewBlock message without latest checkpoint number when client not support ETC64" in sync( new OnTopFixture(testSystem) { goToTop() - val peerWithPV63: (Peer, PeerInfo) = { + val peerWithETH63: (Peer, PeerInfo) = { val id = peerId(handshakedPeers.size) val peer = getPeer(id) - val peerInfo = getPeerInfo(peer, ProtocolVersions.PV63.version) + val peerInfo = getPeerInfo(peer, ProtocolVersions.ETH63.version) (peer, peerInfo) } etcPeerManager.expectMsg(GetHandshakedPeers) - etcPeerManager.reply(HandshakedPeers(Map(peerWithPV63._1 -> peerWithPV63._2))) + etcPeerManager.reply(HandshakedPeers(Map(peerWithETH63._1 -> peerWithETH63._2))) blockFetcher ! MessageFromPeer(BaseETH6XMessages.NewBlock(newBlock, newBlock.number), defaultPeer.id) @@ -701,7 +701,7 @@ class RegularSyncSpec } ) - "send a NewBlock message with latest checkpoint number when client supports PV64" in sync( + "send a NewBlock message with latest checkpoint number when client supports ETC64" in sync( new OnTopFixture(testSystem) { goToTop() diff --git a/src/test/scala/io/iohk/ethereum/db/storage/NodeStorageSuite.scala b/src/test/scala/io/iohk/ethereum/db/storage/NodeStorageSuite.scala index 7a8c1f9125..d5d60cce4a 100644 --- a/src/test/scala/io/iohk/ethereum/db/storage/NodeStorageSuite.scala +++ b/src/test/scala/io/iohk/ethereum/db/storage/NodeStorageSuite.scala @@ -3,7 +3,7 @@ package io.iohk.ethereum.db.storage import akka.util.ByteString import io.iohk.ethereum.ObjectGenerators import io.iohk.ethereum.db.dataSource.EphemDataSource -import io.iohk.ethereum.network.p2p.messages.PV63.MptNodeEncoders._ +import io.iohk.ethereum.network.p2p.messages.ETH63.MptNodeEncoders._ import org.scalacheck.Gen import org.scalatestplus.scalacheck.ScalaCheckPropertyChecks import org.scalatest.funsuite.AnyFunSuite diff --git a/src/test/scala/io/iohk/ethereum/jsonrpc/DebugServiceSpec.scala b/src/test/scala/io/iohk/ethereum/jsonrpc/DebugServiceSpec.scala index 0bfc263c4f..5aa8640d05 100644 --- a/src/test/scala/io/iohk/ethereum/jsonrpc/DebugServiceSpec.scala +++ b/src/test/scala/io/iohk/ethereum/jsonrpc/DebugServiceSpec.scala @@ -64,7 +64,7 @@ class DebugServiceSpec val debugService = new DebugService(peerManager.ref, etcPeerManager.ref) val peerStatus = RemoteStatus( - protocolVersion = ProtocolVersions.PV63.version, + protocolVersion = ProtocolVersions.ETH63.version, networkId = 1, chainWeight = ChainWeight.totalDifficultyOnly(10000), bestHash = Fixtures.Blocks.Block3125369.header.hash, diff --git a/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerSpec.scala b/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerSpec.scala index 1cbbe0696a..0657b53dbe 100644 --- a/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerSpec.scala @@ -117,7 +117,7 @@ class JsonRpcControllerSpec it should "debug_listPeersInfo" in new JsonRpcControllerFixture { val peerStatus = RemoteStatus( - protocolVersion = ProtocolVersions.PV63.version, + protocolVersion = ProtocolVersions.ETH63.version, networkId = 1, chainWeight = ChainWeight.totalDifficultyOnly(10000), bestHash = Fixtures.Blocks.Block3125369.header.hash, diff --git a/src/test/scala/io/iohk/ethereum/network/EtcPeerManagerSpec.scala b/src/test/scala/io/iohk/ethereum/network/EtcPeerManagerSpec.scala index 9b8018b631..fe506ae857 100644 --- a/src/test/scala/io/iohk/ethereum/network/EtcPeerManagerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/EtcPeerManagerSpec.scala @@ -15,9 +15,9 @@ import io.iohk.ethereum.network.PeerEventBusActor.PeerEvent.{MessageFromPeer, Pe import io.iohk.ethereum.network.PeerEventBusActor.SubscriptionClassifier._ import io.iohk.ethereum.network.PeerEventBusActor.{PeerSelector, Subscribe} import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.NewBlock -import io.iohk.ethereum.network.p2p.messages.PV62._ +import io.iohk.ethereum.network.p2p.messages.ETH62._ import io.iohk.ethereum.network.p2p.messages.WireProtocol.Disconnect -import io.iohk.ethereum.network.p2p.messages.{Codes, PV64, ProtocolVersions} +import io.iohk.ethereum.network.p2p.messages.{Codes, ETC64, ProtocolVersions} import io.iohk.ethereum.utils.Config import org.bouncycastle.util.encoders.Hex import org.scalatest.flatspec.AnyFlatSpec @@ -43,7 +43,7 @@ class EtcPeerManagerSpec extends AnyFlatSpec with Matchers { requestSender.expectMsg(HandshakedPeers(Map(peer1 -> peer1Info, peer2 -> peer2Info))) } - it should "update max peer when receiving new block PV63" in new TestSetup { + it should "update max peer when receiving new block ETH63" in new TestSetup { peerEventBus.expectMsg(Subscribe(PeerHandshaked)) setupNewPeer(peer1, peer1Probe, peer1Info) @@ -67,17 +67,17 @@ class EtcPeerManagerSpec extends AnyFlatSpec with Matchers { requestSender.expectMsg(PeerInfoResponse(Some(expectedPeerInfo))) } - it should "update max peer when receiving new block PV64" in new TestSetup { + it should "update max peer when receiving new block ETC64" in new TestSetup { peerEventBus.expectMsg(Subscribe(PeerHandshaked)) - setupNewPeer(peer1, peer1Probe, peer1InfoPV64) + setupNewPeer(peer1, peer1Probe, peer1InfoETC64) //given val newBlockWeight = ChainWeight.totalDifficultyOnly(300) val firstHeader: BlockHeader = baseBlockHeader.copy(number = peer1Info.maxBlockNumber + 4) - val firstBlock = PV64.NewBlock(Block(firstHeader, BlockBody(Nil, Nil)), newBlockWeight) + val firstBlock = ETC64.NewBlock(Block(firstHeader, BlockBody(Nil, Nil)), newBlockWeight) val secondHeader: BlockHeader = baseBlockHeader.copy(number = peer2Info.maxBlockNumber + 2) - val secondBlock = PV64.NewBlock(Block(secondHeader, BlockBody(Nil, Nil)), newBlockWeight) + val secondBlock = ETC64.NewBlock(Block(secondHeader, BlockBody(Nil, Nil)), newBlockWeight) //when peersInfoHolder ! MessageFromPeer(firstBlock, peer1.id) @@ -85,7 +85,7 @@ class EtcPeerManagerSpec extends AnyFlatSpec with Matchers { //then requestSender.send(peersInfoHolder, PeerInfoRequest(peer1.id)) - val expectedPeerInfo = initialPeerInfoPV64 + val expectedPeerInfo = initialPeerInfoETC64 .withBestBlockData(initialPeerInfo.maxBlockNumber + 4, firstHeader.hash) .withChainWeight(newBlockWeight) requestSender.expectMsg(PeerInfoResponse(Some(expectedPeerInfo))) @@ -144,16 +144,16 @@ class EtcPeerManagerSpec extends AnyFlatSpec with Matchers { ) } - it should "update the peer chain weight when receiving a PV64.NewBlock" in new TestSetup { + it should "update the peer chain weight when receiving a ETC64.NewBlock" in new TestSetup { peerEventBus.expectMsg(Subscribe(PeerHandshaked)) - setupNewPeer(peer1, peer1Probe, peer1InfoPV64) + setupNewPeer(peer1, peer1Probe, peer1InfoETC64) //given - val newBlock = PV64.NewBlock( + val newBlock = ETC64.NewBlock( baseBlock, - initialPeerInfoPV64.chainWeight + initialPeerInfoETC64.chainWeight .increaseTotalDifficulty(1) - .copy(lastCheckpointNumber = initialPeerInfoPV64.chainWeight.lastCheckpointNumber + 1) + .copy(lastCheckpointNumber = initialPeerInfoETC64.chainWeight.lastCheckpointNumber + 1) ) //when @@ -161,7 +161,7 @@ class EtcPeerManagerSpec extends AnyFlatSpec with Matchers { //then requestSender.send(peersInfoHolder, PeerInfoRequest(peer1.id)) - requestSender.expectMsg(PeerInfoResponse(Some(peer1InfoPV64.withChainWeight(newBlock.chainWeight)))) + requestSender.expectMsg(PeerInfoResponse(Some(peer1InfoETC64.withChainWeight(newBlock.chainWeight)))) } it should "update the fork accepted when receiving the fork block" in new TestSetup { @@ -285,7 +285,7 @@ class EtcPeerManagerSpec extends AnyFlatSpec with Matchers { val forkResolver = new ForkResolver.EtcForkResolver(blockchainConfig.daoForkConfig.get) val peerStatus = RemoteStatus( - protocolVersion = ProtocolVersions.PV63.version, + protocolVersion = ProtocolVersions.ETH63.version, networkId = 1, chainWeight = ChainWeight.totalDifficultyOnly(10000), bestHash = Fixtures.Blocks.Block3125369.header.hash, @@ -300,8 +300,8 @@ class EtcPeerManagerSpec extends AnyFlatSpec with Matchers { bestBlockHash = peerStatus.bestHash ) - val initialPeerInfoPV64 = PeerInfo( - remoteStatus = peerStatus.copy(protocolVersion = ProtocolVersions.PV64.version), + val initialPeerInfoETC64 = PeerInfo( + remoteStatus = peerStatus.copy(protocolVersion = ProtocolVersions.ETC64.version), chainWeight = peerStatus.chainWeight, forkAccepted = false, maxBlockNumber = Fixtures.Blocks.Block3125369.header.number, @@ -313,7 +313,7 @@ class EtcPeerManagerSpec extends AnyFlatSpec with Matchers { val peer1Probe = TestProbe() val peer1 = Peer(PeerId("peer1"), new InetSocketAddress("127.0.0.1", 1), peer1Probe.ref, false, Some(fakeNodeId)) val peer1Info = initialPeerInfo.withForkAccepted(false) - val peer1InfoPV64 = initialPeerInfoPV64.withForkAccepted(false) + val peer1InfoETC64 = initialPeerInfoETC64.withForkAccepted(false) val peer2Probe = TestProbe() val peer2 = Peer(PeerId("peer2"), new InetSocketAddress("127.0.0.1", 2), peer2Probe.ref, false, Some(fakeNodeId)) val peer2Info = initialPeerInfo.withForkAccepted(false) diff --git a/src/test/scala/io/iohk/ethereum/network/PeerActorHandshakingSpec.scala b/src/test/scala/io/iohk/ethereum/network/PeerActorHandshakingSpec.scala index 691887e12b..a2ac9f573a 100644 --- a/src/test/scala/io/iohk/ethereum/network/PeerActorHandshakingSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/PeerActorHandshakingSpec.scala @@ -35,7 +35,7 @@ class PeerActorHandshakingSpec extends AnyFlatSpec with Matchers { //Establish probe rlpxconnection peerActorHandshakeSucceeds ! ConnectTo(uri) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.ConnectTo(uri)) - rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString(), ProtocolVersions.PV63)) + rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString(), ProtocolVersions.ETH63)) //Test that the handshake succeeded val sender = TestProbe()(system) @@ -52,7 +52,7 @@ class PeerActorHandshakingSpec extends AnyFlatSpec with Matchers { //Establish probe rlpxconnection peerActorHandshakeFails ! ConnectTo(uri) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.ConnectTo(uri)) - rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString(), ProtocolVersions.PV63)) + rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString(), ProtocolVersions.ETH63)) //Test that the handshake failed rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.SendMessage(Disconnect(defaultReasonDisconnect))) @@ -68,7 +68,7 @@ class PeerActorHandshakingSpec extends AnyFlatSpec with Matchers { //Establish probe rlpxconnection peerActorHandshakeRequiresHello ! ConnectTo(uri) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.ConnectTo(uri)) - rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString(), ProtocolVersions.PV63)) + rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString(), ProtocolVersions.ETH63)) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.SendMessage(defaultHello)) peerActorHandshakeRequiresHello ! RLPxConnectionHandler.MessageReceived(defaultHello) @@ -88,7 +88,7 @@ class PeerActorHandshakingSpec extends AnyFlatSpec with Matchers { //Establish probe rlpxconnection peerActorHandshakeRequiresHello ! ConnectTo(uri) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.ConnectTo(uri)) - rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString(), ProtocolVersions.PV63)) + rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString(), ProtocolVersions.ETH63)) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.SendMessage(defaultHello)) time.advance(defaultTimeout * 2) @@ -106,7 +106,7 @@ class PeerActorHandshakingSpec extends AnyFlatSpec with Matchers { //Establish probe rlpxconnection peerActorHandshakeRequiresHello ! ConnectTo(uri) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.ConnectTo(uri)) - rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString(), ProtocolVersions.PV63)) + rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString(), ProtocolVersions.ETH63)) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.SendMessage(defaultHello)) peerActorHandshakeRequiresHello ! RLPxConnectionHandler.MessageReceived(defaultStatusMsg) @@ -124,7 +124,7 @@ class PeerActorHandshakingSpec extends AnyFlatSpec with Matchers { //Establish probe rlpxconnection peerActorHandshakeRequiresHello ! ConnectTo(uri) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.ConnectTo(uri)) - rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString(), ProtocolVersions.PV63)) + rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString(), ProtocolVersions.ETH63)) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.SendMessage(defaultHello)) peerActorHandshakeRequiresHello ! RLPxConnectionHandler.MessageReceived(Pong()) //Ignored @@ -168,7 +168,7 @@ class PeerActorHandshakingSpec extends AnyFlatSpec with Matchers { object DefaultValues { val defaultStatusMsg = Status( - protocolVersion = ProtocolVersions.PV63.version, + protocolVersion = ProtocolVersions.ETH63.version, networkId = 1, totalDifficulty = Fixtures.Blocks.Genesis.header.difficulty, bestHash = Fixtures.Blocks.Genesis.header.hash, diff --git a/src/test/scala/io/iohk/ethereum/network/PeerEventBusActorSpec.scala b/src/test/scala/io/iohk/ethereum/network/PeerEventBusActorSpec.scala index 836dcd1bb0..2e6b27f29e 100644 --- a/src/test/scala/io/iohk/ethereum/network/PeerEventBusActorSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/PeerEventBusActorSpec.scala @@ -243,7 +243,7 @@ class PeerEventBusActorSpec extends AnyFlatSpec with Matchers { val peerEventBusActor = system.actorOf(PeerEventBusActor.props) val peerStatus = RemoteStatus( - protocolVersion = ProtocolVersions.PV63.version, + protocolVersion = ProtocolVersions.ETH63.version, networkId = 1, chainWeight = ChainWeight.totalDifficultyOnly(10000), bestHash = Fixtures.Blocks.Block3125369.header.hash, diff --git a/src/test/scala/io/iohk/ethereum/network/PeerManagerSpec.scala b/src/test/scala/io/iohk/ethereum/network/PeerManagerSpec.scala index c54c96b5a0..9f90d95353 100644 --- a/src/test/scala/io/iohk/ethereum/network/PeerManagerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/PeerManagerSpec.scala @@ -606,7 +606,7 @@ class PeerManagerSpec val blacklist: CacheBasedBlacklist = CacheBasedBlacklist(cache) val peerStatus = RemoteStatus( - protocolVersion = ProtocolVersions.PV63.version, + protocolVersion = ProtocolVersions.ETH63.version, networkId = 1, chainWeight = ChainWeight.totalDifficultyOnly(10000), bestHash = Fixtures.Blocks.Block3125369.header.hash, diff --git a/src/test/scala/io/iohk/ethereum/network/PeerStatisticsSpec.scala b/src/test/scala/io/iohk/ethereum/network/PeerStatisticsSpec.scala index 4dd7be9522..3e6449a950 100644 --- a/src/test/scala/io/iohk/ethereum/network/PeerStatisticsSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/PeerStatisticsSpec.scala @@ -3,7 +3,7 @@ package io.iohk.ethereum.network import akka.actor._ import akka.testkit.{TestKit, TestProbe} import io.iohk.ethereum.network.PeerEventBusActor._ -import io.iohk.ethereum.network.p2p.messages.PV61.NewBlockHashes +import io.iohk.ethereum.network.p2p.messages.ETH61.NewBlockHashes import io.iohk.ethereum.WithActorSystemShutDown import io.iohk.ethereum.utils.MockClock import org.scalatest.flatspec.AnyFlatSpecLike diff --git a/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala b/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala index e3aa1a418b..cf550c7c35 100644 --- a/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala @@ -14,11 +14,11 @@ import io.iohk.ethereum.network.PeerManagerActor.PeerConfiguration import io.iohk.ethereum.network.handshaker.Handshaker.HandshakeComplete.{HandshakeFailure, HandshakeSuccess} import io.iohk.ethereum.network.p2p.messages.Capability.Capabilities._ import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.Status.StatusEnc -import io.iohk.ethereum.network.p2p.messages.PV62.GetBlockHeaders.GetBlockHeadersEnc -import io.iohk.ethereum.network.p2p.messages.PV62.{BlockHeaders, GetBlockHeaders} +import io.iohk.ethereum.network.p2p.messages.ETH62.GetBlockHeaders.GetBlockHeadersEnc +import io.iohk.ethereum.network.p2p.messages.ETH62.{BlockHeaders, GetBlockHeaders} import io.iohk.ethereum.network.p2p.messages.WireProtocol.Hello.HelloEnc import io.iohk.ethereum.network.p2p.messages.WireProtocol.{Disconnect, Hello} -import io.iohk.ethereum.network.p2p.messages.{Capability, BaseETH6XMessages, PV64, ProtocolVersions} +import io.iohk.ethereum.network.p2p.messages.{Capability, BaseETH6XMessages, ETC64, ProtocolVersions} import io.iohk.ethereum.utils._ import io.iohk.ethereum.security.SecureRandomBuilder import io.iohk.ethereum.utils.ByteStringUtils._ @@ -27,8 +27,8 @@ import org.scalatest.matchers.should.Matchers class EtcHandshakerSpec extends AnyFlatSpec with Matchers { - it should "correctly connect during an appropriate handshake if no fork resolver is used" in new LocalPeerPV63Setup - with RemotePeerPV63Setup { + it should "correctly connect during an appropriate handshake if no fork resolver is used" in new LocalPeerETH63Setup + with RemotePeerETH63Setup { initHandshakerWithoutResolver.nextMessage.map(_.messageToSend) shouldBe Right(localHello: HelloEnc) val handshakerAfterHelloOpt = initHandshakerWithoutResolver.applyMessage(remoteHello) @@ -58,8 +58,8 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { } } - it should "send status with total difficulty only when peer does not support PV64" in new LocalPeerPV63Setup - with RemotePeerPV63Setup { + it should "send status with total difficulty only when peer does not support ETC64" in new LocalPeerETH63Setup + with RemotePeerETH63Setup { val newChainWeight = ChainWeight.zero.increase(genesisBlock.header).increase(firstBlock.header) @@ -84,8 +84,8 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { } } - it should "send status with total difficulty and latest checkpoint when peer supports PV64" in new LocalPeerPV64Setup - with RemotePeerPV64Setup { + it should "send status with total difficulty and latest checkpoint when peer supports ETC64" in new LocalPeerETC64Setup + with RemotePeerETC64Setup { val newChainWeight = ChainWeight.zero.increase(genesisBlock.header).increase(firstBlock.header) @@ -116,7 +116,7 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { } it should "correctly connect during an appropriate handshake if a fork resolver is used and the remote peer has the DAO block" in new LocalPeerSetup - with RemotePeerPV63Setup { + with RemotePeerETH63Setup { val handshakerAfterHelloOpt = initHandshakerWithResolver.applyMessage(remoteHello) val handshakerAfterStatusOpt = handshakerAfterHelloOpt.get.applyMessage(remoteStatusMsg) @@ -149,7 +149,7 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { } it should "correctly connect during an appropriate handshake if a fork resolver is used and the remote peer doesn't have the DAO block" in new LocalPeerSetup - with RemotePeerPV63Setup { + with RemotePeerETH63Setup { val handshakerAfterHelloOpt = initHandshakerWithResolver.applyMessage(remoteHello) val handshakerAfterStatusOpt = handshakerAfterHelloOpt.get.applyMessage(remoteStatusMsg) @@ -188,7 +188,7 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { ) } - it should "fail if a timeout happened during status exchange" in new RemotePeerPV63Setup { + it should "fail if a timeout happened during status exchange" in new RemotePeerETH63Setup { val handshakerAfterHelloOpt = initHandshakerWithResolver.applyMessage(remoteHello) val handshakerAfterTimeout = handshakerAfterHelloOpt.get.processTimeout handshakerAfterTimeout.nextMessage.map(_.messageToSend) shouldBe Left( @@ -196,7 +196,7 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { ) } - it should "fail if a timeout happened during fork block exchange" in new RemotePeerPV63Setup { + it should "fail if a timeout happened during fork block exchange" in new RemotePeerETH63Setup { val handshakerAfterHelloOpt = initHandshakerWithResolver.applyMessage(remoteHello) val handshakerAfterStatusOpt = handshakerAfterHelloOpt.get.applyMessage(remoteStatusMsg) val handshakerAfterTimeout = handshakerAfterStatusOpt.get.processTimeout @@ -205,8 +205,8 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { ) } - it should "fail if a status msg is received with invalid network id" in new LocalPeerPV63Setup - with RemotePeerPV63Setup { + it should "fail if a status msg is received with invalid network id" in new LocalPeerETH63Setup + with RemotePeerETH63Setup { val wrongNetworkId = localStatus.networkId + 1 val handshakerAfterHelloOpt = initHandshakerWithResolver.applyMessage(remoteHello) @@ -217,8 +217,8 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { ) } - it should "fail if a status msg is received with invalid genesisHash" in new LocalPeerPV63Setup - with RemotePeerPV63Setup { + it should "fail if a status msg is received with invalid genesisHash" in new LocalPeerETH63Setup + with RemotePeerETH63Setup { val wrongGenesisHash = concatByteStrings((localStatus.genesisHash.head + 1).toByte, localStatus.genesisHash.tail) val handshakerAfterHelloOpt = initHandshakerWithResolver.applyMessage(remoteHello) @@ -229,17 +229,17 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { ) } - it should "fail if the remote peer doesn't support PV63/PV64" in new RemotePeerPV63Setup { - val pv62Capability = ProtocolVersions.PV62 + it should "fail if the remote peer doesn't support ETH63/ETC64" in new RemotePeerETH63Setup { + val eth62Capability = ProtocolVersions.ETH62 val handshakerAfterHelloOpt = - initHandshakerWithResolverETH62.applyMessage(remoteHello.copy(capabilities = Seq(pv62Capability))) + initHandshakerWithResolverETH62.applyMessage(remoteHello.copy(capabilities = Seq(eth62Capability))) assert(handshakerAfterHelloOpt.isDefined) handshakerAfterHelloOpt.get.nextMessage.leftSide shouldBe Left( HandshakeFailure(Disconnect.Reasons.IncompatibleP2pProtocolVersion) ) } - it should "fail if a fork resolver is used and the block from the remote peer isn't accepted" in new RemotePeerPV63Setup { + it should "fail if a fork resolver is used and the block from the remote peer isn't accepted" in new RemotePeerETH63Setup { val handshakerAfterHelloOpt = initHandshakerWithResolver.applyMessage(remoteHello) val handshakerAfterStatusOpt = handshakerAfterHelloOpt.get.applyMessage(remoteStatusMsg) val handshakerAfterForkBlockOpt = handshakerAfterStatusOpt.get.applyMessage( @@ -286,15 +286,15 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { } val initHandshakerWithoutResolver = EtcHandshaker( - new MockEtcHandshakerConfiguration(List(ProtocolVersions.PV64, ProtocolVersions.PV63)), ProtocolVersions.PV63 + new MockEtcHandshakerConfiguration(List(ProtocolVersions.ETC64, ProtocolVersions.ETH63)), ProtocolVersions.ETH63 ) val initHandshakerWithoutResolverETC64 = EtcHandshaker( - new MockEtcHandshakerConfiguration(List(ProtocolVersions.PV64, ProtocolVersions.PV63)), ProtocolVersions.PV64 + new MockEtcHandshakerConfiguration(List(ProtocolVersions.ETC64, ProtocolVersions.ETH63)), ProtocolVersions.ETC64 ) - val initHandshakerWithResolver = EtcHandshaker(etcHandshakerConfigurationWithResolver, ProtocolVersions.PV63) + val initHandshakerWithResolver = EtcHandshaker(etcHandshakerConfigurationWithResolver, ProtocolVersions.ETH63) - val initHandshakerWithResolverETH62 = EtcHandshaker(etcHandshakerConfigurationWithResolver, ProtocolVersions.PV62) + val initHandshakerWithResolverETH62 = EtcHandshaker(etcHandshakerConfigurationWithResolver, ProtocolVersions.ETH62) val firstBlock = genesisBlock.copy(header = genesisBlock.header.copy(parentHash = genesisBlock.header.hash, number = 1)) @@ -313,9 +313,9 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { GetBlockHeaders(Left(forkBlockHeader.number), maxHeaders = 1, skip = 0, reverse = false) } - trait LocalPeerPV63Setup extends LocalPeerSetup { + trait LocalPeerETH63Setup extends LocalPeerSetup { val localStatusMsg = BaseETH6XMessages.Status( - protocolVersion = ProtocolVersions.PV63.version, + protocolVersion = ProtocolVersions.ETH63.version, networkId = Config.Network.peer.networkId, totalDifficulty = genesisBlock.header.difficulty, bestHash = genesisBlock.header.hash, @@ -324,9 +324,9 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { val localStatus = RemoteStatus(localStatusMsg) } - trait LocalPeerPV64Setup extends LocalPeerSetup { - val localStatusMsg = PV64.Status( - protocolVersion = ProtocolVersions.PV64.version, + trait LocalPeerETC64Setup extends LocalPeerSetup { + val localStatusMsg = ETC64.Status( + protocolVersion = ProtocolVersions.ETC64.version, networkId = Config.Network.peer.networkId, chainWeight = ChainWeight.zero.increase(genesisBlock.header), bestHash = genesisBlock.header.hash, @@ -344,7 +344,7 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { val remotePort = 8545 } - trait RemotePeerPV63Setup extends RemotePeerSetup { + trait RemotePeerETH63Setup extends RemotePeerSetup { val remoteHello = Hello( p2pVersion = EtcHelloExchangeState.P2pVersion, clientId = "remote-peer", @@ -354,7 +354,7 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { ) val remoteStatusMsg = BaseETH6XMessages.Status( - protocolVersion = ProtocolVersions.PV63.version, + protocolVersion = ProtocolVersions.ETH63.version, networkId = Config.Network.peer.networkId, totalDifficulty = 0, bestHash = genesisBlock.header.hash, @@ -364,7 +364,7 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { val remoteStatus = RemoteStatus(remoteStatusMsg) } - trait RemotePeerPV64Setup extends RemotePeerSetup { + trait RemotePeerETC64Setup extends RemotePeerSetup { val remoteHello = Hello( p2pVersion = EtcHelloExchangeState.P2pVersion, clientId = "remote-peer", @@ -374,8 +374,8 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { ) val remoteStatusMsg = - PV64.Status( - protocolVersion = ProtocolVersions.PV64.version, + ETC64.Status( + protocolVersion = ProtocolVersions.ETC64.version, networkId = Config.Network.peer.networkId, chainWeight = ChainWeight.zero, bestHash = genesisBlock.header.hash, diff --git a/src/test/scala/io/iohk/ethereum/network/p2p/MessageCodecSpec.scala b/src/test/scala/io/iohk/ethereum/network/p2p/MessageCodecSpec.scala index 57d5767e80..96a99fa4d0 100644 --- a/src/test/scala/io/iohk/ethereum/network/p2p/MessageCodecSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/p2p/MessageCodecSpec.scala @@ -91,7 +91,7 @@ class MessageCodecSpec extends AnyFlatSpec with Matchers { val helloV4 = helloV5.copy(p2pVersion = 4) val status = Status( - protocolVersion = ProtocolVersions.PV63.version, + protocolVersion = ProtocolVersions.ETH63.version, networkId = Config.Network.peer.networkId, totalDifficulty = 1, bestHash = ByteString(1), @@ -100,8 +100,8 @@ class MessageCodecSpec extends AnyFlatSpec with Matchers { val decoder = NetworkMessageDecoder orElse EthereumMessageDecoder - val messageCodec = new MessageCodec(frameCodec, decoder, ProtocolVersions.PV63, negotiatedLocalP2PVersion) - val remoteMessageCodec = new MessageCodec(remoteFrameCodec, decoder, ProtocolVersions.PV63, negotiatedRemoteP2PVersion) + val messageCodec = new MessageCodec(frameCodec, decoder, ProtocolVersions.ETH63, negotiatedLocalP2PVersion) + val remoteMessageCodec = new MessageCodec(remoteFrameCodec, decoder, ProtocolVersions.ETH63, negotiatedRemoteP2PVersion) } diff --git a/src/test/scala/io/iohk/ethereum/network/p2p/MessageDecodersSpec.scala b/src/test/scala/io/iohk/ethereum/network/p2p/MessageDecodersSpec.scala index 1639f7a448..3007fb4a5e 100644 --- a/src/test/scala/io/iohk/ethereum/network/p2p/MessageDecodersSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/p2p/MessageDecodersSpec.scala @@ -19,7 +19,7 @@ class MessageDecodersSpec extends AnyFlatSpec with Matchers with SecureRandomBui val blockHashesFromNumberBytes: Array[Byte] = Hex.decode("c20c28") - val NewBlockHashesPV61bytes: Array[Byte] = + val NewBlockHashesETH61bytes: Array[Byte] = Hex.decode( "f842a0fccdbfe911f9df0a6cc0107d1240f76dfdd1d301b65fdc3cd2ae62752affbef6a0fccdbfe911f9df0a6cc0107d1240f76dfdd1d301b65fdc3cd2ae62752affbef6" ) @@ -40,156 +40,156 @@ class MessageDecodersSpec extends AnyFlatSpec with Matchers with SecureRandomBui ) ) ) - NetworkMessageDecoder.fromBytes(WireProtocol.Hello.code, helloBytes, ProtocolVersions.PV61) shouldBe hello - NetworkMessageDecoder.fromBytes(WireProtocol.Hello.code, helloBytes, ProtocolVersions.PV62) shouldBe hello - NetworkMessageDecoder.fromBytes(WireProtocol.Hello.code, helloBytes, ProtocolVersions.PV63) shouldBe hello - NetworkMessageDecoder.fromBytes(WireProtocol.Hello.code, helloBytes, ProtocolVersions.PV64) shouldBe hello + NetworkMessageDecoder.fromBytes(WireProtocol.Hello.code, helloBytes, ProtocolVersions.ETH61) shouldBe hello + NetworkMessageDecoder.fromBytes(WireProtocol.Hello.code, helloBytes, ProtocolVersions.ETH62) shouldBe hello + NetworkMessageDecoder.fromBytes(WireProtocol.Hello.code, helloBytes, ProtocolVersions.ETH63) shouldBe hello + NetworkMessageDecoder.fromBytes(WireProtocol.Hello.code, helloBytes, ProtocolVersions.ETC64) shouldBe hello } it should "decode NewBlockHashes message for all supported versions of protocol" in { - val newBlockHashesPV61 = PV61.NewBlockHashes(Seq(exampleHash, exampleHash)) + val newBlockHashesETH61 = ETH61.NewBlockHashes(Seq(exampleHash, exampleHash)) - val NewBlockHashesPV62bytes: Array[Byte] = + val NewBlockHashesETH62bytes: Array[Byte] = Hex.decode( "f846e2a0fccdbfe911f9df0a6cc0107d1240f76dfdd1d301b65fdc3cd2ae62752affbef601e2a0fccdbfe911f9df0a6cc0107d1240f76dfdd1d301b65fdc3cd2ae62752affbef602" ) - val newBlockHashesPV62 = PV62.NewBlockHashes(Seq(PV62.BlockHash(exampleHash, 1), PV62.BlockHash(exampleHash, 2))) + val newBlockHashesETH62 = ETH62.NewBlockHashes(Seq(ETH62.BlockHash(exampleHash, 1), ETH62.BlockHash(exampleHash, 2))) - decode(Codes.NewBlockHashesCode, NewBlockHashesPV61bytes, ProtocolVersions.PV61) shouldBe newBlockHashesPV61 - decode(Codes.NewBlockHashesCode, NewBlockHashesPV62bytes, ProtocolVersions.PV62) shouldBe newBlockHashesPV62 - decode(Codes.NewBlockHashesCode, NewBlockHashesPV62bytes, ProtocolVersions.PV63) shouldBe newBlockHashesPV62 - decode(Codes.NewBlockHashesCode, NewBlockHashesPV62bytes, ProtocolVersions.PV64) shouldBe newBlockHashesPV62 + decode(Codes.NewBlockHashesCode, NewBlockHashesETH61bytes, ProtocolVersions.ETH61) shouldBe newBlockHashesETH61 + decode(Codes.NewBlockHashesCode, NewBlockHashesETH62bytes, ProtocolVersions.ETH62) shouldBe newBlockHashesETH62 + decode(Codes.NewBlockHashesCode, NewBlockHashesETH62bytes, ProtocolVersions.ETH63) shouldBe newBlockHashesETH62 + decode(Codes.NewBlockHashesCode, NewBlockHashesETH62bytes, ProtocolVersions.ETC64) shouldBe newBlockHashesETH62 } it should "not decode message from older version of protocol as newer version" in { assertThrows[RuntimeException] { - decode(Codes.NewBlockHashesCode, NewBlockHashesPV61bytes, ProtocolVersions.PV62) + decode(Codes.NewBlockHashesCode, NewBlockHashesETH61bytes, ProtocolVersions.ETH62) } } it should "decode BlockHashesFromNumber message for all supported versions of protocol" in { - val blockHashesFromNumber = PV61.BlockHashesFromNumber(12, 40) + val blockHashesFromNumber = ETH61.BlockHashesFromNumber(12, 40) decode( Codes.BlockHashesFromNumberCode, blockHashesFromNumberBytes, - ProtocolVersions.PV61 + ProtocolVersions.ETH61 ) shouldBe blockHashesFromNumber } it should "decode GetBlockHeaders message for all supported versions of protocol" in { - val getBlockHeaders = PV62.GetBlockHeaders(Left(1), 1, 1, false) + val getBlockHeaders = ETH62.GetBlockHeaders(Left(1), 1, 1, false) val getBlockHeadersBytes: Array[Byte] = getBlockHeaders.toBytes assertThrows[RuntimeException] { - decode(Codes.GetBlockHeadersCode, getBlockHeadersBytes, ProtocolVersions.PV61) + decode(Codes.GetBlockHeadersCode, getBlockHeadersBytes, ProtocolVersions.ETH61) } - decode(Codes.GetBlockHeadersCode, getBlockHeadersBytes, ProtocolVersions.PV62) shouldBe getBlockHeaders - decode(Codes.GetBlockHeadersCode, getBlockHeadersBytes, ProtocolVersions.PV63) shouldBe getBlockHeaders - decode(Codes.GetBlockHeadersCode, getBlockHeadersBytes, ProtocolVersions.PV64) shouldBe getBlockHeaders + decode(Codes.GetBlockHeadersCode, getBlockHeadersBytes, ProtocolVersions.ETH62) shouldBe getBlockHeaders + decode(Codes.GetBlockHeadersCode, getBlockHeadersBytes, ProtocolVersions.ETH63) shouldBe getBlockHeaders + decode(Codes.GetBlockHeadersCode, getBlockHeadersBytes, ProtocolVersions.ETC64) shouldBe getBlockHeaders } it should "decode BlockHeaders message for all supported versions of protocol" in { - val blockHeaders = PV62.BlockHeaders(ObjectGenerators.seqBlockHeaderGen.sample.get) + val blockHeaders = ETH62.BlockHeaders(ObjectGenerators.seqBlockHeaderGen.sample.get) val blockHeadersBytes: Array[Byte] = blockHeaders.toBytes assertThrows[RuntimeException] { - decode(Codes.BlockHeadersCode, blockHeadersBytes, ProtocolVersions.PV61) + decode(Codes.BlockHeadersCode, blockHeadersBytes, ProtocolVersions.ETH61) } - decode(Codes.BlockHeadersCode, blockHeadersBytes, ProtocolVersions.PV62) shouldBe blockHeaders - decode(Codes.BlockHeadersCode, blockHeadersBytes, ProtocolVersions.PV63) shouldBe blockHeaders - decode(Codes.BlockHeadersCode, blockHeadersBytes, ProtocolVersions.PV64) shouldBe blockHeaders + decode(Codes.BlockHeadersCode, blockHeadersBytes, ProtocolVersions.ETH62) shouldBe blockHeaders + decode(Codes.BlockHeadersCode, blockHeadersBytes, ProtocolVersions.ETH63) shouldBe blockHeaders + decode(Codes.BlockHeadersCode, blockHeadersBytes, ProtocolVersions.ETC64) shouldBe blockHeaders } it should "decode GetBlockBodies message for all supported versions of protocol" in { - val getBlockBodies = PV62.GetBlockBodies(Seq(exampleHash)) + val getBlockBodies = ETH62.GetBlockBodies(Seq(exampleHash)) val getBlockBodiesBytes: Array[Byte] = getBlockBodies.toBytes assertThrows[RuntimeException] { - decode(Codes.GetBlockBodiesCode, getBlockBodiesBytes, ProtocolVersions.PV61) + decode(Codes.GetBlockBodiesCode, getBlockBodiesBytes, ProtocolVersions.ETH61) } - decode(Codes.GetBlockBodiesCode, getBlockBodiesBytes, ProtocolVersions.PV62) shouldBe getBlockBodies - decode(Codes.GetBlockBodiesCode, getBlockBodiesBytes, ProtocolVersions.PV63) shouldBe getBlockBodies - decode(Codes.GetBlockBodiesCode, getBlockBodiesBytes, ProtocolVersions.PV64) shouldBe getBlockBodies + decode(Codes.GetBlockBodiesCode, getBlockBodiesBytes, ProtocolVersions.ETH62) shouldBe getBlockBodies + decode(Codes.GetBlockBodiesCode, getBlockBodiesBytes, ProtocolVersions.ETH63) shouldBe getBlockBodies + decode(Codes.GetBlockBodiesCode, getBlockBodiesBytes, ProtocolVersions.ETC64) shouldBe getBlockBodies } it should "decode BlockBodies message for all supported versions of protocol" in { - val blockBodies = PV62.BlockBodies(Seq(Fixtures.Blocks.Block3125369.body, Fixtures.Blocks.DaoForkBlock.body)) + val blockBodies = ETH62.BlockBodies(Seq(Fixtures.Blocks.Block3125369.body, Fixtures.Blocks.DaoForkBlock.body)) val blockBodiesBytes: Array[Byte] = blockBodies.toBytes assertThrows[RuntimeException] { - decode(Codes.BlockBodiesCode, blockBodiesBytes, ProtocolVersions.PV61) + decode(Codes.BlockBodiesCode, blockBodiesBytes, ProtocolVersions.ETH61) } - decode(Codes.BlockBodiesCode, blockBodiesBytes, ProtocolVersions.PV62) shouldBe blockBodies - decode(Codes.BlockBodiesCode, blockBodiesBytes, ProtocolVersions.PV63) shouldBe blockBodies - decode(Codes.BlockBodiesCode, blockBodiesBytes, ProtocolVersions.PV64) shouldBe blockBodies + decode(Codes.BlockBodiesCode, blockBodiesBytes, ProtocolVersions.ETH62) shouldBe blockBodies + decode(Codes.BlockBodiesCode, blockBodiesBytes, ProtocolVersions.ETH63) shouldBe blockBodies + decode(Codes.BlockBodiesCode, blockBodiesBytes, ProtocolVersions.ETC64) shouldBe blockBodies } it should "decode GetNodeData message for all supported versions of protocol" in { - val getNodeData = PV63.GetNodeData(Seq(exampleHash)) + val getNodeData = ETH63.GetNodeData(Seq(exampleHash)) val getNodeDataBytes: Array[Byte] = getNodeData.toBytes assertThrows[RuntimeException] { - decode(Codes.GetNodeDataCode, getNodeDataBytes, ProtocolVersions.PV61) + decode(Codes.GetNodeDataCode, getNodeDataBytes, ProtocolVersions.ETH61) } assertThrows[RuntimeException] { - decode(Codes.GetNodeDataCode, getNodeDataBytes, ProtocolVersions.PV62) + decode(Codes.GetNodeDataCode, getNodeDataBytes, ProtocolVersions.ETH62) } - decode(Codes.GetNodeDataCode, getNodeDataBytes, ProtocolVersions.PV63) shouldBe getNodeData - decode(Codes.GetNodeDataCode, getNodeDataBytes, ProtocolVersions.PV64) shouldBe getNodeData + decode(Codes.GetNodeDataCode, getNodeDataBytes, ProtocolVersions.ETH63) shouldBe getNodeData + decode(Codes.GetNodeDataCode, getNodeDataBytes, ProtocolVersions.ETC64) shouldBe getNodeData } it should "decode NodeData message for all supported versions of protocol" in { - val nodeData = PV63.NodeData(Seq(exampleHash)) + val nodeData = ETH63.NodeData(Seq(exampleHash)) val nodeDataBytes: Array[Byte] = nodeData.toBytes assertThrows[RuntimeException] { - decode(Codes.NodeDataCode, nodeDataBytes, ProtocolVersions.PV61) + decode(Codes.NodeDataCode, nodeDataBytes, ProtocolVersions.ETH61) } assertThrows[RuntimeException] { - decode(Codes.NodeDataCode, nodeDataBytes, ProtocolVersions.PV62) + decode(Codes.NodeDataCode, nodeDataBytes, ProtocolVersions.ETH62) } - decode(Codes.NodeDataCode, nodeDataBytes, ProtocolVersions.PV63) shouldBe nodeData - decode(Codes.NodeDataCode, nodeDataBytes, ProtocolVersions.PV64) shouldBe nodeData + decode(Codes.NodeDataCode, nodeDataBytes, ProtocolVersions.ETH63) shouldBe nodeData + decode(Codes.NodeDataCode, nodeDataBytes, ProtocolVersions.ETC64) shouldBe nodeData } it should "decode GetReceipts message for all supported versions of protocol" in { - val getReceipts = PV63.GetReceipts(Seq(exampleHash)) + val getReceipts = ETH63.GetReceipts(Seq(exampleHash)) val getReceiptsBytes: Array[Byte] = getReceipts.toBytes assertThrows[RuntimeException] { - decode(Codes.GetReceiptsCode, getReceiptsBytes, ProtocolVersions.PV61) + decode(Codes.GetReceiptsCode, getReceiptsBytes, ProtocolVersions.ETH61) } assertThrows[RuntimeException] { - decode(Codes.GetReceiptsCode, getReceiptsBytes, ProtocolVersions.PV62) + decode(Codes.GetReceiptsCode, getReceiptsBytes, ProtocolVersions.ETH62) } - decode(Codes.GetReceiptsCode, getReceiptsBytes, ProtocolVersions.PV63) shouldBe getReceipts - decode(Codes.GetReceiptsCode, getReceiptsBytes, ProtocolVersions.PV64) shouldBe getReceipts + decode(Codes.GetReceiptsCode, getReceiptsBytes, ProtocolVersions.ETH63) shouldBe getReceipts + decode(Codes.GetReceiptsCode, getReceiptsBytes, ProtocolVersions.ETC64) shouldBe getReceipts } it should "decode Receipts message for all supported versions of protocol" in { - val receipts = PV63.Receipts(ObjectGenerators.receiptsGen(3).sample.get) + val receipts = ETH63.Receipts(ObjectGenerators.receiptsGen(3).sample.get) val receiptsBytes: Array[Byte] = receipts.toBytes assertThrows[RuntimeException] { - decode(Codes.ReceiptsCode, receiptsBytes, ProtocolVersions.PV61) + decode(Codes.ReceiptsCode, receiptsBytes, ProtocolVersions.ETH61) } assertThrows[RuntimeException] { - decode(Codes.ReceiptsCode, receiptsBytes, ProtocolVersions.PV62) + decode(Codes.ReceiptsCode, receiptsBytes, ProtocolVersions.ETH62) } - decode(Codes.ReceiptsCode, receiptsBytes, ProtocolVersions.PV63) shouldBe receipts - decode(Codes.ReceiptsCode, receiptsBytes, ProtocolVersions.PV64) shouldBe receipts + decode(Codes.ReceiptsCode, receiptsBytes, ProtocolVersions.ETH63) shouldBe receipts + decode(Codes.ReceiptsCode, receiptsBytes, ProtocolVersions.ETC64) shouldBe receipts } it should "decode Status message for all supported versions of protocol" in { - val status63 = BaseETH6XMessages.Status(ProtocolVersions.PV63.version, 1, BigInt(100), exampleHash, exampleHash) + val status63 = BaseETH6XMessages.Status(ProtocolVersions.ETH63.version, 1, BigInt(100), exampleHash, exampleHash) val status63Bytes: Array[Byte] = status63.toBytes - val status64 = PV64.Status(ProtocolVersions.PV63.version, 1, ChainWeight(1, BigInt(100)), exampleHash, exampleHash) + val status64 = ETC64.Status(ProtocolVersions.ETH63.version, 1, ChainWeight(1, BigInt(100)), exampleHash, exampleHash) - // it's not 100 % true as Status message was different in PV61, but we are not supporting old message - decode(Codes.StatusCode, status63Bytes, ProtocolVersions.PV61) shouldBe status63 - decode(Codes.StatusCode, status63Bytes, ProtocolVersions.PV62) shouldBe status63 - decode(Codes.StatusCode, status63Bytes, ProtocolVersions.PV63) shouldBe status63 - decode(Codes.StatusCode, status64.toBytes, ProtocolVersions.PV64) shouldBe status64 + // it's not 100 % true as Status message was different in ETH61, but we are not supporting old message + decode(Codes.StatusCode, status63Bytes, ProtocolVersions.ETH61) shouldBe status63 + decode(Codes.StatusCode, status63Bytes, ProtocolVersions.ETH62) shouldBe status63 + decode(Codes.StatusCode, status63Bytes, ProtocolVersions.ETH63) shouldBe status63 + decode(Codes.StatusCode, status64.toBytes, ProtocolVersions.ETC64) shouldBe status64 } it should "decode NewBlock message for all supported versions of protocol" in { @@ -197,31 +197,31 @@ class MessageDecodersSpec extends AnyFlatSpec with Matchers with SecureRandomBui val newBlock63Bytes: Array[Byte] = newBlock63.toBytes val newBlock64 = ObjectGenerators.newBlock64Gen(secureRandom, None).sample.get - decode(Codes.NewBlockCode, newBlock63Bytes, ProtocolVersions.PV61) shouldBe newBlock63 - decode(Codes.NewBlockCode, newBlock63Bytes, ProtocolVersions.PV62) shouldBe newBlock63 - decode(Codes.NewBlockCode, newBlock63Bytes, ProtocolVersions.PV63) shouldBe newBlock63 - decode(Codes.NewBlockCode, newBlock64.toBytes, ProtocolVersions.PV64) shouldBe newBlock64 + decode(Codes.NewBlockCode, newBlock63Bytes, ProtocolVersions.ETH61) shouldBe newBlock63 + decode(Codes.NewBlockCode, newBlock63Bytes, ProtocolVersions.ETH62) shouldBe newBlock63 + decode(Codes.NewBlockCode, newBlock63Bytes, ProtocolVersions.ETH63) shouldBe newBlock63 + decode(Codes.NewBlockCode, newBlock64.toBytes, ProtocolVersions.ETC64) shouldBe newBlock64 } it should "decode SignedTransactions message for all supported versions of protocol" in { val signedTransactions = SignedTransactions(ObjectGenerators.signedTxSeqGen(3, secureRandom, None).sample.get) val signedTransactionsBytes: Array[Byte] = signedTransactions.toBytes - decode(Codes.SignedTransactionsCode, signedTransactionsBytes, ProtocolVersions.PV61) shouldBe signedTransactions - decode(Codes.SignedTransactionsCode, signedTransactionsBytes, ProtocolVersions.PV62) shouldBe signedTransactions - decode(Codes.SignedTransactionsCode, signedTransactionsBytes, ProtocolVersions.PV63) shouldBe signedTransactions - decode(Codes.SignedTransactionsCode, signedTransactionsBytes, ProtocolVersions.PV64) shouldBe signedTransactions + decode(Codes.SignedTransactionsCode, signedTransactionsBytes, ProtocolVersions.ETH61) shouldBe signedTransactions + decode(Codes.SignedTransactionsCode, signedTransactionsBytes, ProtocolVersions.ETH62) shouldBe signedTransactions + decode(Codes.SignedTransactionsCode, signedTransactionsBytes, ProtocolVersions.ETH63) shouldBe signedTransactions + decode(Codes.SignedTransactionsCode, signedTransactionsBytes, ProtocolVersions.ETC64) shouldBe signedTransactions } it should "not decode message not existing in given protocol" in { assertThrows[RuntimeException] { - decode(Codes.SignedTransactionsCode, blockHashesFromNumberBytes, ProtocolVersions.PV62) + decode(Codes.SignedTransactionsCode, blockHashesFromNumberBytes, ProtocolVersions.ETH62) } } it should "not decode message of not supported protocol" in { assertThrows[RuntimeException] { - decode(Codes.NewBlockHashesCode, NewBlockHashesPV61bytes, ProtocolVersions.PV61.copy(version = 60)) + decode(Codes.NewBlockHashesCode, NewBlockHashesETH61bytes, ProtocolVersions.ETH61.copy(version = 60)) } } } diff --git a/src/test/scala/io/iohk/ethereum/network/p2p/PeerActorSpec.scala b/src/test/scala/io/iohk/ethereum/network/p2p/PeerActorSpec.scala index 5b5547f8c4..15c4258427 100644 --- a/src/test/scala/io/iohk/ethereum/network/p2p/PeerActorSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/p2p/PeerActorSpec.scala @@ -20,9 +20,9 @@ import io.iohk.ethereum.network.handshaker.{EtcHandshaker, EtcHandshakerConfigur import io.iohk.ethereum.network.p2p.messages.Capability.Capabilities._ import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.Status import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.Status.StatusEnc -import io.iohk.ethereum.network.p2p.messages.PV62.GetBlockHeaders.GetBlockHeadersEnc -import io.iohk.ethereum.network.p2p.messages.PV62._ -import io.iohk.ethereum.network.p2p.messages.{Capability, PV64, ProtocolVersions} +import io.iohk.ethereum.network.p2p.messages.ETH62.GetBlockHeaders.GetBlockHeadersEnc +import io.iohk.ethereum.network.p2p.messages.ETH62._ +import io.iohk.ethereum.network.p2p.messages.{Capability, ETC64, ProtocolVersions} import io.iohk.ethereum.network.p2p.messages.WireProtocol.Disconnect.{DisconnectEnc, Reasons} import io.iohk.ethereum.network.p2p.messages.WireProtocol.Hello.HelloEnc import io.iohk.ethereum.network.p2p.messages.WireProtocol.Pong.PongEnc @@ -79,7 +79,7 @@ class PeerActorSpec it should "try to reconnect on broken rlpx connection" in new NodeStatusSetup with HandshakerSetup { override implicit lazy val system = ActorSystem("PeerActorSpec_System") - override def protocol: Capability = ProtocolVersions.PV63 + override def protocol: Capability = ProtocolVersions.ETH63 val time = new VirtualTime @@ -135,7 +135,7 @@ class PeerActorSpec rlpxConnection.send(peer, RLPxConnectionHandler.MessageReceived(remoteHello)) val remoteStatus = Status( - protocolVersion = ProtocolVersions.PV63.version, + protocolVersion = ProtocolVersions.ETH63.version, networkId = peerConf.networkId, totalDifficulty = daoForkBlockChainTotalDifficulty + 100000, // remote is after the fork bestHash = ByteString("blockhash"), @@ -173,7 +173,7 @@ class PeerActorSpec rlpxConnection.send(peer, RLPxConnectionHandler.MessageReceived(remoteHello)) val remoteStatus = Status( - protocolVersion = ProtocolVersions.PV63.version, + protocolVersion = ProtocolVersions.ETH63.version, networkId = peerConf.networkId, totalDifficulty = daoForkBlockChainTotalDifficulty + 100000, // remote is after the fork bestHash = ByteString("blockhash"), @@ -188,7 +188,7 @@ class PeerActorSpec } it should "successfully connect to ETC peer with protocol 64" in new TestSetup { - override def protocol: Capability = ProtocolVersions.PV64 + override def protocol: Capability = ProtocolVersions.ETC64 val uri = new URI(s"enode://${Hex.toHexString(remoteNodeId.toArray[Byte])}@localhost:9000") val completeUri = new URI(s"enode://${Hex.toHexString(remoteNodeId.toArray[Byte])}@127.0.0.1:9000?discport=9000") peer ! PeerActor.ConnectTo(uri) @@ -202,8 +202,8 @@ class PeerActorSpec rlpxConnection.expectMsgPF() { case RLPxConnectionHandler.SendMessage(_: HelloEnc) => () } rlpxConnection.send(peer, RLPxConnectionHandler.MessageReceived(remoteHello)) - val remoteStatus = PV64.Status( - protocolVersion = ProtocolVersions.PV64.version, + val remoteStatus = ETC64.Status( + protocolVersion = ProtocolVersions.ETC64.version, networkId = peerConf.networkId, chainWeight = ChainWeight.totalDifficultyOnly(daoForkBlockChainTotalDifficulty + 100000), // remote is after the fork @@ -212,7 +212,7 @@ class PeerActorSpec ) //Node status exchange - rlpxConnection.expectMsgPF() { case RLPxConnectionHandler.SendMessage(_: PV64.Status.StatusEnc) => () } + rlpxConnection.expectMsgPF() { case RLPxConnectionHandler.SendMessage(_: ETC64.Status.StatusEnc) => () } rlpxConnection.send(peer, RLPxConnectionHandler.MessageReceived(remoteStatus)) //Fork block exchange @@ -242,7 +242,7 @@ class PeerActorSpec rlpxConnection.send(peer, RLPxConnectionHandler.MessageReceived(remoteHello)) val remoteStatus = Status( - protocolVersion = ProtocolVersions.PV63.version, + protocolVersion = ProtocolVersions.ETH63.version, networkId = peerConf.networkId, totalDifficulty = daoForkBlockChainTotalDifficulty + 100000, // remote is after the fork bestHash = ByteString("blockhash"), @@ -285,7 +285,7 @@ class PeerActorSpec .commit() val remoteStatus = Status( - protocolVersion = ProtocolVersions.PV63.version, + protocolVersion = ProtocolVersions.ETH63.version, networkId = peerConf.networkId, totalDifficulty = daoForkBlockChainTotalDifficulty + 100000, // remote is after the fork bestHash = ByteString("blockhash"), @@ -311,7 +311,7 @@ class PeerActorSpec rlpxConnection.send(peer, RLPxConnectionHandler.MessageReceived(remoteHello)) val remoteStatus = Status( - protocolVersion = ProtocolVersions.PV63.version, + protocolVersion = ProtocolVersions.ETH63.version, networkId = peerConf.networkId, totalDifficulty = daoForkBlockChainTotalDifficulty + 100000, // remote is after the fork bestHash = ByteString("blockhash"), @@ -358,7 +358,7 @@ class PeerActorSpec rlpxConnection.send(peer, RLPxConnectionHandler.MessageReceived(remoteHello)) val remoteStatus = Status( - protocolVersion = ProtocolVersions.PV63.version, + protocolVersion = ProtocolVersions.ETH63.version, networkId = peerConf.networkId, totalDifficulty = daoForkBlockChainTotalDifficulty + 100000, // remote is after the fork bestHash = ByteString("blockhash"), @@ -390,7 +390,7 @@ class PeerActorSpec rlpxConnection.send(peer, RLPxConnectionHandler.MessageReceived(remoteHello)) val remoteStatus = Status( - protocolVersion = ProtocolVersions.PV63.version, + protocolVersion = ProtocolVersions.ETH63.version, networkId = peerConf.networkId, totalDifficulty = daoForkBlockChainTotalDifficulty + 100000, // remote is after the fork bestHash = ByteString("blockhash"), @@ -409,7 +409,7 @@ class PeerActorSpec it should "stay connected to pre fork peer" in new TestSetup { val remoteStatus = RemoteStatus( - protocolVersion = ProtocolVersions.PV63.version, + protocolVersion = ProtocolVersions.ETH63.version, networkId = peerConf.networkId, chainWeight = ChainWeight.totalDifficultyOnly(daoForkBlockChainTotalDifficulty - 200000), // remote is before the fork @@ -452,7 +452,7 @@ class PeerActorSpec rlpxConnection.send(peer, RLPxConnectionHandler.MessageReceived(remoteHello)) val remoteStatus = Status( - protocolVersion = ProtocolVersions.PV63.version, + protocolVersion = ProtocolVersions.ETH63.version, networkId = peerConf.networkId, totalDifficulty = daoForkBlockChainTotalDifficulty + 100000, // remote is after the fork bestHash = ByteString("blockhash"), @@ -571,7 +571,7 @@ class PeerActorSpec } trait TestSetup extends NodeStatusSetup with BlockUtils with HandshakerSetup { - override def protocol: Capability = ProtocolVersions.PV63 + override def protocol: Capability = ProtocolVersions.ETH63 val genesisHash = genesisBlock.hash diff --git a/src/test/scala/io/iohk/ethereum/network/p2p/messages/MessagesSerializationSpec.scala b/src/test/scala/io/iohk/ethereum/network/p2p/messages/MessagesSerializationSpec.scala index d07d738bdd..bbe8a179b4 100644 --- a/src/test/scala/io/iohk/ethereum/network/p2p/messages/MessagesSerializationSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/p2p/messages/MessagesSerializationSpec.scala @@ -4,8 +4,8 @@ import akka.util.ByteString import io.iohk.ethereum.Fixtures import io.iohk.ethereum.domain.ChainWeight import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages._ -import io.iohk.ethereum.network.p2p.messages.PV61.BlockHashesFromNumber -import io.iohk.ethereum.network.p2p.messages.PV62._ +import io.iohk.ethereum.network.p2p.messages.ETH61.BlockHashesFromNumber +import io.iohk.ethereum.network.p2p.messages.ETH62._ import io.iohk.ethereum.network.p2p.messages.WireProtocol._ import io.iohk.ethereum.network.p2p.{EthereumMessageDecoder, NetworkMessageDecoder} import org.scalatest.matchers.should.Matchers @@ -14,7 +14,7 @@ import org.scalatestplus.scalacheck.ScalaCheckPropertyChecks class MessagesSerializationSpec extends AnyWordSpec with ScalaCheckPropertyChecks with Matchers { - // TODO: add tests for messages from PV63 + // TODO: add tests for messages from ETH63 "Wire Protocol" when { "encoding and decoding Hello" should { @@ -23,7 +23,7 @@ class MessagesSerializationSpec extends AnyWordSpec with ScalaCheckPropertyCheck Hello(1, "teest", Seq(Capability("Sample", 1), Capability("Sample", 2)), 1, ByteString("Id")), (m: Hello) => m.toBytes, Hello.code, - ProtocolVersions.PV63 + ProtocolVersions.ETH63 ) } } @@ -34,20 +34,20 @@ class MessagesSerializationSpec extends AnyWordSpec with ScalaCheckPropertyCheck Disconnect(Disconnect.Reasons.AlreadyConnected), (m: Disconnect) => m.toBytes, Disconnect.code, - ProtocolVersions.PV63 + ProtocolVersions.ETH63 ) } } "encoding and decoding Ping" should { "return same result" in { - verify(Ping(), (m: Ping) => m.toBytes, Ping.code, ProtocolVersions.PV63) + verify(Ping(), (m: Ping) => m.toBytes, Ping.code, ProtocolVersions.ETH63) } } "encoding and decoding Pong" should { "return same result" in { - verify(Pong(), (m: Pong) => m.toBytes, Pong.code, ProtocolVersions.PV63) + verify(Pong(), (m: Pong) => m.toBytes, Pong.code, ProtocolVersions.ETH63) } } } @@ -56,47 +56,47 @@ class MessagesSerializationSpec extends AnyWordSpec with ScalaCheckPropertyCheck "encoding and decoding Status" should { "return same result for Status v63" in { val msg = Status(1, 2, 2, ByteString("HASH"), ByteString("HASH2")) - verify(msg, (m: Status) => m.toBytes, Codes.StatusCode, ProtocolVersions.PV63) + verify(msg, (m: Status) => m.toBytes, Codes.StatusCode, ProtocolVersions.ETH63) } } "encoding and decoding SignedTransactions" should { "return same result" in { val msg = SignedTransactions(Fixtures.Blocks.Block3125369.body.transactionList) - verify(msg, (m: SignedTransactions) => m.toBytes, Codes.SignedTransactionsCode, ProtocolVersions.PV63) + verify(msg, (m: SignedTransactions) => m.toBytes, Codes.SignedTransactionsCode, ProtocolVersions.ETH63) } } "encoding and decoding NewBlock" should { "return same result for NewBlock v63" in { val msg = NewBlock(Fixtures.Blocks.Block3125369.block, 2323) - verify(msg, (m: NewBlock) => m.toBytes, Codes.NewBlockCode, ProtocolVersions.PV63) + verify(msg, (m: NewBlock) => m.toBytes, Codes.NewBlockCode, ProtocolVersions.ETH63) } } } - "PV64" when { + "ETC64" when { "encoding and decoding Status" should { "return same result for Status v64" in { - val msg = PV64.Status(1, 2, ChainWeight(2, 5), ByteString("HASH"), ByteString("HASH2")) - verify(msg, (m: PV64.Status) => m.toBytes, Codes.StatusCode, ProtocolVersions.PV64) + val msg = ETC64.Status(1, 2, ChainWeight(2, 5), ByteString("HASH"), ByteString("HASH2")) + verify(msg, (m: ETC64.Status) => m.toBytes, Codes.StatusCode, ProtocolVersions.ETC64) } } "encoding and decoding NewBlock" should { "return same result for NewBlock v64" in { - val msg = PV64.NewBlock(Fixtures.Blocks.Block3125369.block, ChainWeight(2323, 21)) - verify(msg, (m: PV64.NewBlock) => m.toBytes, Codes.NewBlockCode, ProtocolVersions.PV64) + val msg = ETC64.NewBlock(Fixtures.Blocks.Block3125369.block, ChainWeight(2323, 21)) + verify(msg, (m: ETC64.NewBlock) => m.toBytes, Codes.NewBlockCode, ProtocolVersions.ETC64) } } } - "PV61" when { - val version = ProtocolVersions.PV61 + "ETH61" when { + val version = ProtocolVersions.ETH61 "encoding and decoding NewBlockHashes" should { "return same result" in { - val msg = PV61.NewBlockHashes(Seq(ByteString("23"), ByteString("10"), ByteString("36"))) - verify(msg, (m: PV61.NewBlockHashes) => m.toBytes, Codes.NewBlockHashesCode, version) + val msg = ETH61.NewBlockHashes(Seq(ByteString("23"), ByteString("10"), ByteString("36"))) + verify(msg, (m: ETH61.NewBlockHashes) => m.toBytes, Codes.NewBlockHashesCode, version) } } @@ -108,12 +108,12 @@ class MessagesSerializationSpec extends AnyWordSpec with ScalaCheckPropertyCheck } } - "PV62" when { - val version = ProtocolVersions.PV62 + "ETH62" when { + val version = ProtocolVersions.ETH62 "encoding and decoding NewBlockHashes" should { "return same result" in { - val msg = PV62.NewBlockHashes(Seq(BlockHash(ByteString("hash1"), 1), BlockHash(ByteString("hash2"), 2))) - verify(msg, (m: PV62.NewBlockHashes) => m.toBytes, Codes.NewBlockHashesCode, version) + val msg = ETH62.NewBlockHashes(Seq(BlockHash(ByteString("hash1"), 1), BlockHash(ByteString("hash2"), 2))) + verify(msg, (m: ETH62.NewBlockHashes) => m.toBytes, Codes.NewBlockHashesCode, version) } } diff --git a/src/test/scala/io/iohk/ethereum/network/p2p/messages/NewBlockSpec.scala b/src/test/scala/io/iohk/ethereum/network/p2p/messages/NewBlockSpec.scala index 2b17ee5994..5365828c03 100644 --- a/src/test/scala/io/iohk/ethereum/network/p2p/messages/NewBlockSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/p2p/messages/NewBlockSpec.scala @@ -23,10 +23,10 @@ class NewBlockSpec extends AnyFunSuite with ScalaCheckPropertyChecks with Object } test("NewBlock v64 messages are encoded and decoded properly") { - import io.iohk.ethereum.network.p2p.messages.PV64.NewBlock._ + import io.iohk.ethereum.network.p2p.messages.ETC64.NewBlock._ forAll(newBlock64Gen(secureRandom, Some(chainId))) { newBlock => val encoded: Array[Byte] = newBlock.toBytes - val decoded: PV64.NewBlock = encoded.toNewBlock + val decoded: ETC64.NewBlock = encoded.toNewBlock assert(decoded == newBlock) } } @@ -67,7 +67,7 @@ class NewBlockSpec extends AnyFunSuite with ScalaCheckPropertyChecks with Object 983040 ) - val newBlock64 = PV64.NewBlock( + val newBlock64 = ETC64.NewBlock( Block( BlockHeader( parentHash = ByteString(Hex.decode("98352d9c1300bd82334cb3e5034c3ec622d437963f55cf5a00a49642806c2f32")), diff --git a/src/test/scala/io/iohk/ethereum/network/p2p/messages/NodeDataSpec.scala b/src/test/scala/io/iohk/ethereum/network/p2p/messages/NodeDataSpec.scala index 2c3434801a..60d89b9042 100644 --- a/src/test/scala/io/iohk/ethereum/network/p2p/messages/NodeDataSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/p2p/messages/NodeDataSpec.scala @@ -5,8 +5,8 @@ import io.iohk.ethereum.crypto._ import io.iohk.ethereum.domain.Account import io.iohk.ethereum.mpt.{BranchNode, ExtensionNode, HashNode, LeafNode, MptNode, NullNode} import io.iohk.ethereum.mpt.HexPrefix.{bytesToNibbles, encode => hpEncode} -import io.iohk.ethereum.network.p2p.messages.PV63._ -import io.iohk.ethereum.network.p2p.messages.PV63.MptNodeEncoders._ +import io.iohk.ethereum.network.p2p.messages.ETH63._ +import io.iohk.ethereum.network.p2p.messages.ETH63.MptNodeEncoders._ import io.iohk.ethereum.rlp.RLPImplicitConversions._ import io.iohk.ethereum.rlp.RLPImplicits._ import io.iohk.ethereum.rlp.{encode, _} @@ -79,7 +79,7 @@ class NodeDataSpec extends AnyFlatSpec with Matchers { } it should "be decoded properly" in { - val result = EthereumMessageDecoder.fromBytes(Codes.NodeDataCode, encode(encodedNodeData), ProtocolVersions.PV63) + val result = EthereumMessageDecoder.fromBytes(Codes.NodeDataCode, encode(encodedNodeData), ProtocolVersions.ETH63) result match { case m: NodeData => @@ -93,7 +93,7 @@ class NodeDataSpec extends AnyFlatSpec with Matchers { } it should "be decoded previously encoded value" in { - EthereumMessageDecoder.fromBytes(Codes.NodeDataCode, nodeData.toBytes, ProtocolVersions.PV63) shouldBe nodeData + EthereumMessageDecoder.fromBytes(Codes.NodeDataCode, nodeData.toBytes, ProtocolVersions.ETH63) shouldBe nodeData } it should "decode branch node with values in leafs that looks like RLP list" in { diff --git a/src/test/scala/io/iohk/ethereum/network/p2p/messages/ReceiptsSpec.scala b/src/test/scala/io/iohk/ethereum/network/p2p/messages/ReceiptsSpec.scala index fb058c7e44..da9e819b85 100644 --- a/src/test/scala/io/iohk/ethereum/network/p2p/messages/ReceiptsSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/p2p/messages/ReceiptsSpec.scala @@ -4,7 +4,7 @@ import akka.util.ByteString import io.iohk.ethereum.crypto._ import io.iohk.ethereum.domain.{Address, Receipt, TxLogEntry} import io.iohk.ethereum.network.p2p.EthereumMessageDecoder -import io.iohk.ethereum.network.p2p.messages.PV63.Receipts +import io.iohk.ethereum.network.p2p.messages.ETH63.Receipts import io.iohk.ethereum.rlp.RLPImplicitConversions._ import io.iohk.ethereum.rlp.RLPImplicits._ import io.iohk.ethereum.rlp._ @@ -54,11 +54,11 @@ class ReceiptsSpec extends AnyFlatSpec with Matchers { EthereumMessageDecoder.fromBytes( Codes.ReceiptsCode, encode(encodedReceipts), - ProtocolVersions.PV63 + ProtocolVersions.ETH63 ) shouldBe receipts } it should "decode encoded receipts" in { - EthereumMessageDecoder.fromBytes(Codes.ReceiptsCode, receipts.toBytes, ProtocolVersions.PV63) shouldBe receipts + EthereumMessageDecoder.fromBytes(Codes.ReceiptsCode, receipts.toBytes, ProtocolVersions.ETH63) shouldBe receipts } } diff --git a/src/test/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandlerSpec.scala b/src/test/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandlerSpec.scala index 479f4d41ec..b77ff4ef23 100644 --- a/src/test/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandlerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandlerSpec.scala @@ -179,7 +179,7 @@ class RLPxConnectionHandlerSpec override def fromBytes(`type`: Int, payload: Array[Byte], protocolVersion: Capability) = throw new Exception("Mock message decoder fails to decode all messages") } - val protocolVersion = ProtocolVersions.PV63 + val protocolVersion = ProtocolVersions.ETH63 val mockHandshaker = mock[AuthHandshaker] val connection = TestProbe() val mockMessageCodec = mock[MessageCodec] From 01168bc0df8d2b3521d497c7d0f619b5784913fb Mon Sep 17 00:00:00 2001 From: Dominik Zajkowski Date: Wed, 9 Jun 2021 16:02:01 +0200 Subject: [PATCH 10/15] [ETCM-841] Apply scalafmt --- src/main/scala/io/iohk/ethereum/network/PeerActor.scala | 7 ++++++- .../network/handshaker/EtcHelloExchangeState.scala | 4 +++- .../io/iohk/ethereum/network/rlpx/MessageCodec.scala | 7 ++++++- .../ethereum/network/rlpx/RLPxConnectionHandler.scala | 9 +++++++-- 4 files changed, 22 insertions(+), 5 deletions(-) diff --git a/src/main/scala/io/iohk/ethereum/network/PeerActor.scala b/src/main/scala/io/iohk/ethereum/network/PeerActor.scala index 2f32e1ff02..39be9d6565 100644 --- a/src/main/scala/io/iohk/ethereum/network/PeerActor.scala +++ b/src/main/scala/io/iohk/ethereum/network/PeerActor.scala @@ -83,7 +83,12 @@ class PeerActor[R <: HandshakeResult]( case RLPxConnectionHandler.ConnectionEstablished(remoteNodeId, capability) => val newUri = rlpxConnection.uriOpt.map(outGoingUri => modifyOutGoingUri(remoteNodeId, rlpxConnection, outGoingUri)) - processHandshakerNextMessage(initHandshaker(capability), remoteNodeId, rlpxConnection.copy(uriOpt = newUri), numRetries) + processHandshakerNextMessage( + initHandshaker(capability), + remoteNodeId, + rlpxConnection.copy(uriOpt = newUri), + numRetries + ) case RLPxConnectionHandler.ConnectionFailed => log.debug("Failed to establish RLPx connection") diff --git a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala index fc223aea5f..5d3ad99955 100644 --- a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala +++ b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala @@ -30,7 +30,9 @@ case class EtcHelloExchangeState(handshakerConfiguration: EtcHandshakerConfigura case ProtocolVersions.ETC64 => EtcNodeStatus64ExchangeState(handshakerConfiguration) case ProtocolVersions.ETH63 => EtcNodeStatus63ExchangeState(handshakerConfiguration) case _ => - log.debug(s"Connected peer does not support ${ProtocolVersions.ETH63} / ${ProtocolVersions.ETC64} protocol. Disconnecting.") + log.debug( + s"Connected peer does not support ${ProtocolVersions.ETH63} / ${ProtocolVersions.ETC64} protocol. Disconnecting." + ) DisconnectedState(Disconnect.Reasons.IncompatibleP2pProtocolVersion) } } diff --git a/src/main/scala/io/iohk/ethereum/network/rlpx/MessageCodec.scala b/src/main/scala/io/iohk/ethereum/network/rlpx/MessageCodec.scala index ebd67d14e9..f1a5ae7691 100644 --- a/src/main/scala/io/iohk/ethereum/network/rlpx/MessageCodec.scala +++ b/src/main/scala/io/iohk/ethereum/network/rlpx/MessageCodec.scala @@ -10,7 +10,12 @@ import org.xerial.snappy.Snappy import scala.util.{Failure, Success, Try} -class MessageCodec(frameCodec: FrameCodec, messageDecoder: MessageDecoder, protocolVersion: Capability, val remotePeer2PeerVersion: Long) { +class MessageCodec( + frameCodec: FrameCodec, + messageDecoder: MessageDecoder, + protocolVersion: Capability, + val remotePeer2PeerVersion: Long +) { val MaxFramePayloadSize: Int = Int.MaxValue // no framing diff --git a/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala b/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala index cffcbf3e34..2ac3d03d0d 100644 --- a/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala +++ b/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala @@ -311,7 +311,12 @@ object RLPxConnectionHandler { ): Props = Props( new RLPxConnectionHandler( - messageDecoder, capabilities, authHandshaker, messageCodecFactory, rlpxConfiguration, HelloExtractor.apply + messageDecoder, + capabilities, + authHandshaker, + messageCodecFactory, + rlpxConfiguration, + HelloExtractor.apply ) ) @@ -359,7 +364,7 @@ object RLPxConnectionHandler { private def extractHello(frame: Frame): Option[Hello] = { val frameData = frame.payload.toArray - if(frame.`type` == Hello.code) { + if (frame.`type` == Hello.code) { val m = NetworkMessageDecoder.fromBytes(frame.`type`, frameData, Capability.Capabilities.Eth63Capability) Some(m.asInstanceOf[Hello]) } else { From 0a7ef9a3a8204f3aac26174684930479ec9f7c2c Mon Sep 17 00:00:00 2001 From: Dominik Zajkowski Date: Thu, 10 Jun 2021 00:41:15 +0200 Subject: [PATCH 11/15] [ETCM-841] Handle 'Hello' negotiation as a connection stage --- .../ethereum/sync/util/CommonFakePeer.scala | 2 +- .../txExecTest/util/DumpChainApp.scala | 2 +- .../io/iohk/ethereum/network/PeerActor.scala | 11 +- .../ethereum/network/PeerManagerActor.scala | 4 +- .../network/handshaker/EtcHandshaker.scala | 4 +- .../handshaker/EtcHelloExchangeState.scala | 9 +- .../ethereum/network/rlpx/MessageCodec.scala | 14 ++- .../network/rlpx/RLPxConnectionHandler.scala | 116 ++++++++++++------ .../ethereum/nodebuilder/NodeBuilder.scala | 3 +- .../network/PeerActorHandshakingSpec.scala | 26 ++-- .../handshaker/EtcHandshakerSpec.scala | 17 +-- .../ethereum/network/p2p/PeerActorSpec.scala | 30 ++--- 12 files changed, 139 insertions(+), 99 deletions(-) diff --git a/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala b/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala index dd1e2f3e8b..5143c2491a 100644 --- a/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala +++ b/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala @@ -172,7 +172,7 @@ abstract class CommonFakePeer(peerName: String, fakePeerCustomConfig: FakePeerCu override val capabilities: List[Capability] = blockchainConfig.capabilities } - lazy val handshaker: Capability => Handshaker[PeerInfo] = c => EtcHandshaker(handshakerConfiguration, c) + lazy val handshaker: Handshaker[PeerInfo] = EtcHandshaker(handshakerConfiguration) lazy val authHandshaker: AuthHandshaker = AuthHandshaker(nodeKey, secureRandom) diff --git a/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala b/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala index a5989e9c34..98fd09b2d8 100644 --- a/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala +++ b/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala @@ -96,7 +96,7 @@ object DumpChainApp extends App with NodeKeyBuilder with SecureRandomBuilder wit override val capabilities: List[Capability] = blockchainConfig.capabilities } - lazy val handshaker: Capability => Handshaker[PeerInfo] = c => EtcHandshaker(handshakerConfiguration, c) + lazy val handshaker: Handshaker[PeerInfo] = EtcHandshaker(handshakerConfiguration) val peerMessageBus = actorSystem.actorOf(PeerEventBusActor.props) diff --git a/src/main/scala/io/iohk/ethereum/network/PeerActor.scala b/src/main/scala/io/iohk/ethereum/network/PeerActor.scala index 39be9d6565..0cf31581fb 100644 --- a/src/main/scala/io/iohk/ethereum/network/PeerActor.scala +++ b/src/main/scala/io/iohk/ethereum/network/PeerActor.scala @@ -12,7 +12,8 @@ import io.iohk.ethereum.network.handshaker.Handshaker import io.iohk.ethereum.network.handshaker.Handshaker.HandshakeComplete.{HandshakeFailure, HandshakeSuccess} import io.iohk.ethereum.network.handshaker.Handshaker.{HandshakeResult, NextMessage} import io.iohk.ethereum.network.p2p._ -import io.iohk.ethereum.network.p2p.messages.Capability +import io.iohk.ethereum.network.p2p.messages.Capability.Capabilities +import io.iohk.ethereum.network.p2p.messages.{Capability, ETH63} import io.iohk.ethereum.network.p2p.messages.WireProtocol._ import io.iohk.ethereum.network.rlpx.RLPxConnectionHandler.RLPxConfiguration import io.iohk.ethereum.network.rlpx.{AuthHandshaker, RLPxConnectionHandler} @@ -34,7 +35,7 @@ class PeerActor[R <: HandshakeResult]( knownNodesManager: ActorRef, incomingConnection: Boolean, externalSchedulerOpt: Option[Scheduler] = None, - initHandshaker: Capability => Handshaker[R] + initHandshaker: Handshaker[R] ) extends Actor with ActorLogging with Stash { @@ -80,11 +81,11 @@ class PeerActor[R <: HandshakeResult]( def waitingForConnectionResult(rlpxConnection: RLPxConnection, numRetries: Int = 0): Receive = handleTerminated(rlpxConnection, numRetries, Connecting) orElse stashMessages orElse { - case RLPxConnectionHandler.ConnectionEstablished(remoteNodeId, capability) => + case RLPxConnectionHandler.ConnectionEstablished(remoteNodeId) => val newUri = rlpxConnection.uriOpt.map(outGoingUri => modifyOutGoingUri(remoteNodeId, rlpxConnection, outGoingUri)) processHandshakerNextMessage( - initHandshaker(capability), + initHandshaker, remoteNodeId, rlpxConnection.copy(uriOpt = newUri), numRetries @@ -304,7 +305,7 @@ object PeerActor { peerEventBus: ActorRef, knownNodesManager: ActorRef, incomingConnection: Boolean, - handshaker: Capability => Handshaker[R], + handshaker: Handshaker[R], authHandshaker: AuthHandshaker, messageDecoder: MessageDecoder, capabilities: List[Capability] diff --git a/src/main/scala/io/iohk/ethereum/network/PeerManagerActor.scala b/src/main/scala/io/iohk/ethereum/network/PeerManagerActor.scala index 11941124e4..250a7f3f3c 100644 --- a/src/main/scala/io/iohk/ethereum/network/PeerManagerActor.scala +++ b/src/main/scala/io/iohk/ethereum/network/PeerManagerActor.scala @@ -419,7 +419,7 @@ object PeerManagerActor { peerMessageBus: ActorRef, knownNodesManager: ActorRef, peerStatistics: ActorRef, - handshaker: Capability => Handshaker[R], + handshaker: Handshaker[R], authHandshaker: AuthHandshaker, messageDecoder: MessageDecoder, discoveryConfig: DiscoveryConfig, @@ -456,7 +456,7 @@ object PeerManagerActor { config: PeerConfiguration, eventBus: ActorRef, knownNodesManager: ActorRef, - handshaker: Capability => Handshaker[R], + handshaker: Handshaker[R], authHandshaker: AuthHandshaker, messageDecoder: MessageDecoder, capabilities: List[Capability] diff --git a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHandshaker.scala b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHandshaker.scala index 742da29bf5..0da7c37db0 100644 --- a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHandshaker.scala +++ b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHandshaker.scala @@ -22,8 +22,8 @@ case class EtcHandshaker private ( object EtcHandshaker { - def apply(handshakerConfiguration: EtcHandshakerConfiguration, capability: Capability): EtcHandshaker = { - val initialState = EtcHelloExchangeState(handshakerConfiguration, capability) + def apply(handshakerConfiguration: EtcHandshakerConfiguration): EtcHandshaker = { + val initialState = EtcHelloExchangeState(handshakerConfiguration) EtcHandshaker(initialState, handshakerConfiguration) } diff --git a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala index 5d3ad99955..c5ecf3bb1e 100644 --- a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala +++ b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala @@ -10,7 +10,7 @@ import io.iohk.ethereum.network.p2p.messages.{Capability, ProtocolVersions} import io.iohk.ethereum.network.p2p.messages.WireProtocol.{Disconnect, Hello} import io.iohk.ethereum.utils.{Config, Logger, ServerStatus} -case class EtcHelloExchangeState(handshakerConfiguration: EtcHandshakerConfiguration, capability: Capability) +case class EtcHelloExchangeState(handshakerConfiguration: EtcHandshakerConfiguration) extends InProgressState[PeerInfo] with Logger { @@ -26,9 +26,10 @@ case class EtcHelloExchangeState(handshakerConfiguration: EtcHandshakerConfigura override def applyResponseMessage: PartialFunction[Message, HandshakerState[PeerInfo]] = { case hello: Hello => log.debug("Protocol handshake finished with peer ({})", hello) - capability match { - case ProtocolVersions.ETC64 => EtcNodeStatus64ExchangeState(handshakerConfiguration) - case ProtocolVersions.ETH63 => EtcNodeStatus63ExchangeState(handshakerConfiguration) + // FIXME in principle this should be already negotiated + Capability.negotiate(hello.capabilities.toList, handshakerConfiguration.capabilities) match { + case Some(ProtocolVersions.ETC64) => EtcNodeStatus64ExchangeState(handshakerConfiguration) + case Some(ProtocolVersions.ETH63) => EtcNodeStatus63ExchangeState(handshakerConfiguration) case _ => log.debug( s"Connected peer does not support ${ProtocolVersions.ETH63} / ${ProtocolVersions.ETC64} protocol. Disconnecting." diff --git a/src/main/scala/io/iohk/ethereum/network/rlpx/MessageCodec.scala b/src/main/scala/io/iohk/ethereum/network/rlpx/MessageCodec.scala index f1a5ae7691..64819d7c18 100644 --- a/src/main/scala/io/iohk/ethereum/network/rlpx/MessageCodec.scala +++ b/src/main/scala/io/iohk/ethereum/network/rlpx/MessageCodec.scala @@ -10,20 +10,22 @@ import org.xerial.snappy.Snappy import scala.util.{Failure, Success, Try} +object MessageCodec { + val MaxFramePayloadSize: Int = Int.MaxValue // no framing + // 16Mb in base 2 + val MaxDecompressedLength = 16777216 +} + class MessageCodec( frameCodec: FrameCodec, messageDecoder: MessageDecoder, protocolVersion: Capability, val remotePeer2PeerVersion: Long ) { - - val MaxFramePayloadSize: Int = Int.MaxValue // no framing + import MessageCodec._ val contextIdCounter = new AtomicInteger - // 16Mb in base 2 - val maxDecompressedLength = 16777216 - // TODO: ETCM-402 - messageDecoder should use negotiated protocol version def readMessages(data: ByteString): Seq[Try[Message]] = { val frames = frameCodec.readFrames(data) @@ -48,7 +50,7 @@ class MessageCodec( private def decompressData(data: Array[Byte]): Try[Array[Byte]] = { Try(Snappy.uncompressedLength(data)).flatMap { decompressedSize => - if (decompressedSize > maxDecompressedLength) + if (decompressedSize > MaxDecompressedLength) Failure(new RuntimeException("Message size larger than 16mb")) else Try(Snappy.uncompress(data)) diff --git a/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala b/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala index 2ac3d03d0d..058547b84c 100644 --- a/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala +++ b/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala @@ -6,8 +6,9 @@ import akka.io.Tcp._ import akka.io.{IO, Tcp} import akka.util.ByteString import cats.data.NonEmptyList -import io.iohk.ethereum.network.p2p.messages.Capability +import io.iohk.ethereum.network.p2p.messages.{Capability, WireProtocol} import io.iohk.ethereum.network.p2p.messages.WireProtocol.Hello +import io.iohk.ethereum.network.p2p.messages.WireProtocol.Hello.HelloEnc import io.iohk.ethereum.network.p2p.{Message, MessageDecoder, MessageSerializable, NetworkMessageDecoder} import io.iohk.ethereum.network.rlpx.RLPxConnectionHandler.{HelloExtractor, RLPxConfiguration} import io.iohk.ethereum.utils.ByteUtils @@ -152,31 +153,11 @@ class RLPxConnectionHandler( def processHandshakeResult(result: AuthHandshakeResult, remainingData: ByteString): Unit = result match { case AuthHandshakeSuccess(secrets, remotePubKey) => - log.info(s"Auth handshake succeeded for peer $peerId") - val e = extractor(secrets) - val messageCodecOpt = for { - r <- e.readHello(remainingData) - (hello, restFrames) = r - protocolVersion <- Capability.negotiate(hello.capabilities.toList, capabilities) - p2pVersion = hello.p2pVersion - messageCodec = messageCodecFactory(e.frameCodec, messageDecoder, protocolVersion, p2pVersion) - _ = context.parent ! ConnectionEstablished(remotePubKey, protocolVersion) - _ = context.parent ! MessageReceived(hello) - _ = { - if (restFrames.nonEmpty) { - val messagesSoFar = messageCodec.readFrames(restFrames) // omit hello - messagesSoFar foreach processMessage - } - } - } yield messageCodec - messageCodecOpt match { - case Some(messageCodec) => - context become handshaked(messageCodec) - case None => - log.debug(s"[Stopping Connection] Unable to connect to $peerId") - context.parent ! ConnectionFailed - context stop self - } + log.debug(s"Auth handshake succeeded for peer $peerId") + context.parent ! ConnectionEstablished(remotePubKey) + if (remainingData.nonEmpty) + context.self ! Received(remainingData) + context become awaitHello(extractor(secrets)) case AuthHandshakeError => log.debug(s"[Stopping Connection] Auth handshake failed for peer $peerId") @@ -184,6 +165,57 @@ class RLPxConnectionHandler( context stop self } + def awaitHello(extractor: HelloExtractor, cancellableAckTimeout: Option[CancellableAckTimeout] = None, seqNumber: Int = 0): Receive = + handleWriteFailed orElse handleConnectionClosed orElse { + case SendMessage(h: HelloEnc) => + val out = extractor.writeHello(h) + connection ! Write(out, Ack) + val timeout = system.scheduler.scheduleOnce(rlpxConfiguration.waitForTcpAckTimeout, self, AckTimeout(seqNumber)) + context become awaitHello(extractor, Some(CancellableAckTimeout(seqNumber, timeout)), increaseSeqNumber(seqNumber)) + + case Ack if cancellableAckTimeout.nonEmpty => + //Cancel pending message timeout + cancellableAckTimeout.foreach(_.cancellable.cancel()) + context become awaitHello(extractor, None, seqNumber) + + case AckTimeout(ackSeqNumber) if cancellableAckTimeout.exists(_.seqNumber == ackSeqNumber) => + cancellableAckTimeout.foreach(_.cancellable.cancel()) + log.error(s"[Stopping Connection] Sending 'Hello' to $peerId failed") + context stop self + + case Received(data) => + extractor.readHello(data) match { + case Some((hello, restFrames)) => + val messageCodecOpt = for { + messageCodec <- negotiateCodec(hello, extractor) + _ = context.parent ! MessageReceived(hello) + _ = processFrames(restFrames, messageCodec) + } yield messageCodec + messageCodecOpt match { + case Some(messageCodec) => + context become handshaked(messageCodec, cancellableAckTimeout = cancellableAckTimeout, seqNumber = seqNumber) + case None => + log.debug(s"[Stopping Connection] Unable to negotiate protocol with $peerId") + context.parent ! ConnectionFailed + context stop self + } + case None => + log.debug(s"[Stopping Connection] Did not find 'Hello' in message from $peerId") + context become awaitHello(extractor, cancellableAckTimeout, seqNumber) + } + } + + private def negotiateCodec(hello: Hello, extractor: HelloExtractor): Option[MessageCodec] = + Capability.negotiate(hello.capabilities.toList, capabilities).map { negotiated => + messageCodecFactory(extractor.frameCodec, messageDecoder, negotiated, hello.p2pVersion) + } + + private def processFrames(frames: Seq[Frame], messageCodec: MessageCodec): Unit = + if (frames.nonEmpty) { + val messagesSoFar = messageCodec.readFrames(frames) // omit hello + messagesSoFar foreach processMessage + } + def processMessage(messageTry: Try[Message]): Unit = messageTry match { case Success(message) => context.parent ! MessageReceived(message) @@ -299,7 +331,6 @@ class RLPxConnectionHandler( context stop self } } - } object RLPxConnectionHandler { @@ -332,7 +363,7 @@ object RLPxConnectionHandler { case class HandleConnection(connection: ActorRef) - case class ConnectionEstablished(nodeId: ByteString, negotiatedCapability: Capability) + case class ConnectionEstablished(nodeId: ByteString) case object ConnectionFailed @@ -354,21 +385,34 @@ object RLPxConnectionHandler { } case class HelloExtractor(secrets: Secrets) { + import MessageCodec._ lazy val frameCodec = new FrameCodec(secrets) def readHello(remainingData: ByteString): Option[(Hello, Seq[Frame])] = { val frames = frameCodec.readFrames(remainingData) frames.headOption.flatMap(extractHello).map(h => (h, frames.drop(1))) } - } - private def extractHello(frame: Frame): Option[Hello] = { - val frameData = frame.payload.toArray - if (frame.`type` == Hello.code) { - val m = NetworkMessageDecoder.fromBytes(frame.`type`, frameData, Capability.Capabilities.Eth63Capability) - Some(m.asInstanceOf[Hello]) - } else { - None + // 'Hello' will always fit into a frame + def writeHello(h: HelloEnc): ByteString = { + val encoded: Array[Byte] = h.toBytes + val numFrames = Math.ceil(encoded.length / MaxFramePayloadSize.toDouble).toInt + val frames = (0 until numFrames) map { frameNo => + val payload = encoded.drop(frameNo * MaxFramePayloadSize).take(MaxFramePayloadSize) + val header = Header(payload.length, 0, None, None) + Frame(header, h.code, ByteString(payload)) + } + frameCodec.writeFrames(frames) + } + + private def extractHello(frame: Frame): Option[Hello] = { + val frameData = frame.payload.toArray + if (frame.`type` == Hello.code) { + val m = NetworkMessageDecoder.fromBytes(frame.`type`, frameData, Capability.Capabilities.Eth63Capability) + Some(m.asInstanceOf[Hello]) + } else { + None + } } } } diff --git a/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala b/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala index effc144c6c..785be10f71 100644 --- a/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala +++ b/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala @@ -188,8 +188,7 @@ trait HandshakerBuilder { override val capabilities: List[Capability] = self.blockchainConfig.capabilities } - lazy val handshaker: Capability => Handshaker[PeerInfo] = (cap: Capability) => - EtcHandshaker(handshakerConfiguration, cap) + lazy val handshaker: Handshaker[PeerInfo] = EtcHandshaker(handshakerConfiguration) } trait AuthHandshakerBuilder { diff --git a/src/test/scala/io/iohk/ethereum/network/PeerActorHandshakingSpec.scala b/src/test/scala/io/iohk/ethereum/network/PeerActorHandshakingSpec.scala index a2ac9f573a..01e5e441ea 100644 --- a/src/test/scala/io/iohk/ethereum/network/PeerActorHandshakingSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/PeerActorHandshakingSpec.scala @@ -30,12 +30,12 @@ class PeerActorHandshakingSpec extends AnyFlatSpec with Matchers { import DefaultValues._ val peerActorHandshakeSucceeds = - peerActor(_ => MockHandshakerAlwaysSucceeds(defaultStatus, defaultBlockNumber, defaultForkAccepted)) + peerActor(MockHandshakerAlwaysSucceeds(defaultStatus, defaultBlockNumber, defaultForkAccepted)) //Establish probe rlpxconnection peerActorHandshakeSucceeds ! ConnectTo(uri) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.ConnectTo(uri)) - rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString(), ProtocolVersions.ETH63)) + rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString())) //Test that the handshake succeeded val sender = TestProbe()(system) @@ -47,12 +47,12 @@ class PeerActorHandshakingSpec extends AnyFlatSpec with Matchers { import DefaultValues._ - val peerActorHandshakeFails = peerActor(_ => MockHandshakerAlwaysFails(defaultReasonDisconnect)) + val peerActorHandshakeFails = peerActor(MockHandshakerAlwaysFails(defaultReasonDisconnect)) //Establish probe rlpxconnection peerActorHandshakeFails ! ConnectTo(uri) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.ConnectTo(uri)) - rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString(), ProtocolVersions.ETH63)) + rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString())) //Test that the handshake failed rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.SendMessage(Disconnect(defaultReasonDisconnect))) @@ -63,12 +63,12 @@ class PeerActorHandshakingSpec extends AnyFlatSpec with Matchers { import DefaultValues._ - val peerActorHandshakeRequiresHello = peerActor(_ => MockHandshakerRequiresHello()) + val peerActorHandshakeRequiresHello = peerActor(MockHandshakerRequiresHello()) //Establish probe rlpxconnection peerActorHandshakeRequiresHello ! ConnectTo(uri) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.ConnectTo(uri)) - rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString(), ProtocolVersions.ETH63)) + rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString())) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.SendMessage(defaultHello)) peerActorHandshakeRequiresHello ! RLPxConnectionHandler.MessageReceived(defaultHello) @@ -83,12 +83,12 @@ class PeerActorHandshakingSpec extends AnyFlatSpec with Matchers { import DefaultValues._ - val peerActorHandshakeRequiresHello = peerActor(_ => MockHandshakerRequiresHello()) + val peerActorHandshakeRequiresHello = peerActor(MockHandshakerRequiresHello()) //Establish probe rlpxconnection peerActorHandshakeRequiresHello ! ConnectTo(uri) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.ConnectTo(uri)) - rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString(), ProtocolVersions.ETH63)) + rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString())) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.SendMessage(defaultHello)) time.advance(defaultTimeout * 2) @@ -101,12 +101,12 @@ class PeerActorHandshakingSpec extends AnyFlatSpec with Matchers { import DefaultValues._ - val peerActorHandshakeRequiresHello = peerActor(_ => MockHandshakerRequiresHello()) + val peerActorHandshakeRequiresHello = peerActor(MockHandshakerRequiresHello()) //Establish probe rlpxconnection peerActorHandshakeRequiresHello ! ConnectTo(uri) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.ConnectTo(uri)) - rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString(), ProtocolVersions.ETH63)) + rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString())) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.SendMessage(defaultHello)) peerActorHandshakeRequiresHello ! RLPxConnectionHandler.MessageReceived(defaultStatusMsg) @@ -119,12 +119,12 @@ class PeerActorHandshakingSpec extends AnyFlatSpec with Matchers { import DefaultValues._ - val peerActorHandshakeRequiresHello = peerActor(_ => MockHandshakerRequiresHello()) + val peerActorHandshakeRequiresHello = peerActor(MockHandshakerRequiresHello()) //Establish probe rlpxconnection peerActorHandshakeRequiresHello ! ConnectTo(uri) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.ConnectTo(uri)) - rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString(), ProtocolVersions.ETH63)) + rlpxConnectionProbe.reply(RLPxConnectionHandler.ConnectionEstablished(ByteString())) rlpxConnectionProbe.expectMsg(RLPxConnectionHandler.SendMessage(defaultHello)) peerActorHandshakeRequiresHello ! RLPxConnectionHandler.MessageReceived(Pong()) //Ignored @@ -150,7 +150,7 @@ class PeerActorHandshakingSpec extends AnyFlatSpec with Matchers { val peerMessageBus = TestProbe() val knownNodesManager = TestProbe() - def peerActor(handshaker: Capability => Handshaker[PeerInfo]): TestActorRef[PeerActor[PeerInfo]] = TestActorRef( + def peerActor(handshaker: Handshaker[PeerInfo]): TestActorRef[PeerActor[PeerInfo]] = TestActorRef( Props( new PeerActor( new InetSocketAddress("127.0.0.1", 0), diff --git a/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala b/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala index cf550c7c35..a247306dcd 100644 --- a/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala @@ -98,9 +98,9 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { bestHash = firstBlock.header.hash ) - initHandshakerWithoutResolverETC64.nextMessage.map(_.messageToSend) shouldBe Right(localHello: HelloEnc) + initHandshakerWithoutResolver.nextMessage.map(_.messageToSend) shouldBe Right(localHello: HelloEnc) - val handshakerAfterHelloOpt = initHandshakerWithoutResolverETC64.applyMessage(remoteHello) + val handshakerAfterHelloOpt = initHandshakerWithoutResolver.applyMessage(remoteHello) assert(handshakerAfterHelloOpt.isDefined) handshakerAfterHelloOpt.get.nextMessage.map(_.messageToSend.underlyingMsg) shouldBe Right(newLocalStatusMsg) @@ -232,7 +232,7 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { it should "fail if the remote peer doesn't support ETH63/ETC64" in new RemotePeerETH63Setup { val eth62Capability = ProtocolVersions.ETH62 val handshakerAfterHelloOpt = - initHandshakerWithResolverETH62.applyMessage(remoteHello.copy(capabilities = Seq(eth62Capability))) + initHandshakerWithResolver.applyMessage(remoteHello.copy(capabilities = Seq(eth62Capability))) assert(handshakerAfterHelloOpt.isDefined) handshakerAfterHelloOpt.get.nextMessage.leftSide shouldBe Left( HandshakeFailure(Disconnect.Reasons.IncompatibleP2pProtocolVersion) @@ -285,16 +285,9 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { ) } - val initHandshakerWithoutResolver = EtcHandshaker( - new MockEtcHandshakerConfiguration(List(ProtocolVersions.ETC64, ProtocolVersions.ETH63)), ProtocolVersions.ETH63 - ) - val initHandshakerWithoutResolverETC64 = EtcHandshaker( - new MockEtcHandshakerConfiguration(List(ProtocolVersions.ETC64, ProtocolVersions.ETH63)), ProtocolVersions.ETC64 - ) - - val initHandshakerWithResolver = EtcHandshaker(etcHandshakerConfigurationWithResolver, ProtocolVersions.ETH63) + val initHandshakerWithoutResolver = EtcHandshaker(new MockEtcHandshakerConfiguration(List(ProtocolVersions.ETC64, ProtocolVersions.ETH63))) - val initHandshakerWithResolverETH62 = EtcHandshaker(etcHandshakerConfigurationWithResolver, ProtocolVersions.ETH62) + val initHandshakerWithResolver = EtcHandshaker(etcHandshakerConfigurationWithResolver) val firstBlock = genesisBlock.copy(header = genesisBlock.header.copy(parentHash = genesisBlock.header.hash, number = 1)) diff --git a/src/test/scala/io/iohk/ethereum/network/p2p/PeerActorSpec.scala b/src/test/scala/io/iohk/ethereum/network/p2p/PeerActorSpec.scala index 15c4258427..1c3a136854 100644 --- a/src/test/scala/io/iohk/ethereum/network/p2p/PeerActorSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/p2p/PeerActorSpec.scala @@ -56,7 +56,7 @@ class PeerActorSpec peer ! PeerActor.ConnectTo(new URI("encode://localhost:9000")) rlpxConnection.expectMsgClass(classOf[RLPxConnectionHandler.ConnectTo]) - rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId, protocol)) + rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId)) rlpxConnection.expectMsgPF() { case RLPxConnectionHandler.SendMessage(hello: HelloEnc) => () @@ -108,7 +108,7 @@ class PeerActorSpec peer ! PeerActor.ConnectTo(new URI("encode://localhost:9000")) rlpxConnection.expectMsgClass(classOf[RLPxConnectionHandler.ConnectTo]) - rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId, protocol)) + rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId)) rlpxConnection.expectMsgPF() { case RLPxConnectionHandler.SendMessage(hello: HelloEnc) => () @@ -127,7 +127,7 @@ class PeerActorSpec peer ! PeerActor.ConnectTo(uri) rlpxConnection.expectMsgClass(classOf[RLPxConnectionHandler.ConnectTo]) - rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId, protocol)) + rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId)) //Hello exchange val remoteHello = Hello(4, "test-client", Seq(Eth63Capability), 9000, ByteString("unused")) @@ -165,7 +165,7 @@ class PeerActorSpec peer ! PeerActor.ConnectTo(uri) rlpxConnection.expectMsgClass(classOf[RLPxConnectionHandler.ConnectTo]) - rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId, protocol)) + rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId)) //Hello exchange val remoteHello = Hello(4, "test-client", Seq(Eth63Capability), 9000, ByteString("unused")) @@ -195,7 +195,7 @@ class PeerActorSpec peer ! PeerActor.ConnectTo(uri) rlpxConnection.expectMsgClass(classOf[RLPxConnectionHandler.ConnectTo]) - rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId, protocol)) + rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId)) //Hello exchange val remoteHello = Hello(4, "test-client", Seq(Etc64Capability, Eth63Capability), 9000, ByteString("unused")) @@ -234,7 +234,7 @@ class PeerActorSpec peer ! PeerActor.ConnectTo(uri) rlpxConnection.expectMsgClass(classOf[RLPxConnectionHandler.ConnectTo]) - rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId, protocol)) + rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId)) //Hello exchange val remoteHello = Hello(4, "test-client", Seq(Eth63Capability), 9000, ByteString("unused")) @@ -269,7 +269,7 @@ class PeerActorSpec peer ! PeerActor.ConnectTo(new URI("encode://localhost:9000")) rlpxConnection.expectMsgClass(classOf[RLPxConnectionHandler.ConnectTo]) - rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId, protocol)) + rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId)) val remoteHello = Hello(4, "test-client", Seq(Eth63Capability), 9000, ByteString("unused")) rlpxConnection.expectMsgPF() { case RLPxConnectionHandler.SendMessage(_: HelloEnc) => () } @@ -304,7 +304,7 @@ class PeerActorSpec peer ! PeerActor.ConnectTo(new URI("encode://localhost:9000")) rlpxConnection.expectMsgClass(classOf[RLPxConnectionHandler.ConnectTo]) - rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId, protocol)) + rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId)) val remoteHello = Hello(4, "test-client", Seq(Eth63Capability), 9000, ByteString("unused")) rlpxConnection.expectMsgPF() { case RLPxConnectionHandler.SendMessage(_: HelloEnc) => () } @@ -333,7 +333,7 @@ class PeerActorSpec peer ! PeerActor.HandleConnection(connection.ref, new InetSocketAddress("localhost", 9000)) rlpxConnection.expectMsgClass(classOf[RLPxConnectionHandler.HandleConnection]) - rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId, protocol)) + rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId)) rlpxConnection.expectMsgPF() { case RLPxConnectionHandler.SendMessage(_: HelloEnc) => () } time.advance(5.seconds) rlpxConnection.expectMsg( @@ -351,7 +351,7 @@ class PeerActorSpec peer ! PeerActor.ConnectTo(new URI("encode://localhost:9000")) rlpxConnection.expectMsgClass(classOf[RLPxConnectionHandler.ConnectTo]) - rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId, protocol)) + rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId)) val remoteHello = Hello(4, "test-client", Seq(Eth63Capability), 9000, ByteString("unused")) rlpxConnection.expectMsgPF() { case RLPxConnectionHandler.SendMessage(_: HelloEnc) => () } @@ -383,7 +383,7 @@ class PeerActorSpec peer ! PeerActor.DisconnectPeer(Disconnect.Reasons.TooManyPeers) rlpxConnection.expectMsgClass(classOf[RLPxConnectionHandler.ConnectTo]) - rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId, protocol)) + rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId)) val remoteHello = Hello(4, "test-client", Seq(Eth63Capability), 9000, ByteString("unused")) rlpxConnection.expectMsgPF() { case RLPxConnectionHandler.SendMessage(_: HelloEnc) => () } @@ -427,7 +427,7 @@ class PeerActorSpec knownNodesManager.ref, false, None, - _ => Mocks.MockHandshakerAlwaysSucceeds(remoteStatus, 0, false) + Mocks.MockHandshakerAlwaysSucceeds(remoteStatus, 0, false) ) ) ) @@ -435,7 +435,7 @@ class PeerActorSpec peerActor ! PeerActor.ConnectTo(new URI("encode://localhost:9000")) rlpxConnection.expectMsgClass(classOf[RLPxConnectionHandler.ConnectTo]) - rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId, protocol)) + rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId)) rlpxConnection.send(peerActor, RLPxConnectionHandler.MessageReceived(Ping())) rlpxConnection.expectMsgPF() { case RLPxConnectionHandler.SendMessage(_: PongEnc) => () } @@ -445,7 +445,7 @@ class PeerActorSpec peer ! PeerActor.ConnectTo(new URI("encode://localhost:9000")) rlpxConnection.expectMsgClass(classOf[RLPxConnectionHandler.ConnectTo]) - rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId, protocol)) + rlpxConnection.reply(RLPxConnectionHandler.ConnectionEstablished(remoteNodeId)) val remoteHello = Hello(4, "test-client", Seq(Eth63Capability), 9000, ByteString("unused")) rlpxConnection.expectMsgPF() { case RLPxConnectionHandler.SendMessage(_: HelloEnc) => () } @@ -567,7 +567,7 @@ class PeerActorSpec override val capabilities: List[Capability] = List(protocol) } - val handshaker = (c: Capability) => EtcHandshaker(handshakerConfiguration, c) + val handshaker = EtcHandshaker(handshakerConfiguration) } trait TestSetup extends NodeStatusSetup with BlockUtils with HandshakerSetup { From 98ac7edc89971c95a4c532e43e1c46537f871d99 Mon Sep 17 00:00:00 2001 From: Dominik Zajkowski Date: Thu, 10 Jun 2021 00:42:59 +0200 Subject: [PATCH 12/15] [ETCM-841] Apply scalafmt --- .../network/rlpx/RLPxConnectionHandler.scala | 21 +++++++++++++++---- 1 file changed, 17 insertions(+), 4 deletions(-) diff --git a/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala b/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala index 058547b84c..15257853f2 100644 --- a/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala +++ b/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala @@ -165,13 +165,22 @@ class RLPxConnectionHandler( context stop self } - def awaitHello(extractor: HelloExtractor, cancellableAckTimeout: Option[CancellableAckTimeout] = None, seqNumber: Int = 0): Receive = + def awaitHello( + extractor: HelloExtractor, + cancellableAckTimeout: Option[CancellableAckTimeout] = None, + seqNumber: Int = 0 + ): Receive = handleWriteFailed orElse handleConnectionClosed orElse { case SendMessage(h: HelloEnc) => val out = extractor.writeHello(h) connection ! Write(out, Ack) - val timeout = system.scheduler.scheduleOnce(rlpxConfiguration.waitForTcpAckTimeout, self, AckTimeout(seqNumber)) - context become awaitHello(extractor, Some(CancellableAckTimeout(seqNumber, timeout)), increaseSeqNumber(seqNumber)) + val timeout = + system.scheduler.scheduleOnce(rlpxConfiguration.waitForTcpAckTimeout, self, AckTimeout(seqNumber)) + context become awaitHello( + extractor, + Some(CancellableAckTimeout(seqNumber, timeout)), + increaseSeqNumber(seqNumber) + ) case Ack if cancellableAckTimeout.nonEmpty => //Cancel pending message timeout @@ -193,7 +202,11 @@ class RLPxConnectionHandler( } yield messageCodec messageCodecOpt match { case Some(messageCodec) => - context become handshaked(messageCodec, cancellableAckTimeout = cancellableAckTimeout, seqNumber = seqNumber) + context become handshaked( + messageCodec, + cancellableAckTimeout = cancellableAckTimeout, + seqNumber = seqNumber + ) case None => log.debug(s"[Stopping Connection] Unable to negotiate protocol with $peerId") context.parent ! ConnectionFailed From 23dc4d21e3edb811116d09f312d0be885aa9580c Mon Sep 17 00:00:00 2001 From: Dominik Zajkowski Date: Thu, 10 Jun 2021 11:22:54 +0200 Subject: [PATCH 13/15] [ETCM-841] Add a mechanism for notifying PeerActor about 'negotiated protocol' --- .../io/iohk/ethereum/network/PeerActor.scala | 13 ++ .../network/rlpx/RLPxConnectionHandler.scala | 111 +++++++++++------- .../rlpx/RLPxConnectionHandlerSpec.scala | 9 +- 3 files changed, 85 insertions(+), 48 deletions(-) diff --git a/src/main/scala/io/iohk/ethereum/network/PeerActor.scala b/src/main/scala/io/iohk/ethereum/network/PeerActor.scala index 0cf31581fb..7334643618 100644 --- a/src/main/scala/io/iohk/ethereum/network/PeerActor.scala +++ b/src/main/scala/io/iohk/ethereum/network/PeerActor.scala @@ -120,6 +120,19 @@ class PeerActor[R <: HandshakeResult]( handleDisconnectMsg(rlpxConnection, Handshaking(numRetries)) orElse handlePingMsg(rlpxConnection) orElse stashMessages orElse { + case RLPxConnectionHandler.InitialHelloReceived(msg, negotiatedProtocol) => + // Processes the InitialHelloReceived, cancels the timeout and processes a new message but only if the handshaker + // handles the received message + // TODO pass capability to 'EtcHelloExchangeState' + // to pass negotiatedProtocol the PeerActor should be in a "post auth handshake" state + // since ConnectionEstablished(remotePubKey) was already received + // an ETCAwaitingInitialHelloHandshaker would be an approach + handshaker.applyMessage(msg).foreach { newHandshaker => + timeout.cancel() + processHandshakerNextMessage(newHandshaker, remoteNodeId, rlpxConnection, numRetries) + } + handshaker.respondToRequest(msg).foreach(msgToSend => rlpxConnection.sendMessage(msgToSend)) + case RLPxConnectionHandler.MessageReceived(msg) => // Processes the received message, cancels the timeout and processes a new message but only if the handshaker // handles the received message diff --git a/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala b/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala index 15257853f2..f41609068e 100644 --- a/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala +++ b/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala @@ -157,7 +157,9 @@ class RLPxConnectionHandler( context.parent ! ConnectionEstablished(remotePubKey) if (remainingData.nonEmpty) context.self ! Received(remainingData) - context become awaitHello(extractor(secrets)) + // following the specification at https://github.com/ethereum/devp2p/blob/master/rlpx.md#initial-handshake + // point 6 indicates that the next messages needs to be initial 'Hello' + context become awaitInitialHello(extractor(secrets)) case AuthHandshakeError => log.debug(s"[Stopping Connection] Auth handshake failed for peer $peerId") @@ -165,62 +167,79 @@ class RLPxConnectionHandler( context stop self } - def awaitHello( + def awaitInitialHello( extractor: HelloExtractor, cancellableAckTimeout: Option[CancellableAckTimeout] = None, seqNumber: Int = 0 ): Receive = - handleWriteFailed orElse handleConnectionClosed orElse { - case SendMessage(h: HelloEnc) => - val out = extractor.writeHello(h) - connection ! Write(out, Ack) - val timeout = - system.scheduler.scheduleOnce(rlpxConfiguration.waitForTcpAckTimeout, self, AckTimeout(seqNumber)) - context become awaitHello( - extractor, - Some(CancellableAckTimeout(seqNumber, timeout)), - increaseSeqNumber(seqNumber) - ) + handleWriteFailed orElse handleConnectionClosed orElse handleSendHello( + extractor, + cancellableAckTimeout, + seqNumber + ) orElse handleReceiveHello(extractor, cancellableAckTimeout, seqNumber) - case Ack if cancellableAckTimeout.nonEmpty => - //Cancel pending message timeout - cancellableAckTimeout.foreach(_.cancellable.cancel()) - context become awaitHello(extractor, None, seqNumber) + private def handleSendHello( + extractor: HelloExtractor, + cancellableAckTimeout: Option[CancellableAckTimeout] = None, + seqNumber: Int = 0 + ): Receive = { + // TODO when cancellableAckTimeout is Some + case SendMessage(h: HelloEnc) => + val out = extractor.writeHello(h) + connection ! Write(out, Ack) + val timeout = + system.scheduler.scheduleOnce(rlpxConfiguration.waitForTcpAckTimeout, self, AckTimeout(seqNumber)) + context become awaitInitialHello( + extractor, + Some(CancellableAckTimeout(seqNumber, timeout)), + increaseSeqNumber(seqNumber) + ) + case Ack if cancellableAckTimeout.nonEmpty => + //Cancel pending message timeout + cancellableAckTimeout.foreach(_.cancellable.cancel()) + context become awaitInitialHello(extractor, None, seqNumber) + + case AckTimeout(ackSeqNumber) if cancellableAckTimeout.exists(_.seqNumber == ackSeqNumber) => + cancellableAckTimeout.foreach(_.cancellable.cancel()) + log.error(s"[Stopping Connection] Sending 'Hello' to $peerId failed") + context stop self - case AckTimeout(ackSeqNumber) if cancellableAckTimeout.exists(_.seqNumber == ackSeqNumber) => - cancellableAckTimeout.foreach(_.cancellable.cancel()) - log.error(s"[Stopping Connection] Sending 'Hello' to $peerId failed") - context stop self + } - case Received(data) => - extractor.readHello(data) match { - case Some((hello, restFrames)) => - val messageCodecOpt = for { - messageCodec <- negotiateCodec(hello, extractor) - _ = context.parent ! MessageReceived(hello) - _ = processFrames(restFrames, messageCodec) - } yield messageCodec - messageCodecOpt match { - case Some(messageCodec) => - context become handshaked( - messageCodec, - cancellableAckTimeout = cancellableAckTimeout, - seqNumber = seqNumber - ) - case None => - log.debug(s"[Stopping Connection] Unable to negotiate protocol with $peerId") - context.parent ! ConnectionFailed - context stop self - } + private def handleReceiveHello( + extractor: HelloExtractor, + cancellableAckTimeout: Option[CancellableAckTimeout] = None, + seqNumber: Int = 0 + ): Receive = { case Received(data) => + extractor.readHello(data) match { + case Some((hello, restFrames)) => + val messageCodecOpt = for { + opt <- negotiateCodec(hello, extractor) + (messageCodec, negotiated) = opt + _ = context.parent ! InitialHelloReceived(hello, negotiated) + _ = processFrames(restFrames, messageCodec) + } yield messageCodec + messageCodecOpt match { + case Some(messageCodec) => + context become handshaked( + messageCodec, + cancellableAckTimeout = cancellableAckTimeout, + seqNumber = seqNumber + ) case None => - log.debug(s"[Stopping Connection] Did not find 'Hello' in message from $peerId") - context become awaitHello(extractor, cancellableAckTimeout, seqNumber) + log.debug(s"[Stopping Connection] Unable to negotiate protocol with $peerId") + context.parent ! ConnectionFailed + context stop self } + case None => + log.debug(s"[Stopping Connection] Did not find 'Hello' in message from $peerId") + context become awaitInitialHello(extractor, cancellableAckTimeout, seqNumber) } + } - private def negotiateCodec(hello: Hello, extractor: HelloExtractor): Option[MessageCodec] = + private def negotiateCodec(hello: Hello, extractor: HelloExtractor): Option[(MessageCodec, Capability)] = Capability.negotiate(hello.capabilities.toList, capabilities).map { negotiated => - messageCodecFactory(extractor.frameCodec, messageDecoder, negotiated, hello.p2pVersion) + (messageCodecFactory(extractor.frameCodec, messageDecoder, negotiated, hello.p2pVersion), negotiated) } private def processFrames(frames: Seq[Frame], messageCodec: MessageCodec): Unit = @@ -382,6 +401,8 @@ object RLPxConnectionHandler { case class MessageReceived(message: Message) + case class InitialHelloReceived(message: Hello, capability: Capability) + case class SendMessage(serializable: MessageSerializable) private case object AuthHandshakeTimeout diff --git a/src/test/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandlerSpec.scala b/src/test/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandlerSpec.scala index b77ff4ef23..127661d6f3 100644 --- a/src/test/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandlerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandlerSpec.scala @@ -10,7 +10,7 @@ import io.iohk.ethereum.{Timeouts, WithActorSystemShutDown} import io.iohk.ethereum.network.p2p.{MessageDecoder, MessageSerializable} import io.iohk.ethereum.network.p2p.messages.{Capability, ProtocolVersions} import io.iohk.ethereum.network.p2p.messages.WireProtocol.{Hello, Ping} -import io.iohk.ethereum.network.rlpx.RLPxConnectionHandler.{HelloExtractor, MessageReceived, RLPxConfiguration} +import io.iohk.ethereum.network.rlpx.RLPxConnectionHandler.{HelloExtractor, InitialHelloReceived, MessageReceived, RLPxConfiguration} import io.iohk.ethereum.security.SecureRandomBuilder import org.scalamock.scalatest.MockFactory @@ -97,7 +97,7 @@ class RLPxConnectionHandlerSpec rlpxConnection ! RLPxConnectionHandler.SendMessage(Ping()) connection.expectMsg(Tcp.Write(ByteString("ping encoded"), RLPxConnectionHandler.Ack)) - val expectedHello = rlpxConnectionParent.expectMsgType[MessageReceived] + val expectedHello = rlpxConnectionParent.expectMsgType[InitialHelloReceived] expectedHello.message shouldBe a[Hello] //The rlpx connection is closed after a timeout happens (after rlpxConfiguration.waitForTcpAckTimeout) and it is processed @@ -224,12 +224,13 @@ class RLPxConnectionHandlerSpec //AuthHandshaker handles initial message val data = ByteString((0 until AuthHandshaker.InitiatePacketLength).map(_.toByte).toArray) + val hello = ByteString((1 until AuthHandshaker.InitiatePacketLength).map(_.toByte).toArray) val response = ByteString("response data") (mockHandshaker.handleInitialMessage _) .expects(data) .returning((response, AuthHandshakeSuccess(mock[Secrets], ByteString()))) (mockHelloExtractor.readHello _) - .expects(ByteString.empty) + .expects(hello) .returning(Some(Hello(5, "", Capabilities.Eth63Capability::Nil, 30303, ByteString("abc")), Seq.empty)) (mockMessageCodec.readMessages _) .expects(ByteString.empty) @@ -238,6 +239,8 @@ class RLPxConnectionHandlerSpec rlpxConnection ! Tcp.Received(data) connection.expectMsg(Tcp.Write(response)) + rlpxConnection ! Tcp.Received(hello) + //Connection fully established rlpxConnectionParent.expectMsgClass(classOf[RLPxConnectionHandler.ConnectionEstablished]) } From 8648b6523efb794cb5d75a73a8672dae00b37535 Mon Sep 17 00:00:00 2001 From: Dominik Zajkowski Date: Thu, 10 Jun 2021 14:56:27 +0200 Subject: [PATCH 14/15] [ETCM-841] Apply PR remarks --- .../handshaker/EtcHelloExchangeState.scala | 4 ++- .../network/p2p/messages/Capability.scala | 3 +- .../network/rlpx/RLPxConnectionHandler.scala | 34 +++++++++++-------- 3 files changed, 24 insertions(+), 17 deletions(-) diff --git a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala index c5ecf3bb1e..77ed5d5a3c 100644 --- a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala +++ b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala @@ -32,7 +32,9 @@ case class EtcHelloExchangeState(handshakerConfiguration: EtcHandshakerConfigura case Some(ProtocolVersions.ETH63) => EtcNodeStatus63ExchangeState(handshakerConfiguration) case _ => log.debug( - s"Connected peer does not support ${ProtocolVersions.ETH63} / ${ProtocolVersions.ETC64} protocol. Disconnecting." + s"Connected peer does not support {} / {} protocol. Disconnecting.", + ProtocolVersions.ETH63, + ProtocolVersions.ETC64 ) DisconnectedState(Disconnect.Reasons.IncompatibleP2pProtocolVersion) } diff --git a/src/main/scala/io/iohk/ethereum/network/p2p/messages/Capability.scala b/src/main/scala/io/iohk/ethereum/network/p2p/messages/Capability.scala index 6412df0180..367a8c64df 100644 --- a/src/main/scala/io/iohk/ethereum/network/p2p/messages/Capability.scala +++ b/src/main/scala/io/iohk/ethereum/network/p2p/messages/Capability.scala @@ -7,9 +7,8 @@ import io.iohk.ethereum.rlp.{RLPEncodeable, RLPException, RLPList, RLPSerializab case class Capability(name: String, version: Byte) object Capability { - def negotiate(c1: List[Capability], c2: List[Capability]): Option[Capability] = { + def negotiate(c1: List[Capability], c2: List[Capability]): Option[Capability] = c1.intersect(c2).maxByOption(_.version) // FIXME ignores branches and other protocols - } private val pattern = "(.*)/(\\d*)".r diff --git a/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala b/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala index f41609068e..5e4ac93b51 100644 --- a/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala +++ b/src/main/scala/io/iohk/ethereum/network/rlpx/RLPxConnectionHandler.scala @@ -99,7 +99,9 @@ class RLPxConnectionHandler( case Failure(ex) => log.debug( - s"[Stopping Connection] Init AuthHandshaker message handling failed for peer $peerId due to ${ex.getMessage}" + s"[Stopping Connection] Init AuthHandshaker message handling failed for peer {} due to {}", + peerId, + ex.getMessage ) context.parent ! ConnectionFailed context stop self @@ -125,7 +127,9 @@ class RLPxConnectionHandler( case Failure(ex) => log.debug( - s"[Stopping Connection] Response AuthHandshaker message handling failed for peer $peerId due to ${ex.getMessage}" + s"[Stopping Connection] Response AuthHandshaker message handling failed for peer {} due to {}", + peerId, + ex.getMessage ) context.parent ! ConnectionFailed context stop self @@ -145,7 +149,7 @@ class RLPxConnectionHandler( } def handleTimeout: Receive = { case AuthHandshakeTimeout => - log.debug(s"[Stopping Connection] Auth handshake timeout for peer $peerId") + log.debug(s"[Stopping Connection] Auth handshake timeout for peer {}", peerId) context.parent ! ConnectionFailed context stop self } @@ -153,7 +157,7 @@ class RLPxConnectionHandler( def processHandshakeResult(result: AuthHandshakeResult, remainingData: ByteString): Unit = result match { case AuthHandshakeSuccess(secrets, remotePubKey) => - log.debug(s"Auth handshake succeeded for peer $peerId") + log.debug(s"Auth handshake succeeded for peer {}", peerId) context.parent ! ConnectionEstablished(remotePubKey) if (remainingData.nonEmpty) context.self ! Received(remainingData) @@ -162,7 +166,7 @@ class RLPxConnectionHandler( context become awaitInitialHello(extractor(secrets)) case AuthHandshakeError => - log.debug(s"[Stopping Connection] Auth handshake failed for peer $peerId") + log.debug(s"[Stopping Connection] Auth handshake failed for peer {}", peerId) context.parent ! ConnectionFailed context stop self } @@ -201,7 +205,7 @@ class RLPxConnectionHandler( case AckTimeout(ackSeqNumber) if cancellableAckTimeout.exists(_.seqNumber == ackSeqNumber) => cancellableAckTimeout.foreach(_.cancellable.cancel()) - log.error(s"[Stopping Connection] Sending 'Hello' to $peerId failed") + log.error(s"[Stopping Connection] Sending 'Hello' to {} failed", peerId) context stop self } @@ -227,12 +231,12 @@ class RLPxConnectionHandler( seqNumber = seqNumber ) case None => - log.debug(s"[Stopping Connection] Unable to negotiate protocol with $peerId") + log.debug(s"[Stopping Connection] Unable to negotiate protocol with {}", peerId) context.parent ! ConnectionFailed context stop self } case None => - log.debug(s"[Stopping Connection] Did not find 'Hello' in message from $peerId") + log.debug(s"[Stopping Connection] Did not find 'Hello' in message from {}", peerId) context become awaitInitialHello(extractor, cancellableAckTimeout, seqNumber) } } @@ -253,7 +257,7 @@ class RLPxConnectionHandler( context.parent ! MessageReceived(message) case Failure(ex) => - log.info(s"Cannot decode message from $peerId, because of ${ex.getMessage}") + log.info(s"Cannot decode message from {}, because of {}", peerId, ex.getMessage) // break connection in case of failed decoding, to avoid attack which would send us garbage context stop self } @@ -301,7 +305,7 @@ class RLPxConnectionHandler( case AckTimeout(ackSeqNumber) if cancellableAckTimeout.exists(_.seqNumber == ackSeqNumber) => cancellableAckTimeout.foreach(_.cancellable.cancel()) - log.debug(s"[Stopping Connection] Write to $peerId failed") + log.debug(s"[Stopping Connection] Write to {} failed", peerId) context stop self } } @@ -323,7 +327,7 @@ class RLPxConnectionHandler( ): Unit = { val out = messageCodec.encodeMessage(messageToSend) connection ! Write(out, Ack) - log.debug(s"Sent message: ${messageToSend.underlyingMsg.toShortString} to $peerId") + log.debug(s"Sent message: {} to {}", messageToSend.underlyingMsg.toShortString, peerId) val timeout = system.scheduler.scheduleOnce(rlpxConfiguration.waitForTcpAckTimeout, self, AckTimeout(seqNumber)) context become handshaked( @@ -347,17 +351,19 @@ class RLPxConnectionHandler( def handleWriteFailed: Receive = { case CommandFailed(cmd: Write) => log.debug( - s"[Stopping Connection] Write to peer $peerId failed, trying to send ${Hex.toHexString(cmd.data.toArray[Byte])}" + s"[Stopping Connection] Write to peer {} failed, trying to send {}", + peerId, + Hex.toHexString(cmd.data.toArray[Byte]) ) context stop self } def handleConnectionClosed: Receive = { case msg: ConnectionClosed => if (msg.isPeerClosed) { - log.debug(s"[Stopping Connection] Connection with $peerId closed by peer") + log.debug(s"[Stopping Connection] Connection with {} closed by peer", peerId) } if (msg.isErrorClosed) { - log.debug(s"[Stopping Connection] Connection with $peerId closed because of error ${msg.getErrorCause}") + log.debug(s"[Stopping Connection] Connection with {} closed because of error {}", peerId, msg.getErrorCause) } context stop self From 7f4464e4d33b0e1e00db44eca81a1d6ad180c213 Mon Sep 17 00:00:00 2001 From: Dominik Zajkowski Date: Fri, 11 Jun 2021 14:12:40 +0200 Subject: [PATCH 15/15] [ETCM-841] Apply PR remarks --- .../network/p2p/messages/Capability.scala | 16 ++++++++++++---- .../iohk/ethereum/nodebuilder/NodeBuilder.scala | 2 +- .../iohk/ethereum/utils/BlockchainConfig.scala | 2 +- 3 files changed, 14 insertions(+), 6 deletions(-) diff --git a/src/main/scala/io/iohk/ethereum/network/p2p/messages/Capability.scala b/src/main/scala/io/iohk/ethereum/network/p2p/messages/Capability.scala index 367a8c64df..96cd451ac4 100644 --- a/src/main/scala/io/iohk/ethereum/network/p2p/messages/Capability.scala +++ b/src/main/scala/io/iohk/ethereum/network/p2p/messages/Capability.scala @@ -8,17 +8,25 @@ case class Capability(name: String, version: Byte) object Capability { def negotiate(c1: List[Capability], c2: List[Capability]): Option[Capability] = - c1.intersect(c2).maxByOption(_.version) // FIXME ignores branches and other protocols + c1.intersect(c2) match { + case Nil => None + case l => Some(best(l)) + } private val pattern = "(.*)/(\\d*)".r - def from(protocolVersion: String): Capability = + def parseUnsafe(protocolVersion: String): Capability = protocolVersion match { - case pattern(name, version) => Capability(name, version.toByte) + case pattern(name, version) => + val c = Capability(name, version.toByte) + if (Capabilities.All.contains(c)) + c + else + throw new RuntimeException(s"Capability $protocolVersion not supported by Mantis") case _ => throw new RuntimeException(s"Unable to parse capability $protocolVersion") } - //TODO consider how this scoring should be handled with snap and other extended protocols + //TODO consider how this scoring should be handled with 'snap' and other extended protocols def best(capabilities: List[Capability]): Capability = capabilities.maxBy(_.version) diff --git a/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala b/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala index 785be10f71..09d70da818 100644 --- a/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala +++ b/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala @@ -248,7 +248,7 @@ trait PeerManagerActorBuilder { EthereumMessageDecoder, discoveryConfig, blacklist, - blockchainConfig.capabilities // TODO replace with a list of capabilities + blockchainConfig.capabilities ), "peer-manager" ) diff --git a/src/main/scala/io/iohk/ethereum/utils/BlockchainConfig.scala b/src/main/scala/io/iohk/ethereum/utils/BlockchainConfig.scala index ff7dd0674e..3a6c9ea80a 100644 --- a/src/main/scala/io/iohk/ethereum/utils/BlockchainConfig.scala +++ b/src/main/scala/io/iohk/ethereum/utils/BlockchainConfig.scala @@ -127,7 +127,7 @@ object BlockchainConfig { val ecip1099BlockNumber: BigInt = BigInt(blockchainConfig.getString("ecip1099-block-number")) val capabilities: List[Capability] = - blockchainConfig.getStringList("capabilities").asScala.toList.map(Capability.from) + blockchainConfig.getStringList("capabilities").asScala.toList.map(Capability.parseUnsafe) BlockchainConfig( powTargetTime = powTargetTime,