Skip to content

Commit

Permalink
Block Rewards Cache Part 2 (Consensys#7987)
Browse files Browse the repository at this point in the history
  • Loading branch information
tbenr authored Feb 19, 2024
1 parent 9e14631 commit ea16cd2
Show file tree
Hide file tree
Showing 52 changed files with 2,996 additions and 744 deletions.
1 change: 1 addition & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -12,5 +12,6 @@ the [releases page](https://github.com/Consensys/teku/releases).
### Breaking Changes

### Additions and Improvements
- Improve block rewards calculation performance for `/eth/v3/validator/blocks/{slot}` block production beacon node API.

### Bug Fixes
Original file line number Diff line number Diff line change
Expand Up @@ -21,14 +21,14 @@
import tech.pegasys.teku.infrastructure.async.SafeFuture;
import tech.pegasys.teku.infrastructure.unsigned.UInt64;
import tech.pegasys.teku.spec.datastructures.blobs.versions.deneb.BlobSidecar;
import tech.pegasys.teku.spec.datastructures.blocks.BlockContainer;
import tech.pegasys.teku.spec.datastructures.blocks.SignedBeaconBlock;
import tech.pegasys.teku.spec.datastructures.blocks.SignedBlockContainer;
import tech.pegasys.teku.spec.datastructures.metadata.BlockContainerAndMetaData;
import tech.pegasys.teku.spec.datastructures.state.beaconstate.BeaconState;

public interface BlockFactory {

SafeFuture<BlockContainer> createUnsignedBlock(
SafeFuture<BlockContainerAndMetaData> createUnsignedBlock(
BeaconState blockSlotState,
UInt64 proposalSlot,
BLSSignature randaoReveal,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,10 +24,10 @@
import tech.pegasys.teku.spec.SpecMilestone;
import tech.pegasys.teku.spec.datastructures.blobs.versions.deneb.BlobSidecar;
import tech.pegasys.teku.spec.datastructures.blocks.BeaconBlock;
import tech.pegasys.teku.spec.datastructures.blocks.BlockContainer;
import tech.pegasys.teku.spec.datastructures.blocks.SignedBlockContainer;
import tech.pegasys.teku.spec.datastructures.blocks.versions.deneb.BlockContents;
import tech.pegasys.teku.spec.datastructures.execution.BlobsBundle;
import tech.pegasys.teku.spec.datastructures.metadata.BlockContainerAndMetaData;
import tech.pegasys.teku.spec.datastructures.state.beaconstate.BeaconState;
import tech.pegasys.teku.spec.schemas.SchemaDefinitionsDeneb;

Expand All @@ -43,7 +43,7 @@ public BlockFactoryDeneb(final Spec spec, final BlockOperationSelectorFactory op
}

@Override
public SafeFuture<BlockContainer> createUnsignedBlock(
public SafeFuture<BlockContainerAndMetaData> createUnsignedBlock(
final BeaconState blockSlotState,
final UInt64 proposalSlot,
final BLSSignature randaoReveal,
Expand All @@ -59,17 +59,18 @@ public SafeFuture<BlockContainer> createUnsignedBlock(
requestedBlinded,
requestedBuilderBoostFactor,
blockProductionPerformance)
.thenApply(BlockContainer::getBlock)
.thenCompose(
block -> {
blockContainerAndMetaData -> {
final BeaconBlock block = blockContainerAndMetaData.blockContainer().getBlock();
if (block.isBlinded()) {
return SafeFuture.completedFuture(block);
return SafeFuture.completedFuture(blockContainerAndMetaData);
}
// The execution BlobsBundle has been cached as part of the block creation
return operationSelector
.createBlobsBundleSelector()
.apply(block)
.thenApply(blobsBundle -> createBlockContents(block, blobsBundle));
.thenApply(blobsBundle -> createBlockContents(block, blobsBundle))
.thenApply(blockContainerAndMetaData::withBlockContents);
});
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@
package tech.pegasys.teku.validator.coordinator;

import static com.google.common.base.Preconditions.checkArgument;
import static tech.pegasys.teku.spec.constants.EthConstants.GWEI_TO_WEI;

import java.util.Collections;
import java.util.List;
Expand All @@ -26,10 +27,12 @@
import tech.pegasys.teku.spec.Spec;
import tech.pegasys.teku.spec.datastructures.blobs.versions.deneb.BlobSidecar;
import tech.pegasys.teku.spec.datastructures.blocks.BeaconBlockAndState;
import tech.pegasys.teku.spec.datastructures.blocks.BlockContainer;
import tech.pegasys.teku.spec.datastructures.blocks.SignedBeaconBlock;
import tech.pegasys.teku.spec.datastructures.blocks.SignedBlockContainer;
import tech.pegasys.teku.spec.datastructures.metadata.BlockContainerAndMetaData;
import tech.pegasys.teku.spec.datastructures.state.beaconstate.BeaconState;
import tech.pegasys.teku.spec.datastructures.state.beaconstate.BeaconStateCache;
import tech.pegasys.teku.spec.datastructures.state.beaconstate.common.SlotCaches;

public class BlockFactoryPhase0 implements BlockFactory {

Expand All @@ -43,7 +46,7 @@ public BlockFactoryPhase0(
}

@Override
public SafeFuture<BlockContainer> createUnsignedBlock(
public SafeFuture<BlockContainerAndMetaData> createUnsignedBlock(
final BeaconState blockSlotState,
final UInt64 proposalSlot,
final BLSSignature randaoReveal,
Expand Down Expand Up @@ -76,7 +79,17 @@ public SafeFuture<BlockContainer> createUnsignedBlock(
requestedBuilderBoostFactor,
blockProductionPerformance),
blockProductionPerformance)
.thenApply(BeaconBlockAndState::getBlock);
.thenApply(this::beaconBlockAndStateToBlockContainerAndMetaData);
}

private BlockContainerAndMetaData beaconBlockAndStateToBlockContainerAndMetaData(
final BeaconBlockAndState blockAndState) {
final SlotCaches slotCaches = BeaconStateCache.getSlotCaches(blockAndState.getState());
return new BlockContainerAndMetaData(
blockAndState.getBlock(),
spec.atSlot(blockAndState.getSlot()).getMilestone(),
slotCaches.getBlockExecutionValue(),
GWEI_TO_WEI.multiply(slotCaches.getBlockProposerRewards().longValue()));
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@
import tech.pegasys.teku.spec.datastructures.operations.SignedBlsToExecutionChange;
import tech.pegasys.teku.spec.datastructures.operations.SignedVoluntaryExit;
import tech.pegasys.teku.spec.datastructures.state.beaconstate.BeaconState;
import tech.pegasys.teku.spec.datastructures.state.beaconstate.BeaconStateCache;
import tech.pegasys.teku.spec.datastructures.type.SszKZGCommitment;
import tech.pegasys.teku.spec.datastructures.type.SszKZGProof;
import tech.pegasys.teku.spec.executionlayer.ExecutionLayerBlockProductionManager;
Expand Down Expand Up @@ -255,6 +256,7 @@ private SafeFuture<Void> setExecutionData(
blockProductionPerformance);

return SafeFuture.allOf(
cacheExecutionPayloadValue(executionPayloadResult, blockSlotState),
builderSetPayloadPostMerge(
bodyBuilder, setUnblindedContentIfBuilderFallbacks, executionPayloadResult),
builderSetKzgCommitments(
Expand All @@ -264,6 +266,19 @@ private SafeFuture<Void> setExecutionData(
executionPayloadResult));
}

private SafeFuture<Void> cacheExecutionPayloadValue(
final ExecutionPayloadResult executionPayloadResult, final BeaconState blockSlotState) {
return executionPayloadResult
.getExecutionPayloadValueFuture()
.map(
futureValue ->
futureValue.thenAccept(
value ->
BeaconStateCache.getSlotCaches(blockSlotState)
.setBlockExecutionValue(value)))
.orElse(SafeFuture.COMPLETE);
}

private SafeFuture<Void> builderSetPayloadPostMerge(
final BeaconBlockBodyBuilder bodyBuilder,
final Function<HeaderWithFallbackData, Boolean> setUnblindedContentIfBuilderFallbacks,
Expand Down Expand Up @@ -308,16 +323,21 @@ private SafeFuture<Void> builderSetPayload(
.getDefault());
return SafeFuture.COMPLETE;
}
return executionLayerBlockProductionManager
.initiateBlockProduction(

final ExecutionPayloadResult executionPayloadResult =
executionLayerBlockProductionManager.initiateBlockProduction(
executionPayloadContext.get(),
blockSlotState,
false,
Optional.empty(),
blockProductionPerformance)
.getExecutionPayloadFuture()
.orElseThrow()
.thenAccept(bodyBuilder::executionPayload);
blockProductionPerformance);

return SafeFuture.allOf(
cacheExecutionPayloadValue(executionPayloadResult, blockSlotState),
executionPayloadResult
.getExecutionPayloadFuture()
.orElseThrow()
.thenAccept(bodyBuilder::executionPayload));
}

private SafeFuture<Void> builderSetPayloadHeader(
Expand All @@ -337,25 +357,32 @@ private SafeFuture<Void> builderSetPayloadHeader(
return SafeFuture.COMPLETE;
}

return executionLayerBlockProductionManager
.initiateBlockProduction(
final ExecutionPayloadResult executionPayloadResult =
executionLayerBlockProductionManager.initiateBlockProduction(
executionPayloadContext.get(),
blockSlotState,
true,
requestedBuilderBoostFactor,
blockProductionPerformance)
.getHeaderWithFallbackDataFuture()
.orElseThrow()
.thenAccept(
headerWithFallbackData -> {
if (setUnblindedContentIfBuilderFallbacks.apply(headerWithFallbackData)) {
bodyBuilder.executionPayload(
headerWithFallbackData.getFallbackData().orElseThrow().getExecutionPayload());
return;
}
bodyBuilder.executionPayloadHeader(
headerWithFallbackData.getExecutionPayloadHeader());
});
blockProductionPerformance);

return SafeFuture.allOf(
cacheExecutionPayloadValue(executionPayloadResult, blockSlotState),
executionPayloadResult
.getHeaderWithFallbackDataFuture()
.orElseThrow()
.thenAccept(
headerWithFallbackData -> {
if (setUnblindedContentIfBuilderFallbacks.apply(headerWithFallbackData)) {
bodyBuilder.executionPayload(
headerWithFallbackData
.getFallbackData()
.orElseThrow()
.getExecutionPayload());
return;
}
bodyBuilder.executionPayloadHeader(
headerWithFallbackData.getExecutionPayloadHeader());
}));
}

private SafeFuture<Void> builderSetKzgCommitments(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,9 +27,9 @@
import tech.pegasys.teku.spec.Spec;
import tech.pegasys.teku.spec.SpecMilestone;
import tech.pegasys.teku.spec.datastructures.blobs.versions.deneb.BlobSidecar;
import tech.pegasys.teku.spec.datastructures.blocks.BlockContainer;
import tech.pegasys.teku.spec.datastructures.blocks.SignedBeaconBlock;
import tech.pegasys.teku.spec.datastructures.blocks.SignedBlockContainer;
import tech.pegasys.teku.spec.datastructures.metadata.BlockContainerAndMetaData;
import tech.pegasys.teku.spec.datastructures.state.beaconstate.BeaconState;

public class MilestoneBasedBlockFactory implements BlockFactory {
Expand Down Expand Up @@ -61,7 +61,7 @@ public MilestoneBasedBlockFactory(
}

@Override
public SafeFuture<BlockContainer> createUnsignedBlock(
public SafeFuture<BlockContainerAndMetaData> createUnsignedBlock(
final BeaconState blockSlotState,
final UInt64 proposalSlot,
final BLSSignature randaoReveal,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,11 +62,11 @@
import tech.pegasys.teku.spec.SpecVersion;
import tech.pegasys.teku.spec.datastructures.attestation.ValidatableAttestation;
import tech.pegasys.teku.spec.datastructures.blocks.BeaconBlock;
import tech.pegasys.teku.spec.datastructures.blocks.BlockContainer;
import tech.pegasys.teku.spec.datastructures.blocks.SignedBlockAndState;
import tech.pegasys.teku.spec.datastructures.blocks.SignedBlockContainer;
import tech.pegasys.teku.spec.datastructures.builder.SignedValidatorRegistration;
import tech.pegasys.teku.spec.datastructures.genesis.GenesisData;
import tech.pegasys.teku.spec.datastructures.metadata.BlockContainerAndMetaData;
import tech.pegasys.teku.spec.datastructures.operations.Attestation;
import tech.pegasys.teku.spec.datastructures.operations.AttestationData;
import tech.pegasys.teku.spec.datastructures.operations.SignedAggregateAndProof;
Expand Down Expand Up @@ -309,7 +309,7 @@ public SafeFuture<Optional<Map<BLSPublicKey, ValidatorStatus>>> getValidatorStat
}

@Override
public SafeFuture<Optional<BlockContainer>> createUnsignedBlock(
public SafeFuture<Optional<BlockContainerAndMetaData>> createUnsignedBlock(
final UInt64 slot,
final BLSSignature randaoReveal,
final Optional<Bytes32> graffiti,
Expand Down Expand Up @@ -349,7 +349,7 @@ public SafeFuture<Optional<BlockContainer>> createUnsignedBlock(
.alwaysRun(blockProductionPerformance::complete);
}

private SafeFuture<Optional<BlockContainer>> createBlock(
private SafeFuture<Optional<BlockContainerAndMetaData>> createBlock(
final UInt64 slot,
final BLSSignature randaoReveal,
final Optional<Bytes32> graffiti,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,13 +22,15 @@
import static org.mockito.Mockito.verifyNoInteractions;
import static org.mockito.Mockito.when;
import static tech.pegasys.teku.infrastructure.async.SafeFutureAssert.safeJoin;
import static tech.pegasys.teku.spec.constants.EthConstants.GWEI_TO_WEI;

import java.util.Collections;
import java.util.List;
import java.util.Optional;
import java.util.function.Consumer;
import java.util.stream.Collectors;
import org.apache.tuweni.bytes.Bytes32;
import org.apache.tuweni.units.bigints.UInt256;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.BeforeAll;
import tech.pegasys.teku.bls.BLSSignature;
Expand All @@ -41,7 +43,6 @@
import tech.pegasys.teku.spec.SpecMilestone;
import tech.pegasys.teku.spec.datastructures.blobs.versions.deneb.BlobSidecar;
import tech.pegasys.teku.spec.datastructures.blocks.BeaconBlock;
import tech.pegasys.teku.spec.datastructures.blocks.BlockContainer;
import tech.pegasys.teku.spec.datastructures.blocks.Eth1Data;
import tech.pegasys.teku.spec.datastructures.blocks.SignedBeaconBlock;
import tech.pegasys.teku.spec.datastructures.blocks.SignedBlockContainer;
Expand All @@ -57,13 +58,16 @@
import tech.pegasys.teku.spec.datastructures.execution.ExecutionPayloadHeader;
import tech.pegasys.teku.spec.datastructures.execution.ExecutionPayloadResult;
import tech.pegasys.teku.spec.datastructures.execution.HeaderWithFallbackData;
import tech.pegasys.teku.spec.datastructures.metadata.BlockContainerAndMetaData;
import tech.pegasys.teku.spec.datastructures.operations.Attestation;
import tech.pegasys.teku.spec.datastructures.operations.AttesterSlashing;
import tech.pegasys.teku.spec.datastructures.operations.Deposit;
import tech.pegasys.teku.spec.datastructures.operations.ProposerSlashing;
import tech.pegasys.teku.spec.datastructures.operations.SignedBlsToExecutionChange;
import tech.pegasys.teku.spec.datastructures.operations.SignedVoluntaryExit;
import tech.pegasys.teku.spec.datastructures.state.beaconstate.BeaconState;
import tech.pegasys.teku.spec.datastructures.state.beaconstate.BeaconStateCache;
import tech.pegasys.teku.spec.datastructures.state.beaconstate.common.SlotCaches;
import tech.pegasys.teku.spec.datastructures.state.beaconstate.versions.bellatrix.BeaconStateBellatrix;
import tech.pegasys.teku.spec.datastructures.type.SszKZGCommitment;
import tech.pegasys.teku.spec.datastructures.util.BeaconBlockBodyLists;
Expand Down Expand Up @@ -138,7 +142,7 @@ protected ExecutionPayloadHeader getExecutionPayloadHeader(final BeaconBlock blo
return BlindedBeaconBlockBodyBellatrix.required(block.getBody()).getExecutionPayloadHeader();
}

protected BlockContainer assertBlockCreated(
protected BlockContainerAndMetaData assertBlockCreated(
final int blockSlot,
final Spec spec,
final boolean postMerge,
Expand Down Expand Up @@ -202,7 +206,23 @@ protected BlockContainer assertBlockCreated(
.thenAnswer(invocation -> createEmptySyncAggregate(spec));
executionPayloadBuilder.accept(blockSlotState);

final BlockContainer blockContainer =
final UInt256 blockExecutionValue;
final UInt64 blockProposerRewards;

if (milestone.isGreaterThanOrEqualTo(SpecMilestone.BELLATRIX)) {
blockExecutionValue = dataStructureUtil.randomUInt256();
blockProposerRewards = dataStructureUtil.randomUInt64();

// inject values into slot caches
final SlotCaches slotCaches = BeaconStateCache.getSlotCaches(blockSlotState);
slotCaches.setBlockExecutionValue(blockExecutionValue);
slotCaches.increaseBlockProposerRewards(blockProposerRewards);
} else {
blockExecutionValue = UInt256.ZERO;
blockProposerRewards = UInt64.ZERO;
}

final BlockContainerAndMetaData blockContainerAndMetaData =
safeJoin(
blockFactory.createUnsignedBlock(
blockSlotState,
Expand All @@ -213,7 +233,7 @@ protected BlockContainer assertBlockCreated(
Optional.empty(),
BlockProductionPerformance.NOOP));

final BeaconBlock block = blockContainer.getBlock();
final BeaconBlock block = blockContainerAndMetaData.blockContainer().getBlock();

assertThat(block).isNotNull();
assertThat(block.getSlot()).isEqualTo(newSlot);
Expand Down Expand Up @@ -245,7 +265,12 @@ protected BlockContainer assertBlockCreated(
assertThat(block.getBody().getOptionalBlobKzgCommitments()).isEmpty();
}

return blockContainer;
assertThat(blockContainerAndMetaData.consensusBlockValue())
.isEqualByComparingTo(GWEI_TO_WEI.multiply(blockProposerRewards.longValue()));
assertThat(blockContainerAndMetaData.executionPayloadValue())
.isEqualByComparingTo(blockExecutionValue);

return blockContainerAndMetaData;
}

protected SyncAggregate createEmptySyncAggregate(final Spec spec) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,8 @@ void shouldCreateBlockContents() {
final BlobsBundle blobsBundle = prepareBlobsBundle(spec, 3);

final BlockContainer blockContainer =
assertBlockCreated(1, spec, false, state -> prepareValidPayload(spec, state), false);
assertBlockCreated(1, spec, false, state -> prepareValidPayload(spec, state), false)
.blockContainer();

assertThat(blockContainer).isInstanceOf(BlockContents.class);
assertThat(blockContainer.getBlock().getBody().getOptionalBlobKzgCommitments())
Expand All @@ -66,7 +67,8 @@ void shouldCreateBlindedBeaconBlockWhenBlindedBlockRequested() {
final SszList<SszKZGCommitment> blobKzgCommitments = prepareBuilderBlobKzgCommitments(spec, 3);

final BlockContainer blockContainer =
assertBlockCreated(1, spec, false, state -> prepareValidPayload(spec, state), true);
assertBlockCreated(1, spec, false, state -> prepareValidPayload(spec, state), true)
.blockContainer();

assertThat(blockContainer).isInstanceOf(BeaconBlock.class);
final BeaconBlock blindedBeaconBlock = (BeaconBlock) blockContainer;
Expand Down
Loading

0 comments on commit ea16cd2

Please sign in to comment.