From 8ca6e12c30628546a814768abff55250ad29bd2e Mon Sep 17 00:00:00 2001 From: Lukasz Golebiewski Date: Mon, 28 Jun 2021 14:57:55 +0200 Subject: [PATCH 01/13] [ETCM-355] Introduce ETH64 message format --- .../network/p2p/MessageDecoders.scala | 25 +++++++ .../network/p2p/messages/Capability.scala | 3 +- .../ethereum/network/p2p/messages/ETH64.scala | 75 +++++++++++++++++++ .../messages/MessagesSerializationSpec.scala | 31 ++++++-- 4 files changed, 126 insertions(+), 8 deletions(-) create mode 100644 src/main/scala/io/iohk/ethereum/network/p2p/messages/ETH64.scala 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 1025c308cf..c576a672f3 100644 --- a/src/main/scala/io/iohk/ethereum/network/p2p/MessageDecoders.scala +++ b/src/main/scala/io/iohk/ethereum/network/p2p/MessageDecoders.scala @@ -55,6 +55,30 @@ object ETC64MessageDecoder extends MessageDecoder { } } +object ETH64MessageDecoder extends MessageDecoder { + import io.iohk.ethereum.network.p2p.messages.ETH64.Status._ + import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.NewBlock._ + + def fromBytes(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 Codes.NewBlockHashesCode => payload.toNewBlockHashes + case Codes.GetBlockHeadersCode => payload.toGetBlockHeaders + case Codes.BlockHeadersCode => payload.toBlockHeaders + case Codes.GetBlockBodiesCode => payload.toGetBlockBodies + case Codes.BlockBodiesCode => payload.toBlockBodies + case Codes.BlockHashesFromNumberCode => payload.toBlockHashesFromNumber + case Codes.StatusCode => payload.toStatus + case Codes.NewBlockCode => payload.toNewBlock + case Codes.SignedTransactionsCode => payload.toSignedTransactions + case _ => throw new RuntimeException(s"Unknown message type: $msgCode") + } + } +} + object ETH63MessageDecoder extends MessageDecoder { import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.Status._ import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.NewBlock._ @@ -85,6 +109,7 @@ object EthereumMessageDecoder { protocolVersion match { case Capability.Capabilities.Etc64Capability => ETC64MessageDecoder.fromBytes case Capability.Capabilities.Eth63Capability => ETH63MessageDecoder.fromBytes + case Capability.Capabilities.Eth64Capability => ETH64MessageDecoder.fromBytes case _ => throw new RuntimeException(s"Unsupported Protocol Version $protocolVersion") } } 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 394527cc8f..a9bcc8f8a0 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 @@ -51,8 +51,9 @@ object Capability { object Capabilities { val Eth63Capability: Capability = ProtocolVersions.ETH63 + val Eth64Capability: Capability = ProtocolVersions.ETH64 val Etc64Capability: Capability = ProtocolVersions.ETC64 - val All: Seq[Capability] = Seq(ProtocolVersions.ETC64, ProtocolVersions.ETH63) + val All: Seq[Capability] = Seq(ProtocolVersions.ETC64, ProtocolVersions.ETH63, ProtocolVersions.ETH64) } } diff --git a/src/main/scala/io/iohk/ethereum/network/p2p/messages/ETH64.scala b/src/main/scala/io/iohk/ethereum/network/p2p/messages/ETH64.scala new file mode 100644 index 0000000000..d943919bbb --- /dev/null +++ b/src/main/scala/io/iohk/ethereum/network/p2p/messages/ETH64.scala @@ -0,0 +1,75 @@ +package io.iohk.ethereum.network.p2p.messages + +import akka.util.ByteString +import io.iohk.ethereum.domain._ +import io.iohk.ethereum.forkid.ForkId +import io.iohk.ethereum.forkid.ForkId._ +import io.iohk.ethereum.mpt.{MptNode, MptTraversals} +import io.iohk.ethereum.network.p2p.{Message, MessageSerializableImplicit} +import io.iohk.ethereum.rlp.RLPImplicitConversions._ +import io.iohk.ethereum.rlp.RLPImplicits._ +import io.iohk.ethereum.rlp._ +import org.bouncycastle.util.encoders.Hex + +object ETH64 { + + case class Status( + protocolVersion: Int, + networkId: Int, + totalDifficulty: BigInt, + bestHash: ByteString, + genesisHash: ByteString, + forkId: ForkId + ) extends Message { + + override def toString: String = + s"Status { " + + s"code: $code, " + + s"protocolVersion: $protocolVersion, " + + s"networkId: $networkId, " + + s"totalDifficulty: $totalDifficulty, " + + s"bestHash: ${Hex.toHexString(bestHash.toArray[Byte])}, " + + s"genesisHash: ${Hex.toHexString(genesisHash.toArray[Byte])}," + + s"forkId: $forkId," + + s"}" + + override def toShortString: String = toString + override def code: Int = Codes.StatusCode + } + + object Status { + implicit class StatusEnc(val underlyingMsg: Status) + extends MessageSerializableImplicit[Status](underlyingMsg) + with RLPSerializable { + override def code: Int = Codes.StatusCode + + override def toRLPEncodable: RLPEncodeable = { + import msg._ + RLPList(protocolVersion, networkId, totalDifficulty, bestHash, genesisHash, forkId.toRLPEncodable) + } + } + + implicit class StatusDec(val bytes: Array[Byte]) extends AnyVal { + def toStatus: Status = rawDecode(bytes) match { + case RLPList( + protocolVersion, + networkId, + totalDifficulty, + bestHash, + genesisHash, + forkId + ) => + Status( + protocolVersion, + networkId, + totalDifficulty, + bestHash, + genesisHash, + decode[ForkId](forkId) + ) + + case _ => throw new RuntimeException("Cannot decode Status") + } + } + } +} 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 42cb3cdbd7..1e77f4b005 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 @@ -8,6 +8,7 @@ import org.scalatestplus.scalacheck.ScalaCheckPropertyChecks import io.iohk.ethereum.Fixtures import io.iohk.ethereum.domain.ChainWeight +import io.iohk.ethereum.forkid.ForkId import io.iohk.ethereum.network.p2p.EthereumMessageDecoder import io.iohk.ethereum.network.p2p.NetworkMessageDecoder import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages._ @@ -56,13 +57,6 @@ class MessagesSerializationSpec extends AnyWordSpec with ScalaCheckPropertyCheck } "Common Messages" when { - "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.ETH63) - } - } - "encoding and decoding SignedTransactions" should { "return same result" in { val msg = SignedTransactions(Fixtures.Blocks.Block3125369.body.transactionList) @@ -96,6 +90,28 @@ class MessagesSerializationSpec extends AnyWordSpec with ScalaCheckPropertyCheck "ETH63" when { val version = ProtocolVersions.ETH63 + "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.ETH63) + } + } + commonEthAssertions(version) + } + + "ETH64" when { + val version = ProtocolVersions.ETH64 + "encoding and decoding Status" should { + "return same result" in { + val msg = ETH64.Status(1, 2, 3, ByteString("HASH"), ByteString("HASH2"), ForkId(1L, None)) + verify(msg, (m: ETH64.Status) => m.toBytes, Codes.StatusCode, ProtocolVersions.ETH64) + } + } + commonEthAssertions(version) + } + + //scalastyle:off method.length + def commonEthAssertions(version: Capability) = { "encoding and decoding ETH61.NewBlockHashes" should { "throw for unsupported message version" in { val msg = ETH61.NewBlockHashes(Seq(ByteString("23"), ByteString("10"), ByteString("36"))) @@ -157,6 +173,7 @@ class MessagesSerializationSpec extends AnyWordSpec with ScalaCheckPropertyCheck } } } + //scalastyle:on def verify[T](msg: T, encode: T => Array[Byte], code: Int, version: Capability): Unit = messageDecoder(version).fromBytes(code, encode(msg)) shouldEqual msg From cd3547589bd0d041e7f7fd03dfd21e61f79f9225 Mon Sep 17 00:00:00 2001 From: Lukasz Golebiewski Date: Tue, 29 Jun 2021 16:29:38 +0200 Subject: [PATCH 02/13] [ETCM-355] Send fork id in the Status message --- .../ethereum/sync/util/CommonFakePeer.scala | 1 + .../txExecTest/util/DumpChainApp.scala | 2 + .../network/EtcPeerManagerActor.scala | 10 +++ .../network/handshaker/EtcHandshaker.scala | 2 + .../handshaker/EtcHelloExchangeState.scala | 4 +- .../EthNodeStatus64ExchangeState.scala | 37 ++++++++++ .../network/p2p/MessageDecoders.scala | 29 ++++---- .../ethereum/nodebuilder/NodeBuilder.scala | 1 + .../handshaker/EtcHandshakerSpec.scala | 74 ++++++++++++++++++- .../ethereum/network/p2p/PeerActorSpec.scala | 2 + 10 files changed, 144 insertions(+), 18 deletions(-) create mode 100644 src/main/scala/io/iohk/ethereum/network/handshaker/EthNodeStatus64ExchangeState.scala 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 92afca9430..843496740c 100644 --- a/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala +++ b/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala @@ -199,6 +199,7 @@ abstract class CommonFakePeer(peerName: String, fakePeerCustomConfig: FakePeerCu override val blockchain: Blockchain = CommonFakePeer.this.bl override val blockchainReader: BlockchainReader = CommonFakePeer.this.blockchainReader override val appStateStorage: AppStateStorage = storagesInstance.storages.appStateStorage + override val blockchainConfig: BlockchainConfig = Config.blockchains.blockchainConfig override val capabilities: List[Capability] = blockchainConfig.capabilities } 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 66069a4302..b312a26b06 100644 --- a/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala +++ b/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala @@ -47,6 +47,7 @@ import io.iohk.ethereum.network.rlpx.RLPxConnectionHandler.RLPxConfiguration import io.iohk.ethereum.nodebuilder.AuthHandshakerBuilder import io.iohk.ethereum.nodebuilder.NodeKeyBuilder import io.iohk.ethereum.security.SecureRandomBuilder +import io.iohk.ethereum.utils.BlockchainConfig import io.iohk.ethereum.utils.Config import io.iohk.ethereum.utils.NodeStatus import io.iohk.ethereum.utils.ServerStatus @@ -119,6 +120,7 @@ object DumpChainApp override val blockchain: Blockchain = DumpChainApp.blockchain override val blockchainReader: BlockchainReader = DumpChainApp.blockchainReader override val appStateStorage: AppStateStorage = storagesInstance.storages.appStateStorage + override val blockchainConfig: BlockchainConfig = Config.blockchains.blockchainConfig override val capabilities: List[Capability] = blockchainConfig.capabilities } diff --git a/src/main/scala/io/iohk/ethereum/network/EtcPeerManagerActor.scala b/src/main/scala/io/iohk/ethereum/network/EtcPeerManagerActor.scala index faed7d2d32..a7f912b0eb 100644 --- a/src/main/scala/io/iohk/ethereum/network/EtcPeerManagerActor.scala +++ b/src/main/scala/io/iohk/ethereum/network/EtcPeerManagerActor.scala @@ -26,6 +26,7 @@ import io.iohk.ethereum.network.p2p.messages.ETC64.NewBlock import io.iohk.ethereum.network.p2p.messages.ETH62.BlockHeaders 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.ETH64 import io.iohk.ethereum.network.p2p.messages.WireProtocol.Disconnect import io.iohk.ethereum.utils.ByteStringUtils @@ -256,6 +257,15 @@ object EtcPeerManagerActor { } object RemoteStatus { + def apply(status: ETH64.Status): RemoteStatus = + RemoteStatus( + status.protocolVersion, + status.networkId, + ChainWeight.totalDifficultyOnly(status.totalDifficulty), + status.bestHash, + status.genesisHash + ) + 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/EtcHandshaker.scala b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHandshaker.scala index 7b6236d8c0..f667771da4 100644 --- a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHandshaker.scala +++ b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHandshaker.scala @@ -10,6 +10,7 @@ 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 +import io.iohk.ethereum.utils.BlockchainConfig case class EtcHandshaker private ( handshakerState: HandshakerState[PeerInfo], @@ -38,4 +39,5 @@ trait EtcHandshakerConfiguration { val peerConfiguration: PeerConfiguration val forkResolverOpt: Option[ForkResolver] val capabilities: List[Capability] + val blockchainConfig: BlockchainConfig } 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 cc5ba6252e..c2ab9e3d3e 100644 --- a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala +++ b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala @@ -33,10 +33,12 @@ case class EtcHelloExchangeState(handshakerConfiguration: EtcHandshakerConfigura Capability.negotiate(hello.capabilities.toList, handshakerConfiguration.capabilities) match { case Some(ProtocolVersions.ETC64) => EtcNodeStatus64ExchangeState(handshakerConfiguration) case Some(ProtocolVersions.ETH63) => EtcNodeStatus63ExchangeState(handshakerConfiguration) + case Some(ProtocolVersions.ETH64) => EthNodeStatus64ExchangeState(handshakerConfiguration) case _ => log.debug( - s"Connected peer does not support {} / {} protocol. Disconnecting.", + s"Connected peer does not support {} / {} / {} protocol. Disconnecting.", ProtocolVersions.ETH63, + ProtocolVersions.ETH64, ProtocolVersions.ETC64 ) DisconnectedState(Disconnect.Reasons.IncompatibleP2pProtocolVersion) diff --git a/src/main/scala/io/iohk/ethereum/network/handshaker/EthNodeStatus64ExchangeState.scala b/src/main/scala/io/iohk/ethereum/network/handshaker/EthNodeStatus64ExchangeState.scala new file mode 100644 index 0000000000..37d14d8742 --- /dev/null +++ b/src/main/scala/io/iohk/ethereum/network/handshaker/EthNodeStatus64ExchangeState.scala @@ -0,0 +1,37 @@ +package io.iohk.ethereum.network.handshaker + +import io.iohk.ethereum.forkid.ForkId +import io.iohk.ethereum.network.EtcPeerManagerActor.{PeerInfo, RemoteStatus} +import io.iohk.ethereum.network.p2p.messages.{BaseETH6XMessages, ProtocolVersions, ETH64} +import io.iohk.ethereum.network.p2p.{Message, MessageSerializable} + +case class EthNodeStatus64ExchangeState( + handshakerConfiguration: EtcHandshakerConfiguration +) extends EtcNodeStatusExchangeState[ETH64.Status] { + + import handshakerConfiguration._ + + def applyResponseMessage: PartialFunction[Message, HandshakerState[PeerInfo]] = { case status: ETH64.Status => + // TODO: validate fork id of the remote peer + applyRemoteStatusMessage(RemoteStatus(status)) + } + + override protected def createStatusMsg(): MessageSerializable = { + val bestBlockHeader = getBestBlockHeader() + val chainWeight = blockchain.getChainWeightByHash(bestBlockHeader.hash).get + val genesisHash = blockchain.genesisHeader.hash + + val status = ETH64.Status( + protocolVersion = ProtocolVersions.ETH64.version, + networkId = peerConfiguration.networkId, + totalDifficulty = chainWeight.totalDifficulty, + bestHash = bestBlockHeader.hash, + genesisHash = genesisHash, + forkId = ForkId.create(genesisHash, blockchainConfig)(blockchain.getBestBlockNumber()) + ) + + log.debug(s"Sending status $status") + status + } + +} 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 c576a672f3..dc2a61d2c8 100644 --- a/src/main/scala/io/iohk/ethereum/network/p2p/MessageDecoders.scala +++ b/src/main/scala/io/iohk/ethereum/network/p2p/MessageDecoders.scala @@ -59,24 +59,23 @@ object ETH64MessageDecoder extends MessageDecoder { import io.iohk.ethereum.network.p2p.messages.ETH64.Status._ import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.NewBlock._ - def fromBytes(msgCode: Int, payload: Array[Byte]): Message = { + def fromBytes(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 Codes.NewBlockHashesCode => payload.toNewBlockHashes - case Codes.GetBlockHeadersCode => payload.toGetBlockHeaders - case Codes.BlockHeadersCode => payload.toBlockHeaders - case Codes.GetBlockBodiesCode => payload.toGetBlockBodies - case Codes.BlockBodiesCode => payload.toBlockBodies + case Codes.GetNodeDataCode => payload.toGetNodeData + case Codes.NodeDataCode => payload.toNodeData + case Codes.GetReceiptsCode => payload.toGetReceipts + case Codes.ReceiptsCode => payload.toReceipts + case Codes.NewBlockHashesCode => payload.toNewBlockHashes + case Codes.GetBlockHeadersCode => payload.toGetBlockHeaders + case Codes.BlockHeadersCode => payload.toBlockHeaders + case Codes.GetBlockBodiesCode => payload.toGetBlockBodies + case Codes.BlockBodiesCode => payload.toBlockBodies case Codes.BlockHashesFromNumberCode => payload.toBlockHashesFromNumber - case Codes.StatusCode => payload.toStatus - case Codes.NewBlockCode => payload.toNewBlock - case Codes.SignedTransactionsCode => payload.toSignedTransactions - case _ => throw new RuntimeException(s"Unknown message type: $msgCode") + case Codes.StatusCode => payload.toStatus + case Codes.NewBlockCode => payload.toNewBlock + case Codes.SignedTransactionsCode => payload.toSignedTransactions + case _ => throw new RuntimeException(s"Unknown message type: $msgCode") } - } } object ETH63MessageDecoder extends MessageDecoder { diff --git a/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala b/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala index 5abe1a58ed..0234bf516b 100644 --- a/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala +++ b/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala @@ -246,6 +246,7 @@ trait HandshakerBuilder { override val blockchainReader: BlockchainReader = self.blockchainReader override val appStateStorage: AppStateStorage = self.storagesInstance.storages.appStateStorage override val capabilities: List[Capability] = self.blockchainConfig.capabilities + override val blockchainConfig: BlockchainConfig = self.blockchainConfig } lazy val handshaker: Handshaker[PeerInfo] = EtcHandshaker(handshakerConfiguration) 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 6318d8f6b5..4276dae219 100644 --- a/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala @@ -12,6 +12,7 @@ import io.iohk.ethereum.blockchain.sync.EphemBlockchainTestSetup import io.iohk.ethereum.crypto.generateKeyPair import io.iohk.ethereum.db.storage.AppStateStorage import io.iohk.ethereum.domain._ +import io.iohk.ethereum.forkid.ForkId import io.iohk.ethereum.network.EtcPeerManagerActor.PeerInfo import io.iohk.ethereum.network.EtcPeerManagerActor.RemoteStatus import io.iohk.ethereum.network.ForkResolver @@ -26,6 +27,7 @@ import io.iohk.ethereum.network.p2p.messages.ETC64 import io.iohk.ethereum.network.p2p.messages.ETH62.BlockHeaders import io.iohk.ethereum.network.p2p.messages.ETH62.GetBlockHeaders import io.iohk.ethereum.network.p2p.messages.ETH62.GetBlockHeaders.GetBlockHeadersEnc +import io.iohk.ethereum.network.p2p.messages.ETH64 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 @@ -190,6 +192,40 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { } } + it should "send status with fork id when peer supports ETH64" in new LocalPeerETH64Setup + with RemotePeerETH64Setup { + + val newChainWeight = ChainWeight.zero.increase(genesisBlock.header).increase(firstBlock.header) + + blockchain.save(firstBlock, Nil, newChainWeight, saveAsBestBlock = true) + + val newLocalStatusMsg = + localStatusMsg + .copy( + bestHash = firstBlock.header.hash, + totalDifficulty = newChainWeight.totalDifficulty, + forkId = ForkId(0xfc64ec04L, Some(1150000)) + ) + + initHandshakerWithoutResolver.nextMessage.map(_.messageToSend) shouldBe Right(localHello: HelloEnc) + + val handshakerAfterHelloOpt = initHandshakerWithoutResolver.applyMessage(remoteHello) + assert(handshakerAfterHelloOpt.isDefined) + + handshakerAfterHelloOpt.get.nextMessage.map(_.messageToSend.underlyingMsg) shouldBe Right(newLocalStatusMsg) + + val handshakerAfterStatusOpt = handshakerAfterHelloOpt.get.applyMessage(remoteStatusMsg) + assert(handshakerAfterStatusOpt.isDefined) + + handshakerAfterStatusOpt.get.nextMessage match { + case Left(HandshakeSuccess(peerInfo)) => + peerInfo.remoteStatus.protocolVersion shouldBe localStatus.protocolVersion + + case other => + fail(s"Invalid handshaker state: $other") + } + } + it should "fail if a timeout happened during hello exchange" in new TestSetup { val handshakerAfterTimeout = initHandshakerWithoutResolver.processTimeout handshakerAfterTimeout.nextMessage.map(_.messageToSend) shouldBe Left( @@ -287,6 +323,7 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { override val appStateStorage: AppStateStorage = TestSetup.this.storagesInstance.storages.appStateStorage override val capabilities: List[Capability] = pv override val blockchainReader: BlockchainReader = TestSetup.this.blockchainReader + override val blockchainConfig: BlockchainConfig = TestSetup.this.blockchainConfig } val etcHandshakerConfigurationWithResolver: MockEtcHandshakerConfiguration = new MockEtcHandshakerConfiguration { @@ -296,7 +333,7 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { } val initHandshakerWithoutResolver: EtcHandshaker = EtcHandshaker( - new MockEtcHandshakerConfiguration(List(ProtocolVersions.ETC64, ProtocolVersions.ETH63)) + new MockEtcHandshakerConfiguration(List(ProtocolVersions.ETC64, ProtocolVersions.ETH63, ProtocolVersions.ETH64)) ) val initHandshakerWithResolver: EtcHandshaker = EtcHandshaker(etcHandshakerConfigurationWithResolver) @@ -309,7 +346,7 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { val localHello: Hello = Hello( p2pVersion = EtcHelloExchangeState.P2pVersion, clientId = Config.clientId, - capabilities = Seq(Etc64Capability, Eth63Capability), + capabilities = Seq(Etc64Capability, Eth63Capability, Eth64Capability), listenPort = 0, //Local node not listening nodeId = ByteString(nodeStatus.nodeId) ) @@ -329,6 +366,18 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { val localStatus: RemoteStatus = RemoteStatus(localStatusMsg) } + trait LocalPeerETH64Setup extends LocalPeerSetup { + val localStatusMsg = ETH64.Status( + protocolVersion = ProtocolVersions.ETH64.version, + networkId = Config.Network.peer.networkId, + totalDifficulty = genesisBlock.header.difficulty, + bestHash = genesisBlock.header.hash, + genesisHash = genesisBlock.header.hash, + forkId = ForkId(1L, None) + ) + val localStatus = RemoteStatus(localStatusMsg) + } + trait LocalPeerETC64Setup extends LocalPeerSetup { val localStatusMsg: ETC64.Status = ETC64.Status( protocolVersion = ProtocolVersions.ETC64.version, @@ -387,4 +436,25 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { genesisHash = genesisBlock.header.hash ) } + + trait RemotePeerETH64Setup extends RemotePeerSetup { + val remoteHello = Hello( + p2pVersion = EtcHelloExchangeState.P2pVersion, + clientId = "remote-peer", + capabilities = Seq(Eth64Capability), + listenPort = remotePort, + nodeId = ByteString(remoteNodeStatus.nodeId) + ) + + val remoteStatusMsg = ETH64.Status( + protocolVersion = ProtocolVersions.ETH64.version, + networkId = Config.Network.peer.networkId, + totalDifficulty = 0, + bestHash = genesisBlock.header.hash, + genesisHash = genesisBlock.header.hash, + forkId = ForkId(2L, Some(3L)) + ) + + val remoteStatus = RemoteStatus(remoteStatusMsg) + } } 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 bbcf78839e..1e637c3757 100644 --- a/src/test/scala/io/iohk/ethereum/network/p2p/PeerActorSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/p2p/PeerActorSpec.scala @@ -58,6 +58,7 @@ import io.iohk.ethereum.network.p2p.messages.WireProtocol._ import io.iohk.ethereum.network.rlpx.RLPxConnectionHandler import io.iohk.ethereum.network.rlpx.RLPxConnectionHandler.RLPxConfiguration import io.iohk.ethereum.security.SecureRandomBuilder +import io.iohk.ethereum.utils.BlockchainConfig import io.iohk.ethereum.utils.Config import io.iohk.ethereum.utils.NodeStatus import io.iohk.ethereum.utils.ServerStatus @@ -587,6 +588,7 @@ class PeerActorSpec override val blockchainReader: BlockchainReader = self.blockchainReader override val appStateStorage: AppStateStorage = self.storagesInstance.storages.appStateStorage override val capabilities: List[Capability] = List(protocol) + override val blockchainConfig: BlockchainConfig = self.blockchainConfig } val handshaker: EtcHandshaker = EtcHandshaker(handshakerConfiguration) From 355e494020f86a6dffa07fce471e9e463edf01cc Mon Sep 17 00:00:00 2001 From: Lukasz Golebiewski Date: Tue, 29 Jun 2021 17:00:57 +0200 Subject: [PATCH 03/13] [ETCM-355] Remove capabilities from EtcHandshakerConfiguration --- src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala | 1 - .../scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala | 1 - .../io/iohk/ethereum/network/handshaker/EtcHandshaker.scala | 1 - .../ethereum/network/handshaker/EtcHelloExchangeState.scala | 4 ++-- src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala | 1 - .../iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala | 3 +-- .../scala/io/iohk/ethereum/network/p2p/PeerActorSpec.scala | 3 +-- 7 files changed, 4 insertions(+), 10 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 843496740c..a09ba4bf9b 100644 --- a/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala +++ b/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala @@ -200,7 +200,6 @@ abstract class CommonFakePeer(peerName: String, fakePeerCustomConfig: FakePeerCu override val blockchainReader: BlockchainReader = CommonFakePeer.this.blockchainReader override val appStateStorage: AppStateStorage = storagesInstance.storages.appStateStorage override val blockchainConfig: BlockchainConfig = Config.blockchains.blockchainConfig - override val capabilities: List[Capability] = blockchainConfig.capabilities } lazy val handshaker: Handshaker[PeerInfo] = EtcHandshaker(handshakerConfiguration) 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 b312a26b06..68d4d3e613 100644 --- a/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala +++ b/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala @@ -121,7 +121,6 @@ object DumpChainApp override val blockchainReader: BlockchainReader = DumpChainApp.blockchainReader override val appStateStorage: AppStateStorage = storagesInstance.storages.appStateStorage override val blockchainConfig: BlockchainConfig = Config.blockchains.blockchainConfig - override val capabilities: List[Capability] = blockchainConfig.capabilities } lazy val handshaker: Handshaker[PeerInfo] = EtcHandshaker(handshakerConfiguration) 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 f667771da4..ae7dd8ded1 100644 --- a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHandshaker.scala +++ b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHandshaker.scala @@ -38,6 +38,5 @@ trait EtcHandshakerConfiguration { val appStateStorage: AppStateStorage val peerConfiguration: PeerConfiguration val forkResolverOpt: Option[ForkResolver] - val capabilities: List[Capability] val blockchainConfig: BlockchainConfig } 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 c2ab9e3d3e..8241f0849b 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,7 @@ case class EtcHelloExchangeState(handshakerConfiguration: EtcHandshakerConfigura override def applyResponseMessage: PartialFunction[Message, HandshakerState[PeerInfo]] = { case hello: Hello => log.debug("Protocol handshake finished with peer ({})", hello) // FIXME in principle this should be already negotiated - Capability.negotiate(hello.capabilities.toList, handshakerConfiguration.capabilities) match { + Capability.negotiate(hello.capabilities.toList, handshakerConfiguration.blockchainConfig.capabilities) match { case Some(ProtocolVersions.ETC64) => EtcNodeStatus64ExchangeState(handshakerConfiguration) case Some(ProtocolVersions.ETH63) => EtcNodeStatus63ExchangeState(handshakerConfiguration) case Some(ProtocolVersions.ETH64) => EthNodeStatus64ExchangeState(handshakerConfiguration) @@ -59,7 +59,7 @@ case class EtcHelloExchangeState(handshakerConfiguration: EtcHandshakerConfigura Hello( p2pVersion = EtcHelloExchangeState.P2pVersion, clientId = Config.clientId, - capabilities = handshakerConfiguration.capabilities, + capabilities = handshakerConfiguration.blockchainConfig.capabilities, listenPort = listenPort, nodeId = ByteString(nodeStatus.nodeId) ) diff --git a/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala b/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala index 0234bf516b..4230503dc6 100644 --- a/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala +++ b/src/main/scala/io/iohk/ethereum/nodebuilder/NodeBuilder.scala @@ -245,7 +245,6 @@ trait HandshakerBuilder { override val blockchain: Blockchain = self.blockchain override val blockchainReader: BlockchainReader = self.blockchainReader override val appStateStorage: AppStateStorage = self.storagesInstance.storages.appStateStorage - override val capabilities: List[Capability] = self.blockchainConfig.capabilities override val blockchainConfig: BlockchainConfig = self.blockchainConfig } 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 4276dae219..3d9aa99ce7 100644 --- a/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala @@ -321,9 +321,8 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { 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 capabilities: List[Capability] = pv override val blockchainReader: BlockchainReader = TestSetup.this.blockchainReader - override val blockchainConfig: BlockchainConfig = TestSetup.this.blockchainConfig + override val blockchainConfig: BlockchainConfig = TestSetup.this.blockchainConfig.copy(capabilities = pv) } val etcHandshakerConfigurationWithResolver: MockEtcHandshakerConfiguration = new MockEtcHandshakerConfiguration { 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 1e637c3757..64f13de843 100644 --- a/src/test/scala/io/iohk/ethereum/network/p2p/PeerActorSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/p2p/PeerActorSpec.scala @@ -587,8 +587,7 @@ class PeerActorSpec override val blockchain: Blockchain = self.blockchain override val blockchainReader: BlockchainReader = self.blockchainReader override val appStateStorage: AppStateStorage = self.storagesInstance.storages.appStateStorage - override val capabilities: List[Capability] = List(protocol) - override val blockchainConfig: BlockchainConfig = self.blockchainConfig + override val blockchainConfig: BlockchainConfig = self.blockchainConfig.copy(capabilities = List(protocol)) } val handshaker: EtcHandshaker = EtcHandshaker(handshakerConfiguration) From 7283008e5facc166955b9bc1674ec475e84a87d3 Mon Sep 17 00:00:00 2001 From: Lukasz Golebiewski Date: Wed, 30 Jun 2021 09:22:25 +0200 Subject: [PATCH 04/13] [ETCM-355] Rename EtcNodeStatus63ExchangeState -> EthNodeStatus63ExchangeState --- .../ethereum/network/handshaker/EtcHelloExchangeState.scala | 2 +- ...63ExchangeState.scala => EthNodeStatus63ExchangeState.scala} | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) rename src/main/scala/io/iohk/ethereum/network/handshaker/{EtcNodeStatus63ExchangeState.scala => EthNodeStatus63ExchangeState.scala} (96%) 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 8241f0849b..6d968f7e42 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,7 @@ case class EtcHelloExchangeState(handshakerConfiguration: EtcHandshakerConfigura // FIXME in principle this should be already negotiated Capability.negotiate(hello.capabilities.toList, handshakerConfiguration.blockchainConfig.capabilities) match { case Some(ProtocolVersions.ETC64) => EtcNodeStatus64ExchangeState(handshakerConfiguration) - case Some(ProtocolVersions.ETH63) => EtcNodeStatus63ExchangeState(handshakerConfiguration) + case Some(ProtocolVersions.ETH63) => EthNodeStatus63ExchangeState(handshakerConfiguration) case Some(ProtocolVersions.ETH64) => EthNodeStatus64ExchangeState(handshakerConfiguration) case _ => log.debug( diff --git a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcNodeStatus63ExchangeState.scala b/src/main/scala/io/iohk/ethereum/network/handshaker/EthNodeStatus63ExchangeState.scala similarity index 96% rename from src/main/scala/io/iohk/ethereum/network/handshaker/EtcNodeStatus63ExchangeState.scala rename to src/main/scala/io/iohk/ethereum/network/handshaker/EthNodeStatus63ExchangeState.scala index 2c4849bb35..8decf88785 100644 --- a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcNodeStatus63ExchangeState.scala +++ b/src/main/scala/io/iohk/ethereum/network/handshaker/EthNodeStatus63ExchangeState.scala @@ -7,7 +7,7 @@ import io.iohk.ethereum.network.p2p.MessageSerializable import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages import io.iohk.ethereum.network.p2p.messages.ProtocolVersions -case class EtcNodeStatus63ExchangeState( +case class EthNodeStatus63ExchangeState( handshakerConfiguration: EtcHandshakerConfiguration ) extends EtcNodeStatusExchangeState[BaseETH6XMessages.Status] { From 1dea7d6af4a97eafee2f8a5d71a2886235f271ae Mon Sep 17 00:00:00 2001 From: Lukasz Golebiewski Date: Thu, 1 Jul 2021 10:26:35 +0200 Subject: [PATCH 05/13] [ETCM-355] Use eth/64 in the nomad test env --- .../resources/conf/chains/testnet-internal-nomad-chain.conf | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 02fa0fc1f7..c74ea96c85 100644 --- a/src/main/resources/conf/chains/testnet-internal-nomad-chain.conf +++ b/src/main/resources/conf/chains/testnet-internal-nomad-chain.conf @@ -3,7 +3,7 @@ # 1 - mainnet, 3 - ropsten, 7 - mordor network-id = 42 - capabilities = ["etc/64"] + capabilities = ["eth/64"] # Possibility to set Proof of Work target time for testing purposes. # null means that the standard difficulty calculation rules are used From d06829664074905cbcb0e42c17224b5d25204d50 Mon Sep 17 00:00:00 2001 From: Lukasz Golebiewski Date: Tue, 6 Jul 2021 14:22:13 +0200 Subject: [PATCH 06/13] fixup! [ETCM-355] Send fork id in the Status message --- .../iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) 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 3d9aa99ce7..040f9dc0f8 100644 --- a/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala @@ -192,8 +192,7 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { } } - it should "send status with fork id when peer supports ETH64" in new LocalPeerETH64Setup - with RemotePeerETH64Setup { + it should "send status with fork id when peer supports ETH64" in new LocalPeerETH64Setup with RemotePeerETH64Setup { val newChainWeight = ChainWeight.zero.increase(genesisBlock.header).increase(firstBlock.header) From 626d41a5b6e3687e190712ef5befb434bd7606e5 Mon Sep 17 00:00:00 2001 From: Lukasz Golebiewski Date: Tue, 6 Jul 2021 14:37:51 +0200 Subject: [PATCH 07/13] [ETCM-355] scalafix --- .../io/iohk/ethereum/sync/util/CommonFakePeer.scala | 1 - .../iohk/ethereum/txExecTest/util/DumpChainApp.scala | 1 - .../ethereum/network/handshaker/EtcHandshaker.scala | 3 +-- .../handshaker/EthNodeStatus64ExchangeState.scala | 9 ++++++--- .../io/iohk/ethereum/network/p2p/messages/ETH64.scala | 9 +++++---- .../network/handshaker/EtcHandshakerSpec.scala | 10 +++++----- .../p2p/messages/MessagesSerializationSpec.scala | 2 +- 7 files changed, 18 insertions(+), 17 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 a09ba4bf9b..613e1d32a9 100644 --- a/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala +++ b/src/it/scala/io/iohk/ethereum/sync/util/CommonFakePeer.scala @@ -61,7 +61,6 @@ import io.iohk.ethereum.network.discovery.PeerDiscoveryManager.DiscoveredNodesIn import io.iohk.ethereum.network.handshaker.EtcHandshaker import io.iohk.ethereum.network.handshaker.EtcHandshakerConfiguration import io.iohk.ethereum.network.handshaker.Handshaker -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.nodebuilder.PruningConfigBuilder 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 68d4d3e613..0e98309cf8 100644 --- a/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala +++ b/src/it/scala/io/iohk/ethereum/txExecTest/util/DumpChainApp.scala @@ -42,7 +42,6 @@ import io.iohk.ethereum.network.discovery.DiscoveryConfig import io.iohk.ethereum.network.handshaker.EtcHandshaker import io.iohk.ethereum.network.handshaker.EtcHandshakerConfiguration import io.iohk.ethereum.network.handshaker.Handshaker -import io.iohk.ethereum.network.p2p.messages.Capability import io.iohk.ethereum.network.rlpx.RLPxConnectionHandler.RLPxConfiguration import io.iohk.ethereum.nodebuilder.AuthHandshakerBuilder import io.iohk.ethereum.nodebuilder.NodeKeyBuilder 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 ae7dd8ded1..dc197da1a2 100644 --- a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHandshaker.scala +++ b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHandshaker.scala @@ -8,9 +8,8 @@ import io.iohk.ethereum.domain.BlockchainReader 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 import io.iohk.ethereum.utils.BlockchainConfig +import io.iohk.ethereum.utils.NodeStatus case class EtcHandshaker private ( handshakerState: HandshakerState[PeerInfo], diff --git a/src/main/scala/io/iohk/ethereum/network/handshaker/EthNodeStatus64ExchangeState.scala b/src/main/scala/io/iohk/ethereum/network/handshaker/EthNodeStatus64ExchangeState.scala index 37d14d8742..8099da010c 100644 --- a/src/main/scala/io/iohk/ethereum/network/handshaker/EthNodeStatus64ExchangeState.scala +++ b/src/main/scala/io/iohk/ethereum/network/handshaker/EthNodeStatus64ExchangeState.scala @@ -1,9 +1,12 @@ package io.iohk.ethereum.network.handshaker import io.iohk.ethereum.forkid.ForkId -import io.iohk.ethereum.network.EtcPeerManagerActor.{PeerInfo, RemoteStatus} -import io.iohk.ethereum.network.p2p.messages.{BaseETH6XMessages, ProtocolVersions, ETH64} -import io.iohk.ethereum.network.p2p.{Message, MessageSerializable} +import io.iohk.ethereum.network.EtcPeerManagerActor.PeerInfo +import io.iohk.ethereum.network.EtcPeerManagerActor.RemoteStatus +import io.iohk.ethereum.network.p2p.Message +import io.iohk.ethereum.network.p2p.MessageSerializable +import io.iohk.ethereum.network.p2p.messages.ETH64 +import io.iohk.ethereum.network.p2p.messages.ProtocolVersions case class EthNodeStatus64ExchangeState( handshakerConfiguration: EtcHandshakerConfiguration diff --git a/src/main/scala/io/iohk/ethereum/network/p2p/messages/ETH64.scala b/src/main/scala/io/iohk/ethereum/network/p2p/messages/ETH64.scala index d943919bbb..0a5615c9e4 100644 --- a/src/main/scala/io/iohk/ethereum/network/p2p/messages/ETH64.scala +++ b/src/main/scala/io/iohk/ethereum/network/p2p/messages/ETH64.scala @@ -1,15 +1,16 @@ package io.iohk.ethereum.network.p2p.messages import akka.util.ByteString -import io.iohk.ethereum.domain._ + +import org.bouncycastle.util.encoders.Hex + import io.iohk.ethereum.forkid.ForkId import io.iohk.ethereum.forkid.ForkId._ -import io.iohk.ethereum.mpt.{MptNode, MptTraversals} -import io.iohk.ethereum.network.p2p.{Message, MessageSerializableImplicit} +import io.iohk.ethereum.network.p2p.Message +import io.iohk.ethereum.network.p2p.MessageSerializableImplicit import io.iohk.ethereum.rlp.RLPImplicitConversions._ import io.iohk.ethereum.rlp.RLPImplicits._ import io.iohk.ethereum.rlp._ -import org.bouncycastle.util.encoders.Hex object ETH64 { 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 040f9dc0f8..f722b8f18c 100644 --- a/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala @@ -365,7 +365,7 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { } trait LocalPeerETH64Setup extends LocalPeerSetup { - val localStatusMsg = ETH64.Status( + val localStatusMsg: ETH64.Status = ETH64.Status( protocolVersion = ProtocolVersions.ETH64.version, networkId = Config.Network.peer.networkId, totalDifficulty = genesisBlock.header.difficulty, @@ -373,7 +373,7 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { genesisHash = genesisBlock.header.hash, forkId = ForkId(1L, None) ) - val localStatus = RemoteStatus(localStatusMsg) + val localStatus: RemoteStatus = RemoteStatus(localStatusMsg) } trait LocalPeerETC64Setup extends LocalPeerSetup { @@ -436,7 +436,7 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { } trait RemotePeerETH64Setup extends RemotePeerSetup { - val remoteHello = Hello( + val remoteHello: Hello = Hello( p2pVersion = EtcHelloExchangeState.P2pVersion, clientId = "remote-peer", capabilities = Seq(Eth64Capability), @@ -444,7 +444,7 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { nodeId = ByteString(remoteNodeStatus.nodeId) ) - val remoteStatusMsg = ETH64.Status( + val remoteStatusMsg: ETH64.Status = ETH64.Status( protocolVersion = ProtocolVersions.ETH64.version, networkId = Config.Network.peer.networkId, totalDifficulty = 0, @@ -453,6 +453,6 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { forkId = ForkId(2L, Some(3L)) ) - val remoteStatus = RemoteStatus(remoteStatusMsg) + val remoteStatus: RemoteStatus = RemoteStatus(remoteStatusMsg) } } 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 1e77f4b005..03fb321bc4 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 @@ -111,7 +111,7 @@ class MessagesSerializationSpec extends AnyWordSpec with ScalaCheckPropertyCheck } //scalastyle:off method.length - def commonEthAssertions(version: Capability) = { + def commonEthAssertions(version: Capability): Unit = { "encoding and decoding ETH61.NewBlockHashes" should { "throw for unsupported message version" in { val msg = ETH61.NewBlockHashes(Seq(ByteString("23"), ByteString("10"), ByteString("36"))) From 367e6c47ba5a209f94e2b73ae44f42efdd141f2d Mon Sep 17 00:00:00 2001 From: Lukasz Golebiewski Date: Wed, 7 Jul 2021 10:49:35 +0200 Subject: [PATCH 08/13] [ETCM-355] Log protocol name upon unknown message type --- .../io/iohk/ethereum/network/p2p/MessageDecoders.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) 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 dc2a61d2c8..d3ebebd7a2 100644 --- a/src/main/scala/io/iohk/ethereum/network/p2p/MessageDecoders.scala +++ b/src/main/scala/io/iohk/ethereum/network/p2p/MessageDecoders.scala @@ -27,7 +27,7 @@ object NetworkMessageDecoder extends MessageDecoder { case Ping.code => payload.toPing case Pong.code => payload.toPong case Hello.code => payload.toHello - case _ => throw new RuntimeException(s"Unknown message type: $msgCode") + case _ => throw new RuntimeException(s"Unknown network message type: $msgCode") } } @@ -51,7 +51,7 @@ object ETC64MessageDecoder extends MessageDecoder { case Codes.BlockBodiesCode => payload.toBlockBodies case Codes.BlockHashesFromNumberCode => payload.toBlockHashesFromNumber case Codes.SignedTransactionsCode => payload.toSignedTransactions - case _ => throw new RuntimeException(s"Unknown message type: $msgCode") + case _ => throw new RuntimeException(s"Unknown etc/64 message type: $msgCode") } } @@ -74,7 +74,7 @@ object ETH64MessageDecoder extends MessageDecoder { case Codes.StatusCode => payload.toStatus case Codes.NewBlockCode => payload.toNewBlock case Codes.SignedTransactionsCode => payload.toSignedTransactions - case _ => throw new RuntimeException(s"Unknown message type: $msgCode") + case _ => throw new RuntimeException(s"Unknown eth/64 message type: $msgCode") } } @@ -97,7 +97,7 @@ object ETH63MessageDecoder extends MessageDecoder { case Codes.StatusCode => payload.toStatus case Codes.NewBlockCode => payload.toNewBlock case Codes.SignedTransactionsCode => payload.toSignedTransactions - case _ => throw new RuntimeException(s"Unknown message type: $msgCode") + case _ => throw new RuntimeException(s"Unknown eth/63 message type: $msgCode") } } From 0605e8e27d62156050895351be611861d99646bf Mon Sep 17 00:00:00 2001 From: Lukasz Golebiewski Date: Wed, 7 Jul 2021 15:37:42 +0200 Subject: [PATCH 09/13] [ETCM-355] Log negotiated protocol version --- .../handshaker/EtcHelloExchangeState.scala | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 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 6d968f7e42..d4f3b5ccad 100644 --- a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala +++ b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala @@ -31,9 +31,18 @@ case class EtcHelloExchangeState(handshakerConfiguration: EtcHandshakerConfigura log.debug("Protocol handshake finished with peer ({})", hello) // FIXME in principle this should be already negotiated Capability.negotiate(hello.capabilities.toList, handshakerConfiguration.blockchainConfig.capabilities) match { - case Some(ProtocolVersions.ETC64) => EtcNodeStatus64ExchangeState(handshakerConfiguration) - case Some(ProtocolVersions.ETH63) => EthNodeStatus63ExchangeState(handshakerConfiguration) - case Some(ProtocolVersions.ETH64) => EthNodeStatus64ExchangeState(handshakerConfiguration) + case Some(ProtocolVersions.ETC64) => { + log.debug("Negotiated protocol version with client {} is etc/64", hello.clientId) + EtcNodeStatus64ExchangeState(handshakerConfiguration) + } + case Some(ProtocolVersions.ETH63) => { + log.debug("Negotiated protocol version with client {} is eth/63", hello.clientId) + EthNodeStatus63ExchangeState(handshakerConfiguration) + } + case Some(ProtocolVersions.ETH64) => { + log.debug("Negotiated protocol version with client {} is eth/64", hello.clientId) + EthNodeStatus64ExchangeState(handshakerConfiguration) + } case _ => log.debug( s"Connected peer does not support {} / {} / {} protocol. Disconnecting.", From eb751bcc5478bb050ce5eabb226f56ca7be95244 Mon Sep 17 00:00:00 2001 From: Lukasz Golebiewski Date: Wed, 7 Jul 2021 15:41:14 +0200 Subject: [PATCH 10/13] [ETCM-355] Log received messages in PeerActor --- src/main/scala/io/iohk/ethereum/network/PeerActor.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/scala/io/iohk/ethereum/network/PeerActor.scala b/src/main/scala/io/iohk/ethereum/network/PeerActor.scala index 6a2b74760a..f5a142bf77 100644 --- a/src/main/scala/io/iohk/ethereum/network/PeerActor.scala +++ b/src/main/scala/io/iohk/ethereum/network/PeerActor.scala @@ -144,6 +144,7 @@ class PeerActor[R <: HandshakeResult]( 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 + log.debug("Message received: {} from peer {}", msg, peerAddress) handshaker.applyMessage(msg).foreach { newHandshaker => timeout.cancel() processHandshakerNextMessage(newHandshaker, remoteNodeId, rlpxConnection, numRetries) From fda64e92c90544b3a4d2d56c1d53e6ef17b035ba Mon Sep 17 00:00:00 2001 From: Lukasz Golebiewski Date: Thu, 8 Jul 2021 15:18:53 +0200 Subject: [PATCH 11/13] [ETCM-355] Post merge fix --- .../network/handshaker/EthNodeStatus64ExchangeState.scala | 4 ++-- .../iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/main/scala/io/iohk/ethereum/network/handshaker/EthNodeStatus64ExchangeState.scala b/src/main/scala/io/iohk/ethereum/network/handshaker/EthNodeStatus64ExchangeState.scala index 8099da010c..a6344dd30c 100644 --- a/src/main/scala/io/iohk/ethereum/network/handshaker/EthNodeStatus64ExchangeState.scala +++ b/src/main/scala/io/iohk/ethereum/network/handshaker/EthNodeStatus64ExchangeState.scala @@ -22,7 +22,7 @@ case class EthNodeStatus64ExchangeState( override protected def createStatusMsg(): MessageSerializable = { val bestBlockHeader = getBestBlockHeader() val chainWeight = blockchain.getChainWeightByHash(bestBlockHeader.hash).get - val genesisHash = blockchain.genesisHeader.hash + val genesisHash = blockchainReader.genesisHeader.hash val status = ETH64.Status( protocolVersion = ProtocolVersions.ETH64.version, @@ -30,7 +30,7 @@ case class EthNodeStatus64ExchangeState( totalDifficulty = chainWeight.totalDifficulty, bestHash = bestBlockHeader.hash, genesisHash = genesisHash, - forkId = ForkId.create(genesisHash, blockchainConfig)(blockchain.getBestBlockNumber()) + forkId = ForkId.create(genesisHash, blockchainConfig)(blockchainReader.getBestBlockNumber()) ) log.debug(s"Sending status $status") 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 f722b8f18c..dbc80daeb9 100644 --- a/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/handshaker/EtcHandshakerSpec.scala @@ -196,7 +196,7 @@ class EtcHandshakerSpec extends AnyFlatSpec with Matchers { val newChainWeight = ChainWeight.zero.increase(genesisBlock.header).increase(firstBlock.header) - blockchain.save(firstBlock, Nil, newChainWeight, saveAsBestBlock = true) + blockchainWriter.save(firstBlock, Nil, newChainWeight, saveAsBestBlock = true) val newLocalStatusMsg = localStatusMsg From 224f32a8ad6975fbcdca8bc1bca3b90c5c5569ac Mon Sep 17 00:00:00 2001 From: Lukasz Golebiewski Date: Thu, 8 Jul 2021 16:44:23 +0200 Subject: [PATCH 12/13] [ETCM-355] Expand MessageDecoderSpec with eth/64 tests --- .../network/p2p/MessageDecodersSpec.scala | 21 +++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) 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 5c3ca28455..51e317626c 100644 --- a/src/test/scala/io/iohk/ethereum/network/p2p/MessageDecodersSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/p2p/MessageDecodersSpec.scala @@ -9,6 +9,7 @@ import org.scalatest.matchers.should.Matchers import io.iohk.ethereum.Fixtures import io.iohk.ethereum.ObjectGenerators import io.iohk.ethereum.domain.ChainWeight +import io.iohk.ethereum.forkid.ForkId import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.SignedTransactions import io.iohk.ethereum.network.p2p.messages.Capability.Capabilities._ import io.iohk.ethereum.network.p2p.messages._ @@ -60,6 +61,8 @@ class MessageDecodersSpec extends AnyFlatSpec with Matchers with SecureRandomBui .fromBytes(Codes.NewBlockHashesCode, NewBlockHashesETH62bytes) shouldBe newBlockHashesETH62 decode(ProtocolVersions.ETC64) .fromBytes(Codes.NewBlockHashesCode, NewBlockHashesETH62bytes) shouldBe newBlockHashesETH62 + decode(ProtocolVersions.ETH64) + .fromBytes(Codes.NewBlockHashesCode, NewBlockHashesETH62bytes) shouldBe newBlockHashesETH62 } it should "not decode message from older version of protocol as newer version" in { @@ -81,6 +84,7 @@ class MessageDecodersSpec extends AnyFlatSpec with Matchers with SecureRandomBui val getBlockHeadersBytes: Array[Byte] = getBlockHeaders.toBytes decode(ProtocolVersions.ETH63).fromBytes(Codes.GetBlockHeadersCode, getBlockHeadersBytes) shouldBe getBlockHeaders + decode(ProtocolVersions.ETH64).fromBytes(Codes.GetBlockHeadersCode, getBlockHeadersBytes) shouldBe getBlockHeaders decode(ProtocolVersions.ETC64).fromBytes(Codes.GetBlockHeadersCode, getBlockHeadersBytes) shouldBe getBlockHeaders } @@ -89,6 +93,7 @@ class MessageDecodersSpec extends AnyFlatSpec with Matchers with SecureRandomBui val blockHeadersBytes: Array[Byte] = blockHeaders.toBytes decode(ProtocolVersions.ETH63).fromBytes(Codes.BlockHeadersCode, blockHeadersBytes) shouldBe blockHeaders + decode(ProtocolVersions.ETH64).fromBytes(Codes.BlockHeadersCode, blockHeadersBytes) shouldBe blockHeaders decode(ProtocolVersions.ETC64).fromBytes(Codes.BlockHeadersCode, blockHeadersBytes) shouldBe blockHeaders } @@ -97,6 +102,7 @@ class MessageDecodersSpec extends AnyFlatSpec with Matchers with SecureRandomBui val getBlockBodiesBytes: Array[Byte] = getBlockBodies.toBytes decode(ProtocolVersions.ETH63).fromBytes(Codes.GetBlockBodiesCode, getBlockBodiesBytes) shouldBe getBlockBodies + decode(ProtocolVersions.ETH64).fromBytes(Codes.GetBlockBodiesCode, getBlockBodiesBytes) shouldBe getBlockBodies decode(ProtocolVersions.ETC64).fromBytes(Codes.GetBlockBodiesCode, getBlockBodiesBytes) shouldBe getBlockBodies } @@ -105,6 +111,7 @@ class MessageDecodersSpec extends AnyFlatSpec with Matchers with SecureRandomBui val blockBodiesBytes: Array[Byte] = blockBodies.toBytes decode(ProtocolVersions.ETH63).fromBytes(Codes.BlockBodiesCode, blockBodiesBytes) shouldBe blockBodies + decode(ProtocolVersions.ETH64).fromBytes(Codes.BlockBodiesCode, blockBodiesBytes) shouldBe blockBodies decode(ProtocolVersions.ETC64).fromBytes(Codes.BlockBodiesCode, blockBodiesBytes) shouldBe blockBodies } @@ -113,6 +120,7 @@ class MessageDecodersSpec extends AnyFlatSpec with Matchers with SecureRandomBui val getNodeDataBytes: Array[Byte] = getNodeData.toBytes decode(ProtocolVersions.ETH63).fromBytes(Codes.GetNodeDataCode, getNodeDataBytes) shouldBe getNodeData + decode(ProtocolVersions.ETH64).fromBytes(Codes.GetNodeDataCode, getNodeDataBytes) shouldBe getNodeData decode(ProtocolVersions.ETC64).fromBytes(Codes.GetNodeDataCode, getNodeDataBytes) shouldBe getNodeData } @@ -121,6 +129,7 @@ class MessageDecodersSpec extends AnyFlatSpec with Matchers with SecureRandomBui val nodeDataBytes: Array[Byte] = nodeData.toBytes decode(ProtocolVersions.ETH63).fromBytes(Codes.NodeDataCode, nodeDataBytes) shouldBe nodeData + decode(ProtocolVersions.ETH64).fromBytes(Codes.NodeDataCode, nodeDataBytes) shouldBe nodeData decode(ProtocolVersions.ETC64).fromBytes(Codes.NodeDataCode, nodeDataBytes) shouldBe nodeData } @@ -129,6 +138,7 @@ class MessageDecodersSpec extends AnyFlatSpec with Matchers with SecureRandomBui val getReceiptsBytes: Array[Byte] = getReceipts.toBytes decode(ProtocolVersions.ETH63).fromBytes(Codes.GetReceiptsCode, getReceiptsBytes) shouldBe getReceipts + decode(ProtocolVersions.ETH64).fromBytes(Codes.GetReceiptsCode, getReceiptsBytes) shouldBe getReceipts decode(ProtocolVersions.ETC64).fromBytes(Codes.GetReceiptsCode, getReceiptsBytes) shouldBe getReceipts } @@ -137,17 +147,21 @@ class MessageDecodersSpec extends AnyFlatSpec with Matchers with SecureRandomBui val receiptsBytes: Array[Byte] = receipts.toBytes decode(ProtocolVersions.ETH63).fromBytes(Codes.ReceiptsCode, receiptsBytes) shouldBe receipts + decode(ProtocolVersions.ETH64).fromBytes(Codes.ReceiptsCode, receiptsBytes) shouldBe receipts decode(ProtocolVersions.ETC64).fromBytes(Codes.ReceiptsCode, receiptsBytes) shouldBe receipts } it should "decode Status message for all supported versions of protocol" in { val status63 = BaseETH6XMessages.Status(ProtocolVersions.ETH63.version, 1, BigInt(100), exampleHash, exampleHash) val status63Bytes: Array[Byte] = status63.toBytes - val status64 = + val statusEtc64 = ETC64.Status(ProtocolVersions.ETH63.version, 1, ChainWeight(1, BigInt(100)), exampleHash, exampleHash) + val statusEth64 = + ETH64.Status(ProtocolVersions.ETH64.version, 1, BigInt(100), exampleHash, exampleHash, ForkId(1L, None)) decode(ProtocolVersions.ETH63).fromBytes(Codes.StatusCode, status63Bytes) shouldBe status63 - decode(ProtocolVersions.ETC64).fromBytes(Codes.StatusCode, status64.toBytes) shouldBe status64 + decode(ProtocolVersions.ETH64).fromBytes(Codes.StatusCode, statusEth64.toBytes) shouldBe statusEth64 + decode(ProtocolVersions.ETC64).fromBytes(Codes.StatusCode, statusEtc64.toBytes) shouldBe statusEtc64 } it should "decode NewBlock message for all supported versions of protocol" in { @@ -156,6 +170,7 @@ class MessageDecodersSpec extends AnyFlatSpec with Matchers with SecureRandomBui val newBlock64 = ObjectGenerators.newBlock64Gen(secureRandom, None).sample.get decode(ProtocolVersions.ETH63).fromBytes(Codes.NewBlockCode, newBlock63Bytes) shouldBe newBlock63 + decode(ProtocolVersions.ETH64).fromBytes(Codes.NewBlockCode, newBlock63Bytes) shouldBe newBlock63 decode(ProtocolVersions.ETC64).fromBytes(Codes.NewBlockCode, newBlock64.toBytes) shouldBe newBlock64 } @@ -165,6 +180,8 @@ class MessageDecodersSpec extends AnyFlatSpec with Matchers with SecureRandomBui decode(ProtocolVersions.ETH63) .fromBytes(Codes.SignedTransactionsCode, signedTransactionsBytes) shouldBe signedTransactions + decode(ProtocolVersions.ETH64) + .fromBytes(Codes.SignedTransactionsCode, signedTransactionsBytes) shouldBe signedTransactions decode(ProtocolVersions.ETC64) .fromBytes(Codes.SignedTransactionsCode, signedTransactionsBytes) shouldBe signedTransactions } From b0ec96e281e2228d9685a90b0e4bf785cbbcf5e8 Mon Sep 17 00:00:00 2001 From: Lukasz Golebiewski Date: Fri, 9 Jul 2021 11:42:53 +0200 Subject: [PATCH 13/13] [ETCM-355] Introduce ProtocolFamily This also fixes a bug in NewBlock message creation, where only protocol version was taken into account --- .../blockchain/sync/regular/BlockBroadcast.scala | 12 +++++++----- .../ethereum/network/EtcPeerManagerActor.scala | 16 +++++++++++++++- .../handshaker/EtcHelloExchangeState.scala | 9 +++------ .../ethereum/network/p2p/messages/package.scala | 8 ++++++++ .../blockchain/sync/BlockBroadcastSpec.scala | 6 +++++- .../blockchain/sync/PeersClientSpec.scala | 2 ++ .../blockchain/sync/PivotBlockSelectorSpec.scala | 2 ++ .../ethereum/blockchain/sync/StateSyncSpec.scala | 2 ++ .../ethereum/blockchain/sync/TestSyncPeers.scala | 2 ++ .../fast/FastSyncBranchResolverActorSpec.scala | 2 ++ .../sync/regular/RegularSyncFixtures.scala | 8 +++++++- .../sync/regular/RegularSyncSpec.scala | 3 ++- .../iohk/ethereum/jsonrpc/DebugServiceSpec.scala | 2 ++ .../ethereum/jsonrpc/JsonRpcControllerSpec.scala | 2 ++ .../ethereum/network/EtcPeerManagerSpec.scala | 2 ++ .../ethereum/network/PeerEventBusActorSpec.scala | 2 ++ .../iohk/ethereum/network/PeerManagerSpec.scala | 2 ++ .../ethereum/network/p2p/PeerActorSpec.scala | 2 ++ 18 files changed, 69 insertions(+), 15 deletions(-) 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 7ed705cabc..3467e56e0b 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 @@ -17,7 +17,7 @@ import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages import io.iohk.ethereum.network.p2p.messages.ETC64 import io.iohk.ethereum.network.p2p.messages.ETH62 import io.iohk.ethereum.network.p2p.messages.ETH62.BlockHash -import io.iohk.ethereum.network.p2p.messages.ProtocolVersions +import io.iohk.ethereum.network.p2p.messages.ProtocolFamily._ class BlockBroadcast(val etcPeerManager: ActorRef) { @@ -45,10 +45,12 @@ 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.ETC64.version) - blockToBroadcast.as64 - else blockToBroadcast.as63 + val remoteStatus = peers(peer.id).peerInfo.remoteStatus + + val message: MessageSerializable = remoteStatus.protocolFamily match { + case ETH => blockToBroadcast.as63 + case ETC => blockToBroadcast.as64 + } etcPeerManager ! EtcPeerManagerActor.SendMessage(message, peer.id) } diff --git a/src/main/scala/io/iohk/ethereum/network/EtcPeerManagerActor.scala b/src/main/scala/io/iohk/ethereum/network/EtcPeerManagerActor.scala index a7f912b0eb..dacbb9f327 100644 --- a/src/main/scala/io/iohk/ethereum/network/EtcPeerManagerActor.scala +++ b/src/main/scala/io/iohk/ethereum/network/EtcPeerManagerActor.scala @@ -27,6 +27,9 @@ import io.iohk.ethereum.network.p2p.messages.ETH62.BlockHeaders 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.ETH64 +import io.iohk.ethereum.network.p2p.messages.ProtocolFamily +import io.iohk.ethereum.network.p2p.messages.ProtocolFamily.ETC +import io.iohk.ethereum.network.p2p.messages.ProtocolFamily.ETH import io.iohk.ethereum.network.p2p.messages.WireProtocol.Disconnect import io.iohk.ethereum.utils.ByteStringUtils @@ -240,6 +243,7 @@ object EtcPeerManagerActor { * (they are different versions of Status msg) */ case class RemoteStatus( + protocolFamily: ProtocolFamily, protocolVersion: Int, networkId: Int, chainWeight: ChainWeight, @@ -248,6 +252,7 @@ object EtcPeerManagerActor { ) { override def toString: String = s"RemoteStatus { " + + s"protocolFamily: $protocolFamily, " + s"protocolVersion: $protocolVersion, " + s"networkId: $networkId, " + s"chainWeight: $chainWeight, " + @@ -259,6 +264,7 @@ object EtcPeerManagerActor { object RemoteStatus { def apply(status: ETH64.Status): RemoteStatus = RemoteStatus( + ETH, status.protocolVersion, status.networkId, ChainWeight.totalDifficultyOnly(status.totalDifficulty), @@ -267,10 +273,18 @@ object EtcPeerManagerActor { ) def apply(status: ETC64.Status): RemoteStatus = - RemoteStatus(status.protocolVersion, status.networkId, status.chainWeight, status.bestHash, status.genesisHash) + RemoteStatus( + ETC, + status.protocolVersion, + status.networkId, + status.chainWeight, + status.bestHash, + status.genesisHash + ) def apply(status: BaseETH6XMessages.Status): RemoteStatus = RemoteStatus( + ETH, status.protocolVersion, status.networkId, ChainWeight.totalDifficultyOnly(status.totalDifficulty), 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 d4f3b5ccad..55caf8b438 100644 --- a/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala +++ b/src/main/scala/io/iohk/ethereum/network/handshaker/EtcHelloExchangeState.scala @@ -31,18 +31,15 @@ case class EtcHelloExchangeState(handshakerConfiguration: EtcHandshakerConfigura log.debug("Protocol handshake finished with peer ({})", hello) // FIXME in principle this should be already negotiated Capability.negotiate(hello.capabilities.toList, handshakerConfiguration.blockchainConfig.capabilities) match { - case Some(ProtocolVersions.ETC64) => { + case Some(ProtocolVersions.ETC64) => log.debug("Negotiated protocol version with client {} is etc/64", hello.clientId) EtcNodeStatus64ExchangeState(handshakerConfiguration) - } - case Some(ProtocolVersions.ETH63) => { + case Some(ProtocolVersions.ETH63) => log.debug("Negotiated protocol version with client {} is eth/63", hello.clientId) EthNodeStatus63ExchangeState(handshakerConfiguration) - } - case Some(ProtocolVersions.ETH64) => { + case Some(ProtocolVersions.ETH64) => log.debug("Negotiated protocol version with client {} is eth/64", hello.clientId) EthNodeStatus64ExchangeState(handshakerConfiguration) - } case _ => log.debug( s"Connected peer does not support {} / {} / {} protocol. Disconnecting.", 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 c4dfebab71..09473103a4 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,5 +1,13 @@ package io.iohk.ethereum.network.p2p +package messages { + sealed trait ProtocolFamily + object ProtocolFamily { + final case object ETH extends ProtocolFamily + final case object ETC extends ProtocolFamily + } +} + package object messages { object ProtocolVersions { val ETH61: Capability = Capability("eth", 61.toByte) 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 c131a7934e..4d0dfc37db 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/BlockBroadcastSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/BlockBroadcastSpec.scala @@ -27,6 +27,7 @@ import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages import io.iohk.ethereum.network.p2p.messages.ETC64.NewBlock import io.iohk.ethereum.network.p2p.messages.ETH62 import io.iohk.ethereum.network.p2p.messages.ETH62.NewBlockHashes +import io.iohk.ethereum.network.p2p.messages.ProtocolFamily import io.iohk.ethereum.network.p2p.messages.ProtocolVersions class BlockBroadcastSpec @@ -61,7 +62,9 @@ class BlockBroadcastSpec val blockHeader: BlockHeader = baseBlockHeader.copy(number = initialPeerInfo.maxBlockNumber - 3) val newBlockNewHashes = NewBlockHashes(Seq(ETH62.BlockHash(blockHeader.hash, blockHeader.number))) val peerInfo = initialPeerInfo - .copy(remoteStatus = peerStatus.copy(protocolVersion = ProtocolVersions.ETH63.version)) + .copy(remoteStatus = + peerStatus.copy(protocolFamily = ProtocolFamily.ETH, protocolVersion = ProtocolVersions.ETH63.version) + ) .withChainWeight(ChainWeight.totalDifficultyOnly(initialPeerInfo.chainWeight.totalDifficulty)) val newBlock = BaseETH6XMessages.NewBlock(Block(blockHeader, BlockBody(Nil, Nil)), peerInfo.chainWeight.totalDifficulty + 2) @@ -176,6 +179,7 @@ class BlockBroadcastSpec val baseBlockHeader = Fixtures.Blocks.Block3125369.header val peerStatus: RemoteStatus = RemoteStatus( + protocolFamily = ProtocolFamily.ETC, protocolVersion = ProtocolVersions.ETC64.version, networkId = 1, chainWeight = ChainWeight(10, 10000), 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 e6798ac20b..bdbabbac43 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/PeersClientSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/PeersClientSpec.scala @@ -16,6 +16,7 @@ import io.iohk.ethereum.network.EtcPeerManagerActor.PeerInfo import io.iohk.ethereum.network.EtcPeerManagerActor.RemoteStatus import io.iohk.ethereum.network.Peer import io.iohk.ethereum.network.PeerId +import io.iohk.ethereum.network.p2p.messages.ProtocolFamily import io.iohk.ethereum.network.p2p.messages.ProtocolVersions class PeersClientSpec extends AnyFlatSpec with Matchers with ScalaCheckPropertyChecks { @@ -80,6 +81,7 @@ class PeersClientSpec extends AnyFlatSpec with Matchers with ScalaCheckPropertyC val peer3: Peer = Peer(PeerId("peer3"), new InetSocketAddress("127.0.0.1", 3), TestProbe().ref, false) private val peerStatus = RemoteStatus( + protocolFamily = ProtocolFamily.ETH, protocolVersion = ProtocolVersions.ETH63.version, networkId = 1, chainWeight = ChainWeight(0, 0), 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 852b2d1232..b9c925fcd9 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/PivotBlockSelectorSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/PivotBlockSelectorSpec.scala @@ -38,6 +38,7 @@ import io.iohk.ethereum.network.p2p.Message import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.NewBlock import io.iohk.ethereum.network.p2p.messages.Codes import io.iohk.ethereum.network.p2p.messages.ETH62._ +import io.iohk.ethereum.network.p2p.messages.ProtocolFamily import io.iohk.ethereum.network.p2p.messages.ProtocolVersions import io.iohk.ethereum.utils.Config.SyncConfig @@ -587,6 +588,7 @@ class PivotBlockSelectorSpec val peer1Status: RemoteStatus = RemoteStatus( + ProtocolFamily.ETC, ProtocolVersions.ETC64.version, 1, ChainWeight.totalDifficultyOnly(20), 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 f570483343..c326347ae1 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncSpec.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/StateSyncSpec.scala @@ -41,6 +41,7 @@ import io.iohk.ethereum.network.PeerEventBusActor.PeerEvent.MessageFromPeer import io.iohk.ethereum.network.PeerId 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.ProtocolFamily import io.iohk.ethereum.network.p2p.messages.ProtocolVersions import io.iohk.ethereum.utils.Config @@ -131,6 +132,7 @@ class StateSyncSpec val syncInit: TestProbe = TestProbe() val peerStatus: RemoteStatus = RemoteStatus( + protocolFamily = ProtocolFamily.ETH, protocolVersion = ProtocolVersions.ETH63.version, networkId = 1, chainWeight = ChainWeight.totalDifficultyOnly(10000), 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 45965626a8..378ebd7408 100644 --- a/src/test/scala/io/iohk/ethereum/blockchain/sync/TestSyncPeers.scala +++ b/src/test/scala/io/iohk/ethereum/blockchain/sync/TestSyncPeers.scala @@ -10,6 +10,7 @@ import io.iohk.ethereum.network.EtcPeerManagerActor.PeerInfo import io.iohk.ethereum.network.EtcPeerManagerActor.RemoteStatus import io.iohk.ethereum.network.Peer import io.iohk.ethereum.network.PeerId +import io.iohk.ethereum.network.p2p.messages.ProtocolFamily import io.iohk.ethereum.network.p2p.messages.ProtocolVersions trait TestSyncPeers { self: TestSyncConfig => @@ -25,6 +26,7 @@ trait TestSyncPeers { self: TestSyncConfig => val peer1Status: RemoteStatus = RemoteStatus( + ProtocolFamily.ETC, ProtocolVersions.ETC64.version, 1, ChainWeight.totalDifficultyOnly(20), 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 b137424088..8c257a63f1 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 @@ -44,6 +44,7 @@ import io.iohk.ethereum.network.PeerEventBusActor.PeerEvent.MessageFromPeer import io.iohk.ethereum.network.PeerId import io.iohk.ethereum.network.p2p.messages.ETH62.BlockHeaders import io.iohk.ethereum.network.p2p.messages.ETH62.GetBlockHeaders +import io.iohk.ethereum.network.p2p.messages.ProtocolFamily import io.iohk.ethereum.network.p2p.messages.ProtocolVersions import io.iohk.ethereum.utils.Logger @@ -264,6 +265,7 @@ class FastSyncBranchResolverActorSpec def getPeerInfo(peer: Peer, protocolVersion: Int = ProtocolVersions.ETC64.version): PeerInfo = { val status = RemoteStatus( + ProtocolFamily.ETC, protocolVersion, 1, ChainWeight.totalDifficultyOnly(1), 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 d6d8075038..c3ae629e08 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 @@ -48,6 +48,7 @@ import io.iohk.ethereum.network.p2p.messages.ETC64.NewBlock import io.iohk.ethereum.network.p2p.messages.ETH62._ import io.iohk.ethereum.network.p2p.messages.ETH63.GetNodeData import io.iohk.ethereum.network.p2p.messages.ETH63.NodeData +import io.iohk.ethereum.network.p2p.messages.ProtocolFamily import io.iohk.ethereum.network.p2p.messages.ProtocolVersions import io.iohk.ethereum.security.SecureRandomBuilder import io.iohk.ethereum.utils.Config.SyncConfig @@ -122,9 +123,14 @@ 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.ETC64.version): PeerInfo = { + def getPeerInfo( + peer: Peer, + protocolFamily: ProtocolFamily = ProtocolFamily.ETC, + protocolVersion: Int = ProtocolVersions.ETC64.version + ): PeerInfo = { val status = RemoteStatus( + protocolFamily, protocolVersion, 1, ChainWeight.totalDifficultyOnly(1), 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 440d8db616..31bd0e9483 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 @@ -59,6 +59,7 @@ import io.iohk.ethereum.network.p2p.messages.ETC64.NewBlock import io.iohk.ethereum.network.p2p.messages.ETH62._ import io.iohk.ethereum.network.p2p.messages.ETH63.GetNodeData import io.iohk.ethereum.network.p2p.messages.ETH63.NodeData +import io.iohk.ethereum.network.p2p.messages.ProtocolFamily import io.iohk.ethereum.network.p2p.messages.ProtocolVersions import io.iohk.ethereum.utils.Config.SyncConfig @@ -708,7 +709,7 @@ class RegularSyncSpec val peerWithETH63: (Peer, PeerInfo) = { val id = peerId(handshakedPeers.size) val peer = getPeer(id) - val peerInfo = getPeerInfo(peer, ProtocolVersions.ETH63.version) + val peerInfo = getPeerInfo(peer, ProtocolFamily.ETH, ProtocolVersions.ETH63.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 069ec60ed0..bd2b948f73 100644 --- a/src/test/scala/io/iohk/ethereum/jsonrpc/DebugServiceSpec.scala +++ b/src/test/scala/io/iohk/ethereum/jsonrpc/DebugServiceSpec.scala @@ -26,6 +26,7 @@ import io.iohk.ethereum.network.PeerActor import io.iohk.ethereum.network.PeerId import io.iohk.ethereum.network.PeerManagerActor import io.iohk.ethereum.network.PeerManagerActor.Peers +import io.iohk.ethereum.network.p2p.messages.ProtocolFamily import io.iohk.ethereum.network.p2p.messages.ProtocolVersions class DebugServiceSpec @@ -76,6 +77,7 @@ class DebugServiceSpec val debugService = new DebugService(peerManager.ref, etcPeerManager.ref) val peerStatus: RemoteStatus = RemoteStatus( + protocolFamily = ProtocolFamily.ETH, protocolVersion = ProtocolVersions.ETH63.version, networkId = 1, chainWeight = ChainWeight.totalDifficultyOnly(10000), diff --git a/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerSpec.scala b/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerSpec.scala index 4695bf93d7..0cbfb4c9d0 100644 --- a/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/jsonrpc/JsonRpcControllerSpec.scala @@ -36,6 +36,7 @@ import io.iohk.ethereum.jsonrpc.server.http.JsonRpcHttpServer import io.iohk.ethereum.jsonrpc.server.ipc.JsonRpcIpcServer import io.iohk.ethereum.network.EtcPeerManagerActor.PeerInfo import io.iohk.ethereum.network.EtcPeerManagerActor.RemoteStatus +import io.iohk.ethereum.network.p2p.messages.ProtocolFamily import io.iohk.ethereum.network.p2p.messages.ProtocolVersions class JsonRpcControllerSpec @@ -129,6 +130,7 @@ class JsonRpcControllerSpec it should "debug_listPeersInfo" in new JsonRpcControllerFixture { val peerStatus = RemoteStatus( + protocolFamily = ProtocolFamily.ETH, protocolVersion = ProtocolVersions.ETH63.version, networkId = 1, chainWeight = ChainWeight.totalDifficultyOnly(10000), diff --git a/src/test/scala/io/iohk/ethereum/network/EtcPeerManagerSpec.scala b/src/test/scala/io/iohk/ethereum/network/EtcPeerManagerSpec.scala index 24bd5530fa..64b9e162e8 100644 --- a/src/test/scala/io/iohk/ethereum/network/EtcPeerManagerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/EtcPeerManagerSpec.scala @@ -32,6 +32,7 @@ import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.NewBlock import io.iohk.ethereum.network.p2p.messages.Codes import io.iohk.ethereum.network.p2p.messages.ETC64 import io.iohk.ethereum.network.p2p.messages.ETH62._ +import io.iohk.ethereum.network.p2p.messages.ProtocolFamily import io.iohk.ethereum.network.p2p.messages.ProtocolVersions import io.iohk.ethereum.network.p2p.messages.WireProtocol.Disconnect import io.iohk.ethereum.utils.Config @@ -301,6 +302,7 @@ class EtcPeerManagerSpec extends AnyFlatSpec with Matchers { val forkResolver = new ForkResolver.EtcForkResolver(blockchainConfig.daoForkConfig.get) val peerStatus: RemoteStatus = RemoteStatus( + protocolFamily = ProtocolFamily.ETH, protocolVersion = ProtocolVersions.ETH63.version, networkId = 1, chainWeight = ChainWeight.totalDifficultyOnly(10000), diff --git a/src/test/scala/io/iohk/ethereum/network/PeerEventBusActorSpec.scala b/src/test/scala/io/iohk/ethereum/network/PeerEventBusActorSpec.scala index 10f4b81e9c..7b6048c8f7 100644 --- a/src/test/scala/io/iohk/ethereum/network/PeerEventBusActorSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/PeerEventBusActorSpec.scala @@ -19,6 +19,7 @@ import io.iohk.ethereum.network.PeerEventBusActor.PeerEvent.PeerDisconnected import io.iohk.ethereum.network.PeerEventBusActor.PeerEvent.PeerHandshakeSuccessful import io.iohk.ethereum.network.PeerEventBusActor.PeerSelector import io.iohk.ethereum.network.PeerEventBusActor.SubscriptionClassifier._ +import io.iohk.ethereum.network.p2p.messages.ProtocolFamily import io.iohk.ethereum.network.p2p.messages.ProtocolVersions import io.iohk.ethereum.network.p2p.messages.WireProtocol.Ping import io.iohk.ethereum.network.p2p.messages.WireProtocol.Pong @@ -250,6 +251,7 @@ class PeerEventBusActorSpec extends AnyFlatSpec with Matchers { val peerEventBusActor: ActorRef = system.actorOf(PeerEventBusActor.props) val peerStatus: RemoteStatus = RemoteStatus( + protocolFamily = ProtocolFamily.ETH, protocolVersion = ProtocolVersions.ETH63.version, networkId = 1, chainWeight = ChainWeight.totalDifficultyOnly(10000), diff --git a/src/test/scala/io/iohk/ethereum/network/PeerManagerSpec.scala b/src/test/scala/io/iohk/ethereum/network/PeerManagerSpec.scala index 2e4f7f55ec..013199bdcb 100644 --- a/src/test/scala/io/iohk/ethereum/network/PeerManagerSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/PeerManagerSpec.scala @@ -53,6 +53,7 @@ import io.iohk.ethereum.network.discovery.DiscoveryConfig import io.iohk.ethereum.network.discovery.Node import io.iohk.ethereum.network.discovery.PeerDiscoveryManager import io.iohk.ethereum.network.p2p.messages.BaseETH6XMessages.NewBlock +import io.iohk.ethereum.network.p2p.messages.ProtocolFamily import io.iohk.ethereum.network.p2p.messages.ProtocolVersions import io.iohk.ethereum.network.p2p.messages.WireProtocol.Disconnect import io.iohk.ethereum.utils.Config @@ -631,6 +632,7 @@ class PeerManagerSpec val blacklist: CacheBasedBlacklist = CacheBasedBlacklist(cache) val peerStatus: RemoteStatus = RemoteStatus( + protocolFamily = ProtocolFamily.ETH, protocolVersion = ProtocolVersions.ETH63.version, networkId = 1, chainWeight = ChainWeight.totalDifficultyOnly(10000), 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 64f13de843..22614dc09d 100644 --- a/src/test/scala/io/iohk/ethereum/network/p2p/PeerActorSpec.scala +++ b/src/test/scala/io/iohk/ethereum/network/p2p/PeerActorSpec.scala @@ -49,6 +49,7 @@ import io.iohk.ethereum.network.p2p.messages.Capability.Capabilities._ import io.iohk.ethereum.network.p2p.messages.ETC64 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.ProtocolFamily import io.iohk.ethereum.network.p2p.messages.ProtocolVersions import io.iohk.ethereum.network.p2p.messages.WireProtocol.Disconnect.DisconnectEnc import io.iohk.ethereum.network.p2p.messages.WireProtocol.Disconnect.Reasons @@ -431,6 +432,7 @@ class PeerActorSpec it should "stay connected to pre fork peer" in new TestSetup { val remoteStatus = RemoteStatus( + protocolFamily = ProtocolFamily.ETH, protocolVersion = ProtocolVersions.ETH63.version, networkId = peerConf.networkId, chainWeight =