From dd8c576a875aa0d091e3fe0a145cb3cdf878fa98 Mon Sep 17 00:00:00 2001 From: Enrico Del Fante Date: Tue, 3 May 2022 18:30:29 +0200 Subject: [PATCH 1/3] integrate builder client ExecutionPayloadContext remove builder api from ExecutionEngineClient Let ExecutionLayerChannelImpl use builder client complete execution engine channel renaming move throttling from channel to clients execution builder CLI ExecutionEngine* to ExecutionLayer* --- .../BlockOperationSelectorFactory.java | 14 +-- .../coordinator/BlockFactoryTest.java | 11 +- .../BlockOperationSelectorFactoryTest.java | 27 +++-- .../ExecutionEngineClient.java | 8 -- .../ThrottlingExecutionEngineClient.java | 13 -- .../Web3JExecutionEngineClient.java | 32 ----- .../schema/BlindedBeaconBlockBodyV1.java | 113 +++++++++--------- .../executionclient/schema/BuilderBidV1.java | 12 ++ .../ExecutionLayerChannelImpl.java | 80 +++++++++---- .../execution/ExecutionPayloadContext.java | 84 +++++++++++++ .../executionlayer/ExecutionLayerChannel.java | 19 +-- .../ExecutionLayerChannelStub.java | 20 ++-- .../teku/spec/util/DataStructureUtil.java | 6 + .../forkchoice/ForkChoiceNotifier.java | 5 +- .../forkchoice/ForkChoiceNotifierImpl.java | 10 +- .../forkchoice/ForkChoiceUpdateData.java | 20 ++-- .../forkchoice/ForkChoiceNotifierTest.java | 17 ++- .../forkchoice/StubForkChoiceNotifier.java | 4 +- .../ExecutionLayerConfiguration.java | 16 ++- 19 files changed, 311 insertions(+), 200 deletions(-) create mode 100644 ethereum/spec/src/main/java/tech/pegasys/teku/spec/datastructures/execution/ExecutionPayloadContext.java diff --git a/beacon/validator/src/main/java/tech/pegasys/teku/validator/coordinator/BlockOperationSelectorFactory.java b/beacon/validator/src/main/java/tech/pegasys/teku/validator/coordinator/BlockOperationSelectorFactory.java index 69d5a94f17a..51542e10ce6 100644 --- a/beacon/validator/src/main/java/tech/pegasys/teku/validator/coordinator/BlockOperationSelectorFactory.java +++ b/beacon/validator/src/main/java/tech/pegasys/teku/validator/coordinator/BlockOperationSelectorFactory.java @@ -24,7 +24,6 @@ import org.apache.tuweni.bytes.Bytes32; import tech.pegasys.teku.bls.BLSSignature; import tech.pegasys.teku.infrastructure.async.SafeFuture; -import tech.pegasys.teku.infrastructure.bytes.Bytes8; import tech.pegasys.teku.infrastructure.ssz.SszList; import tech.pegasys.teku.infrastructure.unsigned.UInt64; import tech.pegasys.teku.spec.Spec; @@ -33,6 +32,7 @@ import tech.pegasys.teku.spec.datastructures.blocks.Eth1Data; import tech.pegasys.teku.spec.datastructures.blocks.SignedBeaconBlockUnblinder; import tech.pegasys.teku.spec.datastructures.blocks.blockbody.BeaconBlockBodyBuilder; +import tech.pegasys.teku.spec.datastructures.execution.ExecutionPayloadContext; import tech.pegasys.teku.spec.datastructures.operations.Attestation; import tech.pegasys.teku.spec.datastructures.operations.AttesterSlashing; import tech.pegasys.teku.spec.datastructures.operations.ProposerSlashing; @@ -155,9 +155,9 @@ public Consumer createSelector( spec.atSlot(blockSlotState.getSlot()).getSchemaDefinitions()) .getExecutionPayloadHeaderSchema() .getHeaderOfDefaultPayload(), - (payloadId) -> + (executionPayloadContext) -> executionLayerChannel - .getPayloadHeader(payloadId, blockSlotState.getSlot()) + .builderGetHeader(executionPayloadContext, blockSlotState.getSlot()) .join())); return; } @@ -184,9 +184,9 @@ public Consumer createSelector( spec.atSlot(blockSlotState.getSlot()).getSchemaDefinitions()) .getExecutionPayloadSchema() .getDefault(), - (payloadId) -> + (executionPayloadContext) -> executionLayerChannel - .engineGetPayload(payloadId, blockSlotState.getSlot()) + .engineGetPayload(executionPayloadContext, blockSlotState.getSlot()) .join())); }; } @@ -195,7 +195,7 @@ private Supplier payloadProvider( final Bytes32 parentRoot, final BeaconState blockSlotState, Supplier defaultSupplier, - Function supplier) { + Function supplier) { return () -> forkChoiceNotifier .getPayloadId(parentRoot, blockSlotState.getSlot()) @@ -237,7 +237,7 @@ public Consumer createUnblinderSelector() { } else { bodyUnblinder.setExecutionPayloadSupplier( () -> - executionLayerChannel.proposeBlindedBlock( + executionLayerChannel.builderGetPayload( bodyUnblinder.getSignedBlindedBeaconBlock())); } }; diff --git a/beacon/validator/src/test/java/tech/pegasys/teku/validator/coordinator/BlockFactoryTest.java b/beacon/validator/src/test/java/tech/pegasys/teku/validator/coordinator/BlockFactoryTest.java index 2d21ecbc361..742334a2ade 100644 --- a/beacon/validator/src/test/java/tech/pegasys/teku/validator/coordinator/BlockFactoryTest.java +++ b/beacon/validator/src/test/java/tech/pegasys/teku/validator/coordinator/BlockFactoryTest.java @@ -27,7 +27,6 @@ import org.junit.jupiter.api.Test; import tech.pegasys.teku.bls.BLSSignature; import tech.pegasys.teku.infrastructure.async.SafeFuture; -import tech.pegasys.teku.infrastructure.bytes.Bytes8; import tech.pegasys.teku.infrastructure.ssz.SszList; import tech.pegasys.teku.infrastructure.unsigned.UInt64; import tech.pegasys.teku.spec.Spec; @@ -320,10 +319,12 @@ private BeaconBlock assertBlockCreated( when(voluntaryExitPool.getItemsForBlock(any(), any(), any())).thenReturn(voluntaryExits); when(eth1DataCache.getEth1Vote(any())).thenReturn(ETH1_DATA); when(forkChoiceNotifier.getPayloadId(any(), any())) - .thenReturn(SafeFuture.completedFuture(Optional.of(Bytes8.fromHexStringLenient("0x0")))); + .thenReturn( + SafeFuture.completedFuture( + Optional.of(dataStructureUtil.createPayloadExecutionContext(false)))); when(executionLayer.engineGetPayload(any(), any())) .thenReturn(SafeFuture.completedFuture(executionPayload)); - when(executionLayer.getPayloadHeader(any(), any())) + when(executionLayer.builderGetHeader(any(), any())) .thenReturn(SafeFuture.completedFuture(executionPayloadHeader)); beaconChainUtil.initializeStorage(); @@ -366,7 +367,7 @@ private SignedBeaconBlock assertBlockUnblinded( final SignedBeaconBlock beaconBlock, final Spec spec, final boolean isMevBoostEnabled) { final BlockFactory blockFactory = createBlockFactory(spec, isMevBoostEnabled); - when(executionLayer.proposeBlindedBlock(beaconBlock)) + when(executionLayer.builderGetPayload(beaconBlock)) .thenReturn(SafeFuture.completedFuture(executionPayload)); final SignedBeaconBlock block = @@ -375,7 +376,7 @@ private SignedBeaconBlock assertBlockUnblinded( if (!beaconBlock.getMessage().getBody().isBlinded()) { verifyNoInteractions(executionLayer); } else { - verify(executionLayer).proposeBlindedBlock(beaconBlock); + verify(executionLayer).builderGetPayload(beaconBlock); } assertThat(block).isNotNull(); diff --git a/beacon/validator/src/test/java/tech/pegasys/teku/validator/coordinator/BlockOperationSelectorFactoryTest.java b/beacon/validator/src/test/java/tech/pegasys/teku/validator/coordinator/BlockOperationSelectorFactoryTest.java index 09327e727b3..a47b9a62bcf 100644 --- a/beacon/validator/src/test/java/tech/pegasys/teku/validator/coordinator/BlockOperationSelectorFactoryTest.java +++ b/beacon/validator/src/test/java/tech/pegasys/teku/validator/coordinator/BlockOperationSelectorFactoryTest.java @@ -30,7 +30,6 @@ import org.junit.jupiter.api.Test; import tech.pegasys.teku.bls.BLSSignature; import tech.pegasys.teku.infrastructure.async.SafeFuture; -import tech.pegasys.teku.infrastructure.bytes.Bytes8; import tech.pegasys.teku.infrastructure.metrics.StubMetricsSystem; import tech.pegasys.teku.infrastructure.ssz.SszData; import tech.pegasys.teku.infrastructure.ssz.SszList; @@ -45,6 +44,7 @@ import tech.pegasys.teku.spec.datastructures.blocks.blockbody.common.AbstractSignedBeaconBlockUnblinder; import tech.pegasys.teku.spec.datastructures.blocks.blockbody.versions.altair.SyncAggregate; import tech.pegasys.teku.spec.datastructures.execution.ExecutionPayload; +import tech.pegasys.teku.spec.datastructures.execution.ExecutionPayloadContext; import tech.pegasys.teku.spec.datastructures.execution.ExecutionPayloadHeader; import tech.pegasys.teku.spec.datastructures.operations.Attestation; import tech.pegasys.teku.spec.datastructures.operations.AttesterSlashing; @@ -304,12 +304,13 @@ void shouldIncludeNonDefaultExecutionPayload() { final UInt64 slot = UInt64.ONE; final BeaconState blockSlotState = dataStructureUtil.randomBeaconState(slot); - final Bytes8 payloadId = dataStructureUtil.randomBytes8(); + final ExecutionPayloadContext executionPayloadContext = + dataStructureUtil.createPayloadExecutionContext(false); final ExecutionPayload randomExecutionPayload = dataStructureUtil.randomExecutionPayload(); when(forkChoiceNotifier.getPayloadId(any(), any())) - .thenReturn(SafeFuture.completedFuture(Optional.of(payloadId))); - when(executionLayer.engineGetPayload(payloadId, slot)) + .thenReturn(SafeFuture.completedFuture(Optional.of(executionPayloadContext))); + when(executionLayer.engineGetPayload(executionPayloadContext, slot)) .thenReturn(SafeFuture.completedFuture(randomExecutionPayload)); factory @@ -325,13 +326,14 @@ void shouldIncludeExecutionPayloadHeaderIfMevBoostEnabledAndBlindedBlockRequeste final UInt64 slot = UInt64.ONE; final BeaconState blockSlotState = dataStructureUtil.randomBeaconState(slot); - final Bytes8 payloadId = dataStructureUtil.randomBytes8(); + final ExecutionPayloadContext executionPayloadContext = + dataStructureUtil.createPayloadExecutionContext(false); final ExecutionPayloadHeader randomExecutionPayloadHeader = dataStructureUtil.randomExecutionPayloadHeader(); when(forkChoiceNotifier.getPayloadId(any(), any())) - .thenReturn(SafeFuture.completedFuture(Optional.of(payloadId))); - when(executionLayer.getPayloadHeader(payloadId, slot)) + .thenReturn(SafeFuture.completedFuture(Optional.of(executionPayloadContext))); + when(executionLayer.builderGetHeader(executionPayloadContext, slot)) .thenReturn(SafeFuture.completedFuture(randomExecutionPayloadHeader)); factoryWithMevBoost @@ -347,12 +349,13 @@ void shouldIncludeExecutionPayloadIfMevBoostEnabledButNoBlindedBlockRequested() final UInt64 slot = UInt64.ONE; final BeaconState blockSlotState = dataStructureUtil.randomBeaconState(slot); - final Bytes8 payloadId = dataStructureUtil.randomBytes8(); + final ExecutionPayloadContext executionPayloadContext = + dataStructureUtil.createPayloadExecutionContext(false); final ExecutionPayload randomExecutionPayload = dataStructureUtil.randomExecutionPayload(); when(forkChoiceNotifier.getPayloadId(any(), any())) - .thenReturn(SafeFuture.completedFuture(Optional.of(payloadId))); - when(executionLayer.engineGetPayload(payloadId, slot)) + .thenReturn(SafeFuture.completedFuture(Optional.of(executionPayloadContext))); + when(executionLayer.engineGetPayload(executionPayloadContext, slot)) .thenReturn(SafeFuture.completedFuture(randomExecutionPayload)); factoryWithMevBoost @@ -370,7 +373,7 @@ void shouldUnblindSignedBlindedBeaconBlockIfMevBoostEnabled() { final CapturingBeaconBlockUnblinder blockUnblinder = new CapturingBeaconBlockUnblinder(spec.getGenesisSchemaDefinitions(), blindedSignedBlock); - when(executionLayer.proposeBlindedBlock(blindedSignedBlock)) + when(executionLayer.builderGetPayload(blindedSignedBlock)) .thenReturn(SafeFuture.completedFuture(randomExecutionPayload)); factoryWithMevBoost.createUnblinderSelector().accept(blockUnblinder); @@ -385,7 +388,7 @@ void shouldThrowUnblindSignedBlindedBeaconBlockIfMevBoostDisabled() { final CapturingBeaconBlockUnblinder blockUnblinder = new CapturingBeaconBlockUnblinder(spec.getGenesisSchemaDefinitions(), blindedSignedBlock); - when(executionLayer.proposeBlindedBlock(blindedSignedBlock)) + when(executionLayer.builderGetPayload(blindedSignedBlock)) .thenReturn(SafeFuture.completedFuture(randomExecutionPayload)); assertThatThrownBy(() -> factory.createUnblinderSelector().accept(blockUnblinder)) diff --git a/ethereum/executionclient/src/main/java/tech/pegasys/teku/ethereum/executionclient/ExecutionEngineClient.java b/ethereum/executionclient/src/main/java/tech/pegasys/teku/ethereum/executionclient/ExecutionEngineClient.java index da1fb52998d..7be6bda76cd 100644 --- a/ethereum/executionclient/src/main/java/tech/pegasys/teku/ethereum/executionclient/ExecutionEngineClient.java +++ b/ethereum/executionclient/src/main/java/tech/pegasys/teku/ethereum/executionclient/ExecutionEngineClient.java @@ -15,7 +15,6 @@ import java.util.Optional; import org.apache.tuweni.bytes.Bytes32; -import tech.pegasys.teku.ethereum.executionclient.schema.ExecutionPayloadHeaderV1; import tech.pegasys.teku.ethereum.executionclient.schema.ExecutionPayloadV1; import tech.pegasys.teku.ethereum.executionclient.schema.ForkChoiceStateV1; import tech.pegasys.teku.ethereum.executionclient.schema.ForkChoiceUpdatedResult; @@ -25,7 +24,6 @@ import tech.pegasys.teku.ethereum.executionclient.schema.TransitionConfigurationV1; import tech.pegasys.teku.infrastructure.async.SafeFuture; import tech.pegasys.teku.infrastructure.bytes.Bytes8; -import tech.pegasys.teku.spec.datastructures.blocks.SignedBeaconBlock; import tech.pegasys.teku.spec.datastructures.execution.PowBlock; public interface ExecutionEngineClient { @@ -44,10 +42,4 @@ SafeFuture> forkChoiceUpdated( SafeFuture> exchangeTransitionConfiguration( TransitionConfigurationV1 transitionConfiguration); - - // builder namespace - SafeFuture> getPayloadHeader(Bytes8 payloadId); - - SafeFuture> proposeBlindedBlock( - SignedBeaconBlock signedBlindedBeaconBlock); } diff --git a/ethereum/executionclient/src/main/java/tech/pegasys/teku/ethereum/executionclient/ThrottlingExecutionEngineClient.java b/ethereum/executionclient/src/main/java/tech/pegasys/teku/ethereum/executionclient/ThrottlingExecutionEngineClient.java index cf884f1cbb2..143ec7259f4 100644 --- a/ethereum/executionclient/src/main/java/tech/pegasys/teku/ethereum/executionclient/ThrottlingExecutionEngineClient.java +++ b/ethereum/executionclient/src/main/java/tech/pegasys/teku/ethereum/executionclient/ThrottlingExecutionEngineClient.java @@ -16,7 +16,6 @@ import java.util.Optional; import org.apache.tuweni.bytes.Bytes32; import org.hyperledger.besu.plugin.services.MetricsSystem; -import tech.pegasys.teku.ethereum.executionclient.schema.ExecutionPayloadHeaderV1; import tech.pegasys.teku.ethereum.executionclient.schema.ExecutionPayloadV1; import tech.pegasys.teku.ethereum.executionclient.schema.ForkChoiceStateV1; import tech.pegasys.teku.ethereum.executionclient.schema.ForkChoiceUpdatedResult; @@ -28,7 +27,6 @@ import tech.pegasys.teku.infrastructure.async.ThrottlingTaskQueue; import tech.pegasys.teku.infrastructure.bytes.Bytes8; import tech.pegasys.teku.infrastructure.metrics.TekuMetricCategory; -import tech.pegasys.teku.spec.datastructures.blocks.SignedBeaconBlock; import tech.pegasys.teku.spec.datastructures.execution.PowBlock; public class ThrottlingExecutionEngineClient implements ExecutionEngineClient { @@ -83,15 +81,4 @@ public SafeFuture> exchangeTransitionConfigu return taskQueue.queueTask( () -> delegate.exchangeTransitionConfiguration(transitionConfiguration)); } - - @Override - public SafeFuture> getPayloadHeader(final Bytes8 payloadId) { - return taskQueue.queueTask(() -> delegate.getPayloadHeader(payloadId)); - } - - @Override - public SafeFuture> proposeBlindedBlock( - final SignedBeaconBlock signedBlindedBeaconBlock) { - return taskQueue.queueTask(() -> delegate.proposeBlindedBlock(signedBlindedBeaconBlock)); - } } diff --git a/ethereum/executionclient/src/main/java/tech/pegasys/teku/ethereum/executionclient/Web3JExecutionEngineClient.java b/ethereum/executionclient/src/main/java/tech/pegasys/teku/ethereum/executionclient/Web3JExecutionEngineClient.java index 9d366031f43..9e682f7ed7c 100644 --- a/ethereum/executionclient/src/main/java/tech/pegasys/teku/ethereum/executionclient/Web3JExecutionEngineClient.java +++ b/ethereum/executionclient/src/main/java/tech/pegasys/teku/ethereum/executionclient/Web3JExecutionEngineClient.java @@ -25,7 +25,6 @@ import org.web3j.protocol.core.DefaultBlockParameterName; import org.web3j.protocol.core.Request; import org.web3j.protocol.core.methods.response.EthBlock; -import tech.pegasys.teku.ethereum.executionclient.schema.ExecutionPayloadHeaderV1; import tech.pegasys.teku.ethereum.executionclient.schema.ExecutionPayloadV1; import tech.pegasys.teku.ethereum.executionclient.schema.ForkChoiceStateV1; import tech.pegasys.teku.ethereum.executionclient.schema.ForkChoiceUpdatedResult; @@ -36,7 +35,6 @@ import tech.pegasys.teku.infrastructure.async.SafeFuture; import tech.pegasys.teku.infrastructure.bytes.Bytes8; import tech.pegasys.teku.infrastructure.unsigned.UInt64; -import tech.pegasys.teku.spec.datastructures.blocks.SignedBeaconBlock; import tech.pegasys.teku.spec.datastructures.execution.PowBlock; public class Web3JExecutionEngineClient implements ExecutionEngineClient { @@ -127,33 +125,6 @@ public SafeFuture> exchangeTransitionConfigu return web3JClient.doRequest(web3jRequest, NON_EXECUTION_TIMEOUT); } - @Override - public SafeFuture> getPayloadHeader(Bytes8 payloadId) { - Request web3jRequest = - new Request<>( - "builder_getPayloadHeaderV1", - Collections.singletonList(payloadId.toHexString()), - web3JClient.getWeb3jService(), - ExecutionPayloadHeaderV1Web3jResponse.class); - return web3JClient.doRequest(web3jRequest, NON_EXECUTION_TIMEOUT); - } - - @Override - public SafeFuture> proposeBlindedBlock( - SignedBeaconBlock signedBlindedBeaconBlock) { - Request web3jRequest = - new Request<>( - "builder_proposeBlindedBlockV1", - Collections.singletonList(signedBlindedBeaconBlock), - web3JClient.getWeb3jService(), - ExecutionPayloadV1Web3jResponse.class); - return web3JClient.doRequest(web3jRequest, NON_EXECUTION_TIMEOUT); - } - - protected Web3JClient getWeb3JClient() { - return web3JClient; - } - static class ExecutionPayloadV1Web3jResponse extends org.web3j.protocol.core.Response {} @@ -166,9 +137,6 @@ static class ForkChoiceUpdatedResultWeb3jResponse static class TransitionConfigurationV1Web3jResponse extends org.web3j.protocol.core.Response {} - static class ExecutionPayloadHeaderV1Web3jResponse - extends org.web3j.protocol.core.Response {} - /** * Returns a list that supports null items. * diff --git a/ethereum/executionclient/src/main/java/tech/pegasys/teku/ethereum/executionclient/schema/BlindedBeaconBlockBodyV1.java b/ethereum/executionclient/src/main/java/tech/pegasys/teku/ethereum/executionclient/schema/BlindedBeaconBlockBodyV1.java index e8233721829..616e2482cca 100644 --- a/ethereum/executionclient/src/main/java/tech/pegasys/teku/ethereum/executionclient/schema/BlindedBeaconBlockBodyV1.java +++ b/ethereum/executionclient/src/main/java/tech/pegasys/teku/ethereum/executionclient/schema/BlindedBeaconBlockBodyV1.java @@ -120,62 +120,61 @@ public BlindedBeaconBlockBodyV1( final BeaconBlockBodySchema schema = getBeaconBlockBodySchema(spec); return schema.createBlockBody( - builder -> { - builder - .randaoReveal(randaoReveal.asInternalBLSSignature()) - .eth1Data( - new tech.pegasys.teku.spec.datastructures.blocks.Eth1Data( - eth1Data.depositRoot, eth1Data.depositCount, eth1Data.blockHash)) - .graffiti(graffiti) - .attestations( - attestations.stream() - .map(attestation -> attestation.asInternalAttestation(spec)) - .collect(schema.getAttestationsSchema().collector())) - .proposerSlashings( - proposerSlashings.stream() - .map(ProposerSlashingV1::asInternalProposerSlashing) - .collect(schema.getProposerSlashingsSchema().collector())) - .attesterSlashings( - attesterSlashings.stream() - .map(slashing -> slashing.asInternalAttesterSlashing(spec)) - .collect(schema.getAttesterSlashingsSchema().collector())) - .deposits( - deposits.stream() - .map(DepositV1::asInternalDeposit) - .collect(schema.getDepositsSchema().collector())) - .voluntaryExits( - voluntaryExits.stream() - .map( - voluntaryExit -> - new SignedVoluntaryExit( - voluntaryExit.getMessage().asInternalVoluntaryExit(), - voluntaryExit.getSignature().asInternalBLSSignature())) - .collect(schema.getVoluntaryExitsSchema().collector())) - .syncAggregate( - () -> - syncAggregateSchema.create( - syncAggregateSchema - .getSyncCommitteeBitsSchema() - .fromBytes(syncAggregate.syncCommitteeBits) - .getAllSetBits(), - syncAggregate.syncCommitteeSignature.asInternalBLSSignature())) - .executionPayloadHeader( - () -> - executionPayloadHeaderSchema.create( - executionPayloadHeader.parentHash, - executionPayloadHeader.feeRecipient, - executionPayloadHeader.stateRoot, - executionPayloadHeader.receiptsRoot, - executionPayloadHeader.logsBloom, - executionPayloadHeader.prevRandao, - executionPayloadHeader.blockNumber, - executionPayloadHeader.gasLimit, - executionPayloadHeader.gasUsed, - executionPayloadHeader.timestamp, - executionPayloadHeader.extraData, - executionPayloadHeader.baseFeePerGas, - executionPayloadHeader.blockHash, - executionPayloadHeader.transactionsRoot)); - }); + builder -> + builder + .randaoReveal(randaoReveal.asInternalBLSSignature()) + .eth1Data( + new tech.pegasys.teku.spec.datastructures.blocks.Eth1Data( + eth1Data.depositRoot, eth1Data.depositCount, eth1Data.blockHash)) + .graffiti(graffiti) + .attestations( + attestations.stream() + .map(attestation -> attestation.asInternalAttestation(spec)) + .collect(schema.getAttestationsSchema().collector())) + .proposerSlashings( + proposerSlashings.stream() + .map(ProposerSlashingV1::asInternalProposerSlashing) + .collect(schema.getProposerSlashingsSchema().collector())) + .attesterSlashings( + attesterSlashings.stream() + .map(slashing -> slashing.asInternalAttesterSlashing(spec)) + .collect(schema.getAttesterSlashingsSchema().collector())) + .deposits( + deposits.stream() + .map(DepositV1::asInternalDeposit) + .collect(schema.getDepositsSchema().collector())) + .voluntaryExits( + voluntaryExits.stream() + .map( + voluntaryExit -> + new SignedVoluntaryExit( + voluntaryExit.getMessage().asInternalVoluntaryExit(), + voluntaryExit.getSignature().asInternalBLSSignature())) + .collect(schema.getVoluntaryExitsSchema().collector())) + .syncAggregate( + () -> + syncAggregateSchema.create( + syncAggregateSchema + .getSyncCommitteeBitsSchema() + .fromBytes(syncAggregate.syncCommitteeBits) + .getAllSetBits(), + syncAggregate.syncCommitteeSignature.asInternalBLSSignature())) + .executionPayloadHeader( + () -> + executionPayloadHeaderSchema.create( + executionPayloadHeader.parentHash, + executionPayloadHeader.feeRecipient, + executionPayloadHeader.stateRoot, + executionPayloadHeader.receiptsRoot, + executionPayloadHeader.logsBloom, + executionPayloadHeader.prevRandao, + executionPayloadHeader.blockNumber, + executionPayloadHeader.gasLimit, + executionPayloadHeader.gasUsed, + executionPayloadHeader.timestamp, + executionPayloadHeader.extraData, + executionPayloadHeader.baseFeePerGas, + executionPayloadHeader.blockHash, + executionPayloadHeader.transactionsRoot))); } } diff --git a/ethereum/executionclient/src/main/java/tech/pegasys/teku/ethereum/executionclient/schema/BuilderBidV1.java b/ethereum/executionclient/src/main/java/tech/pegasys/teku/ethereum/executionclient/schema/BuilderBidV1.java index 59d2d1ad3e9..714a9653ff9 100644 --- a/ethereum/executionclient/src/main/java/tech/pegasys/teku/ethereum/executionclient/schema/BuilderBidV1.java +++ b/ethereum/executionclient/src/main/java/tech/pegasys/teku/ethereum/executionclient/schema/BuilderBidV1.java @@ -53,6 +53,18 @@ public BuilderBidV1( this.pubkey = pubkey; } + public ExecutionPayloadHeaderV1 getHeader() { + return header; + } + + public UInt256 getValue() { + return value; + } + + public BLSPubKey getPubkey() { + return pubkey; + } + @Override public boolean equals(final Object o) { if (this == o) { diff --git a/ethereum/executionlayer/src/main/java/tech/pegasys/teku/ethereum/executionlayer/ExecutionLayerChannelImpl.java b/ethereum/executionlayer/src/main/java/tech/pegasys/teku/ethereum/executionlayer/ExecutionLayerChannelImpl.java index f7ae02be8af..cc426e6c2a7 100644 --- a/ethereum/executionlayer/src/main/java/tech/pegasys/teku/ethereum/executionlayer/ExecutionLayerChannelImpl.java +++ b/ethereum/executionlayer/src/main/java/tech/pegasys/teku/ethereum/executionlayer/ExecutionLayerChannelImpl.java @@ -15,6 +15,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; import static tech.pegasys.teku.spec.config.Constants.MAXIMUM_CONCURRENT_EB_REQUESTS; import static tech.pegasys.teku.spec.config.Constants.MAXIMUM_CONCURRENT_EE_REQUESTS; @@ -22,6 +23,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.tuweni.bytes.Bytes32; +import org.apache.tuweni.bytes.Bytes48; import org.hyperledger.besu.plugin.services.MetricsSystem; import tech.pegasys.teku.ethereum.executionclient.ExecutionBuilderClient; import tech.pegasys.teku.ethereum.executionclient.ExecutionEngineClient; @@ -30,22 +32,25 @@ import tech.pegasys.teku.ethereum.executionclient.Web3JClient; import tech.pegasys.teku.ethereum.executionclient.Web3JExecutionBuilderClient; import tech.pegasys.teku.ethereum.executionclient.Web3JExecutionEngineClient; -import tech.pegasys.teku.ethereum.executionclient.schema.ExecutionPayloadHeaderV1; +import tech.pegasys.teku.ethereum.executionclient.schema.BlindedBeaconBlockV1; +import tech.pegasys.teku.ethereum.executionclient.schema.BuilderBidV1; import tech.pegasys.teku.ethereum.executionclient.schema.ExecutionPayloadV1; import tech.pegasys.teku.ethereum.executionclient.schema.ForkChoiceStateV1; import tech.pegasys.teku.ethereum.executionclient.schema.ForkChoiceUpdatedResult; import tech.pegasys.teku.ethereum.executionclient.schema.PayloadAttributesV1; import tech.pegasys.teku.ethereum.executionclient.schema.PayloadStatusV1; import tech.pegasys.teku.ethereum.executionclient.schema.Response; +import tech.pegasys.teku.ethereum.executionclient.schema.SignedMessage; import tech.pegasys.teku.ethereum.executionclient.schema.TransitionConfigurationV1; import tech.pegasys.teku.infrastructure.async.SafeFuture; -import tech.pegasys.teku.infrastructure.bytes.Bytes8; import tech.pegasys.teku.infrastructure.exceptions.InvalidConfigurationException; import tech.pegasys.teku.infrastructure.unsigned.UInt64; import tech.pegasys.teku.spec.Spec; import tech.pegasys.teku.spec.datastructures.blocks.SignedBeaconBlock; import tech.pegasys.teku.spec.datastructures.execution.ExecutionPayload; +import tech.pegasys.teku.spec.datastructures.execution.ExecutionPayloadContext; import tech.pegasys.teku.spec.datastructures.execution.ExecutionPayloadHeader; +import tech.pegasys.teku.spec.datastructures.execution.ExecutionPayloadHeaderSchema; import tech.pegasys.teku.spec.datastructures.execution.PowBlock; import tech.pegasys.teku.spec.executionlayer.ExecutionLayerChannel; import tech.pegasys.teku.spec.executionlayer.ForkChoiceState; @@ -58,8 +63,6 @@ public class ExecutionLayerChannelImpl implements ExecutionLayerChannel { private static final Logger LOG = LogManager.getLogger(); private final ExecutionEngineClient executionEngineClient; - - @SuppressWarnings("UnusedVariable") // will be removed in upcoming PR private final Optional executionBuilderClient; private final Spec spec; @@ -160,11 +163,13 @@ public SafeFuture eth1GetPowChainHead() { } @Override - public SafeFuture engineGetPayload(final Bytes8 payloadId, final UInt64 slot) { - LOG.trace("calling getPayload(payloadId={}, slot={})", payloadId, slot); + public SafeFuture engineGetPayload( + final ExecutionPayloadContext executionPayloadContext, final UInt64 slot) { + LOG.trace( + "calling getPayload(executionPayloadContext={}, slot={})", executionPayloadContext, slot); return executionEngineClient - .getPayload(payloadId) + .getPayload(executionPayloadContext.getPayloadId()) .thenApply(ExecutionLayerChannelImpl::unwrapResponseOrThrow) .thenCombine( SafeFuture.of( @@ -175,7 +180,10 @@ public SafeFuture engineGetPayload(final Bytes8 payloadId, fin .thenPeek( executionPayload -> LOG.trace( - "getPayload(payloadId={}, slot={}) -> {}", payloadId, slot, executionPayload)); + "getPayload(executionPayloadContext={}, slot={}) -> {}", + executionPayloadContext, + slot, + executionPayload)); } @Override @@ -213,39 +221,59 @@ public SafeFuture engineExchangeTransitionConfiguration } @Override - public SafeFuture getPayloadHeader( - final Bytes8 payloadId, final UInt64 slot) { - LOG.trace("calling getPayloadHeader(payloadId={}, slot={})", payloadId, slot); + public SafeFuture builderGetHeader( + final ExecutionPayloadContext executionPayloadContext, final UInt64 slot) { + checkState(executionBuilderClient.isPresent()); - return executionEngineClient - .getPayloadHeader(payloadId) + LOG.trace( + "calling builder builderGetHeader(payloadId={}, slot={})", executionPayloadContext, slot); + + return executionBuilderClient + .get() + .getHeader(slot, Bytes48.ZERO, executionPayloadContext.getParentHash()) .thenApply(ExecutionLayerChannelImpl::unwrapResponseOrThrow) - .thenCombine( - SafeFuture.of( - () -> - SchemaDefinitionsBellatrix.required(spec.atSlot(slot).getSchemaDefinitions()) - .getExecutionPayloadHeaderSchema()), - ExecutionPayloadHeaderV1::asInternalExecutionPayloadHeader) + .thenApply( + builderBidV1SignedMessage -> + getExecutionHeaderFromBuilderBid(builderBidV1SignedMessage, slot)) .thenPeek( executionPayloadHeader -> LOG.trace( - "getPayloadHeader(payloadId={}, slot={}) -> {}", - payloadId, + "builderGetHeader(payloadId={}, slot={}) -> {}", + executionPayloadContext, slot, executionPayloadHeader)); } + private ExecutionPayloadHeader getExecutionHeaderFromBuilderBid( + SignedMessage signedBuilderBid, UInt64 slot) { + ExecutionPayloadHeaderSchema executionPayloadHeaderSchema = + SchemaDefinitionsBellatrix.required(spec.atSlot(slot).getSchemaDefinitions()) + .getExecutionPayloadHeaderSchema(); + // validate signature + + return signedBuilderBid + .getMessage() + .getHeader() + .asInternalExecutionPayloadHeader(executionPayloadHeaderSchema); + } + @Override - public SafeFuture proposeBlindedBlock( + public SafeFuture builderGetPayload( SignedBeaconBlock signedBlindedBeaconBlock) { - LOG.trace("calling proposeBlindedBlock(signedBlindedBeaconBlock={})", signedBlindedBeaconBlock); + LOG.trace("calling builderGetPayload(signedBlindedBeaconBlock={})", signedBlindedBeaconBlock); checkArgument( signedBlindedBeaconBlock.getMessage().getBody().isBlinded(), "SignedBeaconBlock must be blind"); - return executionEngineClient - .proposeBlindedBlock(signedBlindedBeaconBlock) + checkState(executionBuilderClient.isPresent()); + + return executionBuilderClient + .get() + .getPayload( + new SignedMessage<>( + new BlindedBeaconBlockV1(signedBlindedBeaconBlock.getMessage()), + signedBlindedBeaconBlock.getSignature())) .thenApply(ExecutionLayerChannelImpl::unwrapResponseOrThrow) .thenCombine( SafeFuture.of( @@ -257,7 +285,7 @@ public SafeFuture proposeBlindedBlock( .thenPeek( executionPayload -> LOG.trace( - "proposeBlindedBlock(signedBlindedBeaconBlock={}) -> {}", + "builderGetPayload(signedBlindedBeaconBlock={}) -> {}", signedBlindedBeaconBlock, executionPayload)); } diff --git a/ethereum/spec/src/main/java/tech/pegasys/teku/spec/datastructures/execution/ExecutionPayloadContext.java b/ethereum/spec/src/main/java/tech/pegasys/teku/spec/datastructures/execution/ExecutionPayloadContext.java new file mode 100644 index 00000000000..ea57ca48f4e --- /dev/null +++ b/ethereum/spec/src/main/java/tech/pegasys/teku/spec/datastructures/execution/ExecutionPayloadContext.java @@ -0,0 +1,84 @@ +/* + * Copyright 2022 ConsenSys AG. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package tech.pegasys.teku.spec.datastructures.execution; + +import com.google.common.base.MoreObjects; +import java.util.Objects; +import java.util.Optional; +import org.apache.tuweni.bytes.Bytes32; +import tech.pegasys.teku.infrastructure.bytes.Bytes8; +import tech.pegasys.teku.spec.executionlayer.ForkChoiceState; +import tech.pegasys.teku.spec.executionlayer.PayloadAttributes; + +public class ExecutionPayloadContext { + private final Bytes8 payloadId; + private final ForkChoiceState forkChoiceState; + private final Optional payloadAttributes; + + public ExecutionPayloadContext( + final Bytes8 payloadId, + final ForkChoiceState forkChoiceState, + final Optional payloadAttributes) { + this.payloadId = payloadId; + this.forkChoiceState = forkChoiceState; + this.payloadAttributes = payloadAttributes; + } + + public Bytes8 getPayloadId() { + return payloadId; + } + + public ForkChoiceState getForkChoiceState() { + return forkChoiceState; + } + + public Optional getPayloadAttributes() { + return payloadAttributes; + } + + public Bytes32 getParentHash() { + return forkChoiceState.getHeadExecutionBlockHash(); + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final ExecutionPayloadContext that = (ExecutionPayloadContext) o; + return Objects.equals(payloadId, that.payloadId) + && Objects.equals(forkChoiceState, that.forkChoiceState) + && Objects.equals(payloadAttributes, that.payloadAttributes); + } + + @Override + public int hashCode() { + return Objects.hash( + payloadId, + forkChoiceState, + payloadAttributes); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("payloadId", payloadId) + .add("forkChoiceState", forkChoiceState) + .add("payloadAttributes", payloadAttributes) + .toString(); + } +} diff --git a/ethereum/spec/src/main/java/tech/pegasys/teku/spec/executionlayer/ExecutionLayerChannel.java b/ethereum/spec/src/main/java/tech/pegasys/teku/spec/executionlayer/ExecutionLayerChannel.java index e9396f9ea4c..f88c909fb7b 100644 --- a/ethereum/spec/src/main/java/tech/pegasys/teku/spec/executionlayer/ExecutionLayerChannel.java +++ b/ethereum/spec/src/main/java/tech/pegasys/teku/spec/executionlayer/ExecutionLayerChannel.java @@ -16,11 +16,11 @@ import java.util.Optional; import org.apache.tuweni.bytes.Bytes32; import tech.pegasys.teku.infrastructure.async.SafeFuture; -import tech.pegasys.teku.infrastructure.bytes.Bytes8; import tech.pegasys.teku.infrastructure.events.ChannelInterface; import tech.pegasys.teku.infrastructure.unsigned.UInt64; import tech.pegasys.teku.spec.datastructures.blocks.SignedBeaconBlock; import tech.pegasys.teku.spec.datastructures.execution.ExecutionPayload; +import tech.pegasys.teku.spec.datastructures.execution.ExecutionPayloadContext; import tech.pegasys.teku.spec.datastructures.execution.ExecutionPayloadHeader; import tech.pegasys.teku.spec.datastructures.execution.PowBlock; @@ -48,7 +48,7 @@ public SafeFuture engineForkChoiceUpdated( @Override public SafeFuture engineGetPayload( - final Bytes8 payloadId, final UInt64 slot) { + final ExecutionPayloadContext executionPayloadContext, final UInt64 slot) { return SafeFuture.completedFuture(null); } @@ -64,13 +64,13 @@ public SafeFuture engineExchangeTransitionConfiguration } @Override - public SafeFuture getPayloadHeader( - final Bytes8 payloadId, final UInt64 slot) { + public SafeFuture builderGetHeader( + final ExecutionPayloadContext executionPayloadContext, final UInt64 slot) { return SafeFuture.completedFuture(null); } @Override - public SafeFuture proposeBlindedBlock( + public SafeFuture builderGetPayload( SignedBeaconBlock signedBlindedBeaconBlock) { return SafeFuture.completedFuture(null); } @@ -85,7 +85,8 @@ public SafeFuture proposeBlindedBlock( SafeFuture engineForkChoiceUpdated( final ForkChoiceState forkChoiceState, final Optional payloadAttributes); - SafeFuture engineGetPayload(final Bytes8 payloadId, final UInt64 slot); + SafeFuture engineGetPayload( + final ExecutionPayloadContext executionPayloadContext, final UInt64 slot); SafeFuture engineNewPayload(final ExecutionPayload executionPayload); @@ -93,10 +94,10 @@ SafeFuture engineExchangeTransitionConfiguration( final TransitionConfiguration transitionConfiguration); // builder namespace - SafeFuture getPayloadHeader(final Bytes8 payloadId, final UInt64 slot); + SafeFuture builderGetHeader( + final ExecutionPayloadContext executionPayloadContext, final UInt64 slot); - SafeFuture proposeBlindedBlock( - final SignedBeaconBlock signedBlindedBeaconBlock); + SafeFuture builderGetPayload(final SignedBeaconBlock signedBlindedBeaconBlock); enum Version { KILNV2; diff --git a/ethereum/spec/src/main/java/tech/pegasys/teku/spec/executionlayer/ExecutionLayerChannelStub.java b/ethereum/spec/src/main/java/tech/pegasys/teku/spec/executionlayer/ExecutionLayerChannelStub.java index 7f86dd0b9fe..4cbb493ff00 100644 --- a/ethereum/spec/src/main/java/tech/pegasys/teku/spec/executionlayer/ExecutionLayerChannelStub.java +++ b/ethereum/spec/src/main/java/tech/pegasys/teku/spec/executionlayer/ExecutionLayerChannelStub.java @@ -40,6 +40,7 @@ import tech.pegasys.teku.spec.config.SpecConfigBellatrix; import tech.pegasys.teku.spec.datastructures.blocks.SignedBeaconBlock; import tech.pegasys.teku.spec.datastructures.execution.ExecutionPayload; +import tech.pegasys.teku.spec.datastructures.execution.ExecutionPayloadContext; import tech.pegasys.teku.spec.datastructures.execution.ExecutionPayloadHeader; import tech.pegasys.teku.spec.datastructures.execution.PowBlock; import tech.pegasys.teku.spec.schemas.SchemaDefinitionsBellatrix; @@ -159,7 +160,8 @@ public SafeFuture engineForkChoiceUpdated( } @Override - public SafeFuture engineGetPayload(final Bytes8 payloadId, final UInt64 slot) { + public SafeFuture engineGetPayload( + final ExecutionPayloadContext executionPayloadContext, final UInt64 slot) { if (!bellatrixActivationDetected) { LOG.info( "getPayload received before terminalBlock has been sent. Assuming transition already happened"); @@ -178,7 +180,7 @@ public SafeFuture engineGetPayload(final Bytes8 payloadId, fin } final Optional maybeHeadAndAttrs = - payloadIdToHeadAndAttrsCache.getCached(payloadId); + payloadIdToHeadAndAttrsCache.getCached(executionPayloadContext.getPayloadId()); if (maybeHeadAndAttrs.isEmpty()) { return SafeFuture.failedFuture(new RuntimeException("payloadId not found in cache")); } @@ -217,7 +219,7 @@ public SafeFuture engineGetPayload(final Bytes8 payloadId, fin LOG.info( "getPayload: payloadId: {} slot: {} -> executionPayload blockHash: {}", - payloadId, + executionPayloadContext.getPayloadId(), slot, executionPayload.getBlockHash()); @@ -257,19 +259,19 @@ public SafeFuture engineExchangeTransitionConfiguration } @Override - public SafeFuture getPayloadHeader( - final Bytes8 payloadId, final UInt64 slot) { + public SafeFuture builderGetHeader( + final ExecutionPayloadContext executionPayloadContext, final UInt64 slot) { LOG.info( "getPayloadHeader: payloadId: {} slot: {} ... delegating to getPayload ...", - payloadId, + executionPayloadContext, slot); - return engineGetPayload(payloadId, slot) + return engineGetPayload(executionPayloadContext, slot) .thenApply( executionPayload -> { LOG.info( "getPayloadHeader: payloadId: {} slot: {} -> executionPayload blockHash: {}", - payloadId, + executionPayloadContext, slot, executionPayload.getBlockHash()); lastMevBoostPayloadToBeUnblinded = Optional.of(executionPayload); @@ -283,7 +285,7 @@ public SafeFuture getPayloadHeader( } @Override - public SafeFuture proposeBlindedBlock( + public SafeFuture builderGetPayload( SignedBeaconBlock signedBlindedBeaconBlock) { final Optional schemaDefinitionsBellatrix = spec.atSlot(signedBlindedBeaconBlock.getSlot()).getSchemaDefinitions().toVersionBellatrix(); diff --git a/ethereum/spec/src/testFixtures/java/tech/pegasys/teku/spec/util/DataStructureUtil.java b/ethereum/spec/src/testFixtures/java/tech/pegasys/teku/spec/util/DataStructureUtil.java index 884315e795d..b810ba0c0de 100644 --- a/ethereum/spec/src/testFixtures/java/tech/pegasys/teku/spec/util/DataStructureUtil.java +++ b/ethereum/spec/src/testFixtures/java/tech/pegasys/teku/spec/util/DataStructureUtil.java @@ -82,6 +82,7 @@ import tech.pegasys.teku.spec.datastructures.blocks.blockbody.versions.altair.SyncAggregateSchema; import tech.pegasys.teku.spec.datastructures.eth1.Eth1Address; import tech.pegasys.teku.spec.datastructures.execution.ExecutionPayload; +import tech.pegasys.teku.spec.datastructures.execution.ExecutionPayloadContext; import tech.pegasys.teku.spec.datastructures.execution.ExecutionPayloadHeader; import tech.pegasys.teku.spec.datastructures.forkchoice.VoteTracker; import tech.pegasys.teku.spec.datastructures.networking.libp2p.rpc.EnrForkId; @@ -1087,6 +1088,11 @@ public EnrForkId randomEnrForkId() { return new EnrForkId(randomBytes4(), randomBytes4(), randomUInt64()); } + public ExecutionPayloadContext createPayloadExecutionContext(final boolean optimisticHead) { + return new ExecutionPayloadContext( + randomBytes8(), randomForkChoiceState(optimisticHead), Optional.empty()); + } + public ForkChoiceState randomForkChoiceState(final boolean optimisticHead) { return randomForkChoiceState(randomUInt64(), optimisticHead); } diff --git a/ethereum/statetransition/src/main/java/tech/pegasys/teku/statetransition/forkchoice/ForkChoiceNotifier.java b/ethereum/statetransition/src/main/java/tech/pegasys/teku/statetransition/forkchoice/ForkChoiceNotifier.java index e5fafb1b200..7a0ba0edef3 100644 --- a/ethereum/statetransition/src/main/java/tech/pegasys/teku/statetransition/forkchoice/ForkChoiceNotifier.java +++ b/ethereum/statetransition/src/main/java/tech/pegasys/teku/statetransition/forkchoice/ForkChoiceNotifier.java @@ -17,8 +17,8 @@ import java.util.Optional; import org.apache.tuweni.bytes.Bytes32; import tech.pegasys.teku.infrastructure.async.SafeFuture; -import tech.pegasys.teku.infrastructure.bytes.Bytes8; import tech.pegasys.teku.infrastructure.unsigned.UInt64; +import tech.pegasys.teku.spec.datastructures.execution.ExecutionPayloadContext; import tech.pegasys.teku.spec.datastructures.operations.versions.bellatrix.BeaconPreparableProposer; import tech.pegasys.teku.spec.executionlayer.ForkChoiceState; @@ -31,7 +31,8 @@ public interface ForkChoiceNotifier { void onSyncingStatusChanged(boolean inSync); - SafeFuture> getPayloadId(Bytes32 parentBeaconBlockRoot, UInt64 blockSlot); + SafeFuture> getPayloadId( + Bytes32 parentBeaconBlockRoot, UInt64 blockSlot); void onTerminalBlockReached(Bytes32 executionBlockHash); diff --git a/ethereum/statetransition/src/main/java/tech/pegasys/teku/statetransition/forkchoice/ForkChoiceNotifierImpl.java b/ethereum/statetransition/src/main/java/tech/pegasys/teku/statetransition/forkchoice/ForkChoiceNotifierImpl.java index 4a197daae25..b26c43a6bf9 100644 --- a/ethereum/statetransition/src/main/java/tech/pegasys/teku/statetransition/forkchoice/ForkChoiceNotifierImpl.java +++ b/ethereum/statetransition/src/main/java/tech/pegasys/teku/statetransition/forkchoice/ForkChoiceNotifierImpl.java @@ -22,12 +22,12 @@ import tech.pegasys.teku.infrastructure.async.SafeFuture; import tech.pegasys.teku.infrastructure.async.eventthread.AsyncRunnerEventThread; import tech.pegasys.teku.infrastructure.async.eventthread.EventThread; -import tech.pegasys.teku.infrastructure.bytes.Bytes8; import tech.pegasys.teku.infrastructure.subscribers.Subscribers; import tech.pegasys.teku.infrastructure.unsigned.UInt64; import tech.pegasys.teku.spec.Spec; import tech.pegasys.teku.spec.config.SpecConfig; import tech.pegasys.teku.spec.datastructures.eth1.Eth1Address; +import tech.pegasys.teku.spec.datastructures.execution.ExecutionPayloadContext; import tech.pegasys.teku.spec.datastructures.operations.versions.bellatrix.BeaconPreparableProposer; import tech.pegasys.teku.spec.executionlayer.ExecutionLayerChannel; import tech.pegasys.teku.spec.executionlayer.ForkChoiceState; @@ -116,7 +116,7 @@ public void onSyncingStatusChanged(final boolean inSync) { } @Override - public SafeFuture> getPayloadId( + public SafeFuture> getPayloadId( final Bytes32 parentBeaconBlockRoot, final UInt64 blockSlot) { return eventThread.executeFuture(() -> internalGetPayloadId(parentBeaconBlockRoot, blockSlot)); } @@ -148,7 +148,7 @@ private void internalTerminalBlockReached(Bytes32 executionBlockHash) { * parentBeaconBlockRoot 2. builds on top of the terminal block *

in all other cases it must Throw to avoid block production */ - private SafeFuture> internalGetPayloadId( + private SafeFuture> internalGetPayloadId( final Bytes32 parentBeaconBlockRoot, final UInt64 blockSlot) { eventThread.checkOnEventThread(); @@ -167,7 +167,7 @@ private SafeFuture> internalGetPayloadId( final UInt64 timestamp = spec.getSlotStartTime(blockSlot, recentChainData.getGenesisTime()); if (forkChoiceUpdateData.isPayloadIdSuitable(parentExecutionHash, timestamp)) { - return forkChoiceUpdateData.getPayloadId(); + return forkChoiceUpdateData.getExecutionPayloadContext(); } else if (parentExecutionHash.isZero() && !forkChoiceUpdateData.hasTerminalBlockHash()) { // Pre-merge so ok to use default payload return SafeFuture.completedFuture(Optional.empty()); @@ -188,7 +188,7 @@ private SafeFuture> internalGetPayloadId( localForkChoiceUpdateData.withPayloadAttributes(newPayloadAttributes); sendForkChoiceUpdated(); return forkChoiceUpdateData - .getPayloadId() + .getExecutionPayloadContext() .thenApply( payloadId -> { if (payloadId.isEmpty()) { diff --git a/ethereum/statetransition/src/main/java/tech/pegasys/teku/statetransition/forkchoice/ForkChoiceUpdateData.java b/ethereum/statetransition/src/main/java/tech/pegasys/teku/statetransition/forkchoice/ForkChoiceUpdateData.java index 2a88f23ebf9..11076bbe50a 100644 --- a/ethereum/statetransition/src/main/java/tech/pegasys/teku/statetransition/forkchoice/ForkChoiceUpdateData.java +++ b/ethereum/statetransition/src/main/java/tech/pegasys/teku/statetransition/forkchoice/ForkChoiceUpdateData.java @@ -21,8 +21,8 @@ import org.apache.logging.log4j.Logger; import org.apache.tuweni.bytes.Bytes32; import tech.pegasys.teku.infrastructure.async.SafeFuture; -import tech.pegasys.teku.infrastructure.bytes.Bytes8; import tech.pegasys.teku.infrastructure.unsigned.UInt64; +import tech.pegasys.teku.spec.datastructures.execution.ExecutionPayloadContext; import tech.pegasys.teku.spec.executionlayer.ExecutionLayerChannel; import tech.pegasys.teku.spec.executionlayer.ForkChoiceState; import tech.pegasys.teku.spec.executionlayer.ForkChoiceUpdatedResult; @@ -37,7 +37,8 @@ public class ForkChoiceUpdateData { private final ForkChoiceState forkChoiceState; private final Optional payloadAttributes; private final Optional terminalBlockHash; - private final SafeFuture> payloadId = new SafeFuture<>(); + private final SafeFuture> executionPayloadContext = + new SafeFuture<>(); private boolean sent = false; private long payloadAttributesSequenceProducer = 0; @@ -146,8 +147,8 @@ public boolean isPayloadIdSuitable(final Bytes32 parentExecutionHash, final UInt } } - public SafeFuture> getPayloadId() { - return payloadId; + public SafeFuture> getExecutionPayloadContext() { + return executionPayloadContext; } public SafeFuture> send( @@ -160,7 +161,7 @@ public SafeFuture> send( if (forkChoiceState.getHeadExecutionBlockHash().isZero()) { LOG.debug("send - getHeadBlockHash is zero - returning empty"); - payloadId.complete(Optional.empty()); + executionPayloadContext.complete(Optional.empty()); return SafeFuture.completedFuture(Optional.empty()); } @@ -177,7 +178,12 @@ public SafeFuture> send( payloadId, forkChoiceState, payloadAttributes)) - .propagateTo(payloadId); + .thenApply( + maybePayloadId -> + maybePayloadId.map( + payloadId -> + new ExecutionPayloadContext(payloadId, forkChoiceState, payloadAttributes))) + .propagateTo(executionPayloadContext); return forkChoiceUpdatedResult.thenApply(Optional::of); } @@ -200,7 +206,7 @@ public String toString() { .add("forkChoiceState", forkChoiceState) .add("payloadAttributes", payloadAttributes) .add("terminalBlockHash", terminalBlockHash) - .add("payloadId", payloadId) + .add("payloadId", executionPayloadContext) .toString(); } } diff --git a/ethereum/statetransition/src/test/java/tech/pegasys/teku/statetransition/forkchoice/ForkChoiceNotifierTest.java b/ethereum/statetransition/src/test/java/tech/pegasys/teku/statetransition/forkchoice/ForkChoiceNotifierTest.java index bd9133f0b22..7530e0fe4dc 100644 --- a/ethereum/statetransition/src/test/java/tech/pegasys/teku/statetransition/forkchoice/ForkChoiceNotifierTest.java +++ b/ethereum/statetransition/src/test/java/tech/pegasys/teku/statetransition/forkchoice/ForkChoiceNotifierTest.java @@ -44,6 +44,7 @@ import tech.pegasys.teku.spec.datastructures.blocks.SignedBlockAndState; import tech.pegasys.teku.spec.datastructures.blocks.SlotAndBlockRoot; import tech.pegasys.teku.spec.datastructures.eth1.Eth1Address; +import tech.pegasys.teku.spec.datastructures.execution.ExecutionPayloadContext; import tech.pegasys.teku.spec.datastructures.forkchoice.ReadOnlyForkChoiceStrategy; import tech.pegasys.teku.spec.datastructures.operations.versions.bellatrix.BeaconPreparableProposer; import tech.pegasys.teku.spec.datastructures.state.beaconstate.BeaconState; @@ -478,10 +479,12 @@ void getPayloadId_shouldReturnLatestPayloadId() { assertThatSafeFuture(notifier.getPayloadId(blockRoot, blockSlot)).isNotCompleted(); // But becomes available once we receive the response + final ExecutionPayloadContext executionPayloadContext = + new ExecutionPayloadContext(payloadId, forkChoiceState, Optional.of(payloadAttributes)); responseFuture.complete( createForkChoiceUpdatedResult(ExecutionPayloadStatus.VALID, Optional.of(payloadId))); assertThatSafeFuture(notifier.getPayloadId(blockRoot, blockSlot)) - .isCompletedWithOptionalContaining(payloadId); + .isCompletedWithOptionalContaining(executionPayloadContext); } @Test @@ -707,16 +710,20 @@ private void validateGetPayloadIdOnTheFlyRetrieval( .thenReturn(responseFuture); // Initially has no payload ID. - SafeFuture> futurePayloadId = notifier.getPayloadId(blockRoot, blockSlot); - assertThatSafeFuture(futurePayloadId).isNotCompleted(); + SafeFuture> futureExecutionPayloadContext = + notifier.getPayloadId(blockRoot, blockSlot); + assertThatSafeFuture(futureExecutionPayloadContext).isNotCompleted(); responseFuture.complete( createForkChoiceUpdatedResult(ExecutionPayloadStatus.VALID, Optional.of(payloadId))); if (mustFail) { - assertThatSafeFuture(futurePayloadId).isCompletedExceptionally(); + assertThatSafeFuture(futureExecutionPayloadContext).isCompletedExceptionally(); } else { - assertThatSafeFuture(futurePayloadId).isCompletedWithOptionalContaining(payloadId); + final ExecutionPayloadContext executionPayloadContext = + new ExecutionPayloadContext(payloadId, forkChoiceState, Optional.of(payloadAttributes)); + assertThatSafeFuture(futureExecutionPayloadContext) + .isCompletedWithOptionalContaining(executionPayloadContext); } } diff --git a/ethereum/statetransition/src/testFixtures/java/tech/pegasys/teku/statetransition/forkchoice/StubForkChoiceNotifier.java b/ethereum/statetransition/src/testFixtures/java/tech/pegasys/teku/statetransition/forkchoice/StubForkChoiceNotifier.java index f5136d12320..f99e3f99597 100644 --- a/ethereum/statetransition/src/testFixtures/java/tech/pegasys/teku/statetransition/forkchoice/StubForkChoiceNotifier.java +++ b/ethereum/statetransition/src/testFixtures/java/tech/pegasys/teku/statetransition/forkchoice/StubForkChoiceNotifier.java @@ -17,9 +17,9 @@ import java.util.Optional; import org.apache.tuweni.bytes.Bytes32; import tech.pegasys.teku.infrastructure.async.SafeFuture; -import tech.pegasys.teku.infrastructure.bytes.Bytes8; import tech.pegasys.teku.infrastructure.subscribers.Subscribers; import tech.pegasys.teku.infrastructure.unsigned.UInt64; +import tech.pegasys.teku.spec.datastructures.execution.ExecutionPayloadContext; import tech.pegasys.teku.spec.datastructures.operations.versions.bellatrix.BeaconPreparableProposer; import tech.pegasys.teku.spec.executionlayer.ForkChoiceState; import tech.pegasys.teku.spec.executionlayer.ForkChoiceUpdatedResult; @@ -65,7 +65,7 @@ public void onAttestationsDue(UInt64 slot) {} public void onSyncingStatusChanged(boolean inSync) {} @Override - public SafeFuture> getPayloadId( + public SafeFuture> getPayloadId( Bytes32 parentBeaconBlockRoot, UInt64 blockSlot) { return null; } diff --git a/services/executionlayer/src/main/java/tech/pegasys/teku/services/executionlayer/ExecutionLayerConfiguration.java b/services/executionlayer/src/main/java/tech/pegasys/teku/services/executionlayer/ExecutionLayerConfiguration.java index 0374b4a43c8..1119b2b4b34 100644 --- a/services/executionlayer/src/main/java/tech/pegasys/teku/services/executionlayer/ExecutionLayerConfiguration.java +++ b/services/executionlayer/src/main/java/tech/pegasys/teku/services/executionlayer/ExecutionLayerConfiguration.java @@ -13,6 +13,9 @@ package tech.pegasys.teku.services.executionlayer; +import static com.google.common.base.Preconditions.checkState; +import static tech.pegasys.teku.spec.executionlayer.ExecutionLayerChannel.STUB_ENDPOINT_IDENTIFIER; + import java.util.Optional; import org.apache.commons.lang3.StringUtils; import tech.pegasys.teku.infrastructure.exceptions.InvalidConfigurationException; @@ -82,7 +85,7 @@ public static class Builder { private Builder() {} public ExecutionLayerConfiguration build() { - + validateStubEndpoints(); return new ExecutionLayerConfiguration( spec, engineEndpoint, engineVersion, engineJwtSecretFile, builderEndpoint); } @@ -111,5 +114,16 @@ public Builder builderEndpoint(final String builderEndpoint) { this.builderEndpoint = Optional.ofNullable(builderEndpoint); return this; } + + private void validateStubEndpoints() { + final boolean builderIsStub = + engineEndpoint.map(endpoint -> endpoint.equals(STUB_ENDPOINT_IDENTIFIER)).orElse(false); + final boolean engineIsStub = + builderEndpoint.map(endpoint -> endpoint.equals(STUB_ENDPOINT_IDENTIFIER)).orElse(false); + + checkState( + engineIsStub == builderIsStub, + "mixed configuration with stubbed and non-stubbed execution layer endpoints is not supported"); + } } } From 77650921345d6805f9874002c1b71ea8b8746aeb Mon Sep 17 00:00:00 2001 From: Enrico Del Fante Date: Mon, 9 May 2022 14:06:15 +0200 Subject: [PATCH 2/3] spotlessly --- .../datastructures/execution/ExecutionPayloadContext.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/ethereum/spec/src/main/java/tech/pegasys/teku/spec/datastructures/execution/ExecutionPayloadContext.java b/ethereum/spec/src/main/java/tech/pegasys/teku/spec/datastructures/execution/ExecutionPayloadContext.java index ea57ca48f4e..0ad6d361d5a 100644 --- a/ethereum/spec/src/main/java/tech/pegasys/teku/spec/datastructures/execution/ExecutionPayloadContext.java +++ b/ethereum/spec/src/main/java/tech/pegasys/teku/spec/datastructures/execution/ExecutionPayloadContext.java @@ -67,10 +67,7 @@ public boolean equals(final Object o) { @Override public int hashCode() { - return Objects.hash( - payloadId, - forkChoiceState, - payloadAttributes); + return Objects.hash(payloadId, forkChoiceState, payloadAttributes); } @Override From dfa677fe9811a3545e2e6028137d9c5680440a4f Mon Sep 17 00:00:00 2001 From: Enrico Del Fante Date: Mon, 9 May 2022 15:28:25 +0200 Subject: [PATCH 3/3] fix trace logs --- .../ExecutionLayerChannelImpl.java | 40 +++++++++++-------- 1 file changed, 24 insertions(+), 16 deletions(-) diff --git a/ethereum/executionlayer/src/main/java/tech/pegasys/teku/ethereum/executionlayer/ExecutionLayerChannelImpl.java b/ethereum/executionlayer/src/main/java/tech/pegasys/teku/ethereum/executionlayer/ExecutionLayerChannelImpl.java index cc426e6c2a7..f3f55a3519f 100644 --- a/ethereum/executionlayer/src/main/java/tech/pegasys/teku/ethereum/executionlayer/ExecutionLayerChannelImpl.java +++ b/ethereum/executionlayer/src/main/java/tech/pegasys/teku/ethereum/executionlayer/ExecutionLayerChannelImpl.java @@ -120,20 +120,21 @@ private static K unwrapResponseOrThrow(Response response) { @Override public SafeFuture> eth1GetPowBlock(final Bytes32 blockHash) { - LOG.trace("calling getPowBlock(blockHash={})", blockHash); + LOG.trace("calling eth1GetPowBlock(blockHash={})", blockHash); return executionEngineClient .getPowBlock(blockHash) - .thenPeek(powBlock -> LOG.trace("getPowBlock(blockHash={}) -> {}", blockHash, powBlock)); + .thenPeek( + powBlock -> LOG.trace("eth1GetPowBlock(blockHash={}) -> {}", blockHash, powBlock)); } @Override public SafeFuture eth1GetPowChainHead() { - LOG.trace("calling getPowChainHead()"); + LOG.trace("calling eth1GetPowChainHead()"); return executionEngineClient .getPowChainHead() - .thenPeek(powBlock -> LOG.trace("getPowChainHead() -> {}", powBlock)); + .thenPeek(powBlock -> LOG.trace("eth1GetPowChainHead() -> {}", powBlock)); } @Override @@ -143,7 +144,7 @@ public SafeFuture eth1GetPowChainHead() { final Optional payloadAttributes) { LOG.trace( - "calling forkChoiceUpdated(forkChoiceState={}, payloadAttributes={})", + "calling engineForkChoiceUpdated(forkChoiceState={}, payloadAttributes={})", forkChoiceState, payloadAttributes); @@ -156,7 +157,7 @@ public SafeFuture eth1GetPowChainHead() { .thenPeek( forkChoiceUpdatedResult -> LOG.trace( - "forkChoiceUpdated(forkChoiceState={}, payloadAttributes={}) -> {}", + "engineForkChoiceUpdated(forkChoiceState={}, payloadAttributes={}) -> {}", forkChoiceState, payloadAttributes, forkChoiceUpdatedResult)); @@ -166,7 +167,9 @@ public SafeFuture eth1GetPowChainHead() { public SafeFuture engineGetPayload( final ExecutionPayloadContext executionPayloadContext, final UInt64 slot) { LOG.trace( - "calling getPayload(executionPayloadContext={}, slot={})", executionPayloadContext, slot); + "calling engineGetPayload(payloadId={}, slot={})", + executionPayloadContext.getPayloadId(), + slot); return executionEngineClient .getPayload(executionPayloadContext.getPayloadId()) @@ -180,15 +183,15 @@ public SafeFuture engineGetPayload( .thenPeek( executionPayload -> LOG.trace( - "getPayload(executionPayloadContext={}, slot={}) -> {}", - executionPayloadContext, + "engineGetPayload(payloadId={}, slot={}) -> {}", + executionPayloadContext.getPayloadId(), slot, executionPayload)); } @Override public SafeFuture engineNewPayload(final ExecutionPayload executionPayload) { - LOG.trace("calling newPayload(executionPayload={})", executionPayload); + LOG.trace("calling engineNewPayload(executionPayload={})", executionPayload); return executionEngineClient .newPayload(ExecutionPayloadV1.fromInternalExecutionPayload(executionPayload)) @@ -196,7 +199,8 @@ public SafeFuture engineNewPayload(final ExecutionPayload executi .thenApply(PayloadStatusV1::asInternalExecutionPayload) .thenPeek( payloadStatus -> - LOG.trace("newPayload(executionPayload={}) -> {}", executionPayload, payloadStatus)) + LOG.trace( + "engineNewPayload(executionPayload={}) -> {}", executionPayload, payloadStatus)) .exceptionally(PayloadStatus::failedExecution); } @@ -204,7 +208,7 @@ public SafeFuture engineNewPayload(final ExecutionPayload executi public SafeFuture engineExchangeTransitionConfiguration( TransitionConfiguration transitionConfiguration) { LOG.trace( - "calling exchangeTransitionConfiguration(transitionConfiguration={})", + "calling engineExchangeTransitionConfiguration(transitionConfiguration={})", transitionConfiguration); return executionEngineClient @@ -215,7 +219,7 @@ public SafeFuture engineExchangeTransitionConfiguration .thenPeek( remoteTransitionConfiguration -> LOG.trace( - "exchangeTransitionConfiguration(transitionConfiguration={}) -> {}", + "engineExchangeTransitionConfiguration(transitionConfiguration={}) -> {}", transitionConfiguration, remoteTransitionConfiguration)); } @@ -226,7 +230,10 @@ public SafeFuture builderGetHeader( checkState(executionBuilderClient.isPresent()); LOG.trace( - "calling builder builderGetHeader(payloadId={}, slot={})", executionPayloadContext, slot); + "calling builderGetHeader(slot={}, pubKey={}, parentHash={})", + slot, + Bytes48.ZERO, + executionPayloadContext.getParentHash()); return executionBuilderClient .get() @@ -238,9 +245,10 @@ public SafeFuture builderGetHeader( .thenPeek( executionPayloadHeader -> LOG.trace( - "builderGetHeader(payloadId={}, slot={}) -> {}", - executionPayloadContext, + "builderGetHeader(slot={}, pubKey={}, parentHash={}) -> {}", slot, + Bytes48.ZERO, + executionPayloadContext.getParentHash(), executionPayloadHeader)); }