Skip to content

Commit

Permalink
use slot as payloadId, better prepare handling (Consensys#4346)
Browse files Browse the repository at this point in the history
  • Loading branch information
tbenr committed Sep 17, 2021
1 parent 1370791 commit be2a5b1
Show file tree
Hide file tree
Showing 7 changed files with 44 additions and 19 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -133,7 +133,7 @@ private static ExecutionPayload createExecutionPayload(Spec spec, BeaconState ge
return spec.atSlot(state.getSlot())
.getExecutionPayloadUtil()
.orElseThrow()
.produceExecutionPayload(executionParentHash, timestamp);
.getExecutionPayload(executionParentHash, timestamp, UInt64.ZERO);
}

public int getProposerIndexForSlot(final BeaconState preState, final UInt64 slot) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,8 @@ public void prepareExecutionPayload(Bytes32 parentHash, UInt64 timestamp, UInt64
executionEngineService.prepareBlock(parentHash, timestamp, payloadId);
}

public ExecutionPayload produceExecutionPayload(Bytes32 parentHash, UInt64 timestamp) {
public ExecutionPayload getExecutionPayload(
Bytes32 parentHash, UInt64 timestamp, UInt64 payloadId) {
checkNotNull(executionEngineService);
return executionEngineService.assembleBlock(parentHash, timestamp).asInternalExecutionPayload();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ public void onBlockProductionDue(final UInt64 slot) {
public void onAttestationCreationDue(final UInt64 slot) {
final UInt64 preparationDueSlot = slot.plus(1);
calculateDuties(spec.computeEpochAtSlot(preparationDueSlot));
notifyEpochDuties(PendingDuties::onProductionDuePreparation, preparationDueSlot);
notifyEpochDuties(PendingDuties::onProductionPreparationDue, preparationDueSlot);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -84,7 +84,7 @@ public void onAggregationDue(final UInt64 slot) {
error -> reportDutyFailure(error, duties.getAggregationType(), slot)));
}

public void onProductionDuePreparation(final UInt64 slot) {
public void onProductionPreparationDue(final UInt64 slot) {
execute(
duties ->
duties
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,11 +16,11 @@
import static com.google.common.base.Preconditions.checkArgument;

import java.util.Optional;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import tech.pegasys.teku.bls.BLSSignature;
import tech.pegasys.teku.infrastructure.async.SafeFuture;
import tech.pegasys.teku.infrastructure.crypto.SecureRandomProvider;
import tech.pegasys.teku.infrastructure.unsigned.UInt64;
import tech.pegasys.teku.spec.Spec;
import tech.pegasys.teku.spec.datastructures.blocks.BeaconBlock;
Expand All @@ -32,12 +32,14 @@

public class BlockProductionDuty implements Duty {
private static final Logger LOG = LogManager.getLogger();
private static final AtomicLong PAYLOAD_ID_COUNTER = new AtomicLong(0);
private final Validator validator;
private final UInt64 slot;
private final ForkProvider forkProvider;
private final ValidatorApiChannel validatorApiChannel;
private final Spec spec;
private UInt64 executionPayloadId;
private final UInt64 executionPayloadId;
private Optional<SafeFuture<Void>> maybePrepareFuture;

public BlockProductionDuty(
final Validator validator,
Expand All @@ -50,19 +52,28 @@ public BlockProductionDuty(
this.forkProvider = forkProvider;
this.validatorApiChannel = validatorApiChannel;
this.spec = spec;
this.executionPayloadId = UInt64.fromLongBits(PAYLOAD_ID_COUNTER.incrementAndGet());
this.maybePrepareFuture = Optional.empty();
}

@Override
public SafeFuture<DutyResult> performDuty() {
LOG.trace("Creating block for validator {} at slot {}", validator.getPublicKey(), slot);
return forkProvider.getForkInfo(slot).thenCompose(this::produceBlock);
// TODO do we need to call prepareDuty if it has not been called before?

// make sure we prepared at least once before producing block
return maybePrepareFuture
.orElseGet(this::prepareDuty)
.thenCompose(__ -> forkProvider.getForkInfo(slot))
.thenCompose(this::produceBlock);
}

@Override
public SafeFuture<Void> prepareDuty() {
executionPayloadId = UInt64.fromLongBits(SecureRandomProvider.publicSecureRandom().nextLong());
return validatorApiChannel.prepareExecutionPayload(slot, executionPayloadId);
LOG.trace("Preparing block for validator {} at slot {}", validator.getPublicKey(), slot);

maybePrepareFuture =
Optional.of(validatorApiChannel.prepareExecutionPayload(slot, executionPayloadId));
return maybePrepareFuture.get();
}

public SafeFuture<DutyResult> produceBlock(final ForkInfo forkInfo) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -85,11 +85,15 @@ public BlockFactory(

public void prepareExecutionPayload(
final Optional<BeaconState> maybeCurrentSlotState, UInt64 payloadId) {
if (maybeCurrentSlotState.isEmpty()) return;
if (maybeCurrentSlotState.isEmpty()) {
return;
}
final Optional<BeaconStateMerge> maybeCurrentMergeState =
maybeCurrentSlotState.get().toVersionMerge();

if (maybeCurrentMergeState.isEmpty()) return;
if (maybeCurrentMergeState.isEmpty()) {
return;
}
final BeaconStateMerge currentMergeState = maybeCurrentMergeState.get();

final ExecutionPayloadUtil executionPayloadUtil =
Expand All @@ -106,7 +110,8 @@ public BeaconBlock createUnsignedBlock(
final Optional<BeaconState> maybeBlockSlotState,
final UInt64 newSlot,
final BLSSignature randaoReveal,
final Optional<Bytes32> optionalGraffiti)
final Optional<Bytes32> optionalGraffiti,
final UInt64 executionPayloadId)
throws EpochProcessingException, SlotProcessingException, StateTransitionException {
checkArgument(
maybeBlockSlotState.isEmpty() || maybeBlockSlotState.get().getSlot().equals(newSlot),
Expand Down Expand Up @@ -166,13 +171,14 @@ public BeaconBlock createUnsignedBlock(
.attesterSlashings(attesterSlashings)
.deposits(deposits)
.voluntaryExits(voluntaryExits)
.executionPayload(() -> getExecutionPayload(blockSlotState))
.executionPayload(() -> getExecutionPayload(blockSlotState, executionPayloadId))
.syncAggregate(
() -> contributionPool.createSyncAggregateForBlock(newSlot, parentRoot)))
.getBlock();
}

private ExecutionPayload getExecutionPayload(BeaconState genericState) {
private ExecutionPayload getExecutionPayload(
BeaconState genericState, UInt64 executionPayloadId) {
final BeaconStateMerge state = BeaconStateMerge.required(genericState);
final ExecutionPayloadUtil executionPayloadUtil =
spec.atSlot(state.getSlot()).getExecutionPayloadUtil().orElseThrow();
Expand Down Expand Up @@ -209,13 +215,15 @@ private ExecutionPayload getExecutionPayload(BeaconState genericState) {
transitionStore.getTransitionTotalDifficulty().toBigInteger()),
Color.YELLOW));
UInt64 timestamp = spec.computeTimeAtSlot(state, state.getSlot());
return executionPayloadUtil.produceExecutionPayload(powHead.blockHash, timestamp);
return executionPayloadUtil.getExecutionPayload(
powHead.blockHash, timestamp, executionPayloadId);
}
}

// Post-merge, normal payload
final Bytes32 executionParentHash = state.getLatest_execution_payload_header().getBlock_hash();
final UInt64 timestamp = spec.computeTimeAtSlot(state, state.getSlot());
return executionPayloadUtil.produceExecutionPayload(executionParentHash, timestamp);
return executionPayloadUtil.getExecutionPayload(
executionParentHash, timestamp, executionPayloadId);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -266,7 +266,7 @@ public SafeFuture<Void> prepareExecutionPayload(UInt64 preparingSlot, UInt64 pay
final SafeFuture<Optional<BeaconState>> currentSlotStateFuture =
combinedChainDataClient.getStateAtSlotExact(preparingSlot.decrement());

return currentSlotStateFuture.thenApplyChecked(
return currentSlotStateFuture.thenApply(
preState -> {
blockFactory.prepareExecutionPayload(preState, payloadId);
return null;
Expand Down Expand Up @@ -311,7 +311,12 @@ private Optional<BeaconBlock> createBlock(
"Delegating to block factory. Has block slot state? {}", maybeBlockSlotState.isPresent());
return Optional.of(
blockFactory.createUnsignedBlock(
maybePreState.get(), maybeBlockSlotState, slot, randaoReveal, graffiti));
maybePreState.get(),
maybeBlockSlotState,
slot,
randaoReveal,
graffiti,
slot)); // we are using slot number as payloadId
}

@Override
Expand Down

0 comments on commit be2a5b1

Please sign in to comment.