Skip to content

Commit

Permalink
Forkchoice v3 (hyperledger#5768)
Browse files Browse the repository at this point in the history
* adds new rpc method for fcu3
* pulls up fork checking for reuse across engine api methods
* more reuse of timestamp/fork checks, getPayload only needs 1 return type
---------

Signed-off-by: Justin Florentine <justin+github@florentine.us>
  • Loading branch information
jflo authored and eum602 committed Nov 3, 2023
1 parent fbb2003 commit dcbc04f
Show file tree
Hide file tree
Showing 12 changed files with 295 additions and 131 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,8 @@
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.response.JsonRpcErrorResponse;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.response.JsonRpcResponse;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.response.RpcErrorType;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
import org.hyperledger.besu.ethereum.mainnet.ValidationResult;

import java.util.Optional;
import java.util.concurrent.CompletableFuture;
Expand All @@ -45,14 +47,29 @@ public enum EngineStatus {
private static final Logger LOG = LoggerFactory.getLogger(ExecutionEngineJsonRpcMethod.class);
protected final Optional<MergeContext> mergeContextOptional;
protected final Supplier<MergeContext> mergeContext;
protected final Optional<ProtocolSchedule> protocolSchedule;
protected final ProtocolContext protocolContext;
protected final EngineCallListener engineCallListener;

protected ExecutionEngineJsonRpcMethod(
final Vertx vertx,
final ProtocolSchedule protocolSchedule,
final ProtocolContext protocolContext,
final EngineCallListener engineCallListener) {
this.syncVertx = vertx;
this.protocolSchedule = Optional.of(protocolSchedule);
this.protocolContext = protocolContext;
this.mergeContextOptional = protocolContext.safeConsensusContext(MergeContext.class);
this.mergeContext = mergeContextOptional::orElseThrow;
this.engineCallListener = engineCallListener;
}

protected ExecutionEngineJsonRpcMethod(
final Vertx vertx,
final ProtocolContext protocolContext,
final EngineCallListener engineCallListener) {
this.syncVertx = vertx;
this.protocolSchedule = Optional.empty();
this.protocolContext = protocolContext;
this.mergeContextOptional = protocolContext.safeConsensusContext(MergeContext.class);
this.mergeContext = mergeContextOptional::orElseThrow;
Expand Down Expand Up @@ -106,4 +123,8 @@ public final JsonRpcResponse response(final JsonRpcRequestContext request) {
}

public abstract JsonRpcResponse syncResponse(final JsonRpcRequestContext request);

protected ValidationResult<RpcErrorType> validateForkSupported(final long blockTimestamp) {
return ValidationResult.valid();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
import org.hyperledger.besu.ethereum.core.Withdrawal;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
import org.hyperledger.besu.ethereum.mainnet.ScheduledProtocolSpec;
import org.hyperledger.besu.ethereum.mainnet.ValidationResult;

import java.util.List;
import java.util.Optional;
Expand All @@ -50,7 +51,6 @@

public abstract class AbstractEngineForkchoiceUpdated extends ExecutionEngineJsonRpcMethod {
private static final Logger LOG = LoggerFactory.getLogger(AbstractEngineForkchoiceUpdated.class);
private final ProtocolSchedule protocolSchedule;
private final MergeMiningCoordinator mergeCoordinator;
private final Long cancunTimestamp;

Expand All @@ -60,14 +60,19 @@ public AbstractEngineForkchoiceUpdated(
final ProtocolContext protocolContext,
final MergeMiningCoordinator mergeCoordinator,
final EngineCallListener engineCallListener) {
super(vertx, protocolContext, engineCallListener);
this.protocolSchedule = protocolSchedule;
super(vertx, protocolSchedule, protocolContext, engineCallListener);

this.mergeCoordinator = mergeCoordinator;
Optional<ScheduledProtocolSpec.Hardfork> cancun =
protocolSchedule.hardforkFor(s -> s.fork().name().equalsIgnoreCase("Cancun"));
cancunTimestamp = cancun.map(ScheduledProtocolSpec.Hardfork::milestone).orElse(Long.MAX_VALUE);
}

protected ValidationResult<RpcErrorType> validateParameter(
final EngineForkchoiceUpdatedParameter forkchoiceUpdatedParameter) {
return ValidationResult.valid();
}

@Override
public JsonRpcResponse syncResponse(final JsonRpcRequestContext requestContext) {
engineCallListener.executionEngineCalled();
Expand All @@ -81,6 +86,20 @@ public JsonRpcResponse syncResponse(final JsonRpcRequestContext requestContext)

LOG.debug("Forkchoice parameters {}", forkChoice);

if (maybePayloadAttributes.isPresent()) {
final EnginePayloadAttributesParameter payloadAttributes = maybePayloadAttributes.get();
ValidationResult<RpcErrorType> forkValidationResult =
validateForkSupported(payloadAttributes.getTimestamp());
if (!forkValidationResult.isValid()) {
return new JsonRpcSuccessResponse(requestId, forkValidationResult);
}
}

ValidationResult<RpcErrorType> parameterValidationResult = validateParameter(forkChoice);
if (!parameterValidationResult.isValid()) {
return new JsonRpcSuccessResponse(requestId, parameterValidationResult);
}

mergeContext
.get()
.fireNewUnverifiedForkchoiceEvent(
Expand Down Expand Up @@ -222,7 +241,7 @@ private boolean isPayloadAttributesValid(
return false;
}
if (!getWithdrawalsValidator(
protocolSchedule, headBlockHeader, payloadAttributes.getTimestamp())
protocolSchedule.get(), headBlockHeader, payloadAttributes.getTimestamp())
.validateWithdrawals(maybeWithdrawals)) {
return false;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,10 +22,13 @@
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.methods.ExecutionEngineJsonRpcMethod;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.response.JsonRpcErrorResponse;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.response.JsonRpcResponse;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.response.JsonRpcSuccessResponse;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.response.RpcErrorType;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.results.BlockResultFactory;
import org.hyperledger.besu.ethereum.core.BlockHeader;
import org.hyperledger.besu.ethereum.core.BlockWithReceipts;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
import org.hyperledger.besu.ethereum.mainnet.ValidationResult;

import java.util.Optional;

Expand All @@ -39,6 +42,18 @@ public abstract class AbstractEngineGetPayload extends ExecutionEngineJsonRpcMet
protected final BlockResultFactory blockResultFactory;
private static final Logger LOG = LoggerFactory.getLogger(AbstractEngineGetPayload.class);

public AbstractEngineGetPayload(
final Vertx vertx,
final ProtocolSchedule schedule,
final ProtocolContext protocolContext,
final MergeMiningCoordinator mergeMiningCoordinator,
final BlockResultFactory blockResultFactory,
final EngineCallListener engineCallListener) {
super(vertx, schedule, protocolContext, engineCallListener);
this.mergeMiningCoordinator = mergeMiningCoordinator;
this.blockResultFactory = blockResultFactory;
}

public AbstractEngineGetPayload(
final Vertx vertx,
final ProtocolContext protocolContext,
Expand All @@ -61,6 +76,11 @@ public JsonRpcResponse syncResponse(final JsonRpcRequestContext request) {
if (blockWithReceipts.isPresent()) {
final var proposal = blockWithReceipts.get();
LOG.atDebug().setMessage("assembledBlock {}").addArgument(() -> proposal).log();
ValidationResult<RpcErrorType> forkValidationResult =
validateForkSupported(proposal.getHeader().getTimestamp());
if (!forkValidationResult.isValid()) {
return new JsonRpcSuccessResponse(request.getRequest().getId(), forkValidationResult);
}
return createResponse(request, payloadId, proposal);
}
return new JsonRpcErrorResponse(request.getRequest().getId(), RpcErrorType.UNKNOWN_PAYLOAD);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,6 @@ public abstract class AbstractEngineNewPayload extends ExecutionEngineJsonRpcMet
private static final Hash OMMERS_HASH_CONSTANT = Hash.EMPTY_LIST_HASH;
private static final Logger LOG = LoggerFactory.getLogger(AbstractEngineNewPayload.class);
private static final BlockHeaderFunctions headerFunctions = new MainnetBlockHeaderFunctions();
private final ProtocolSchedule protocolSchedule;
private final MergeMiningCoordinator mergeCoordinator;
private final EthPeers ethPeers;

Expand All @@ -92,8 +91,7 @@ public AbstractEngineNewPayload(
final MergeMiningCoordinator mergeCoordinator,
final EthPeers ethPeers,
final EngineCallListener engineCallListener) {
super(vertx, protocolContext, engineCallListener);
this.protocolSchedule = protocolSchedule;
super(vertx, protocolSchedule, protocolContext, engineCallListener);
this.mergeCoordinator = mergeCoordinator;
this.ethPeers = ethPeers;
}
Expand All @@ -115,13 +113,18 @@ public JsonRpcResponse syncResponse(final JsonRpcRequestContext requestContext)
final Optional<Bytes32> maybeParentBeaconBlockRoot =
maybeParentBeaconBlockRootParam.map(Bytes32::fromHexString);

ValidationResult<RpcErrorType> forkValidationResult =
validateParamsAndForkSupported(
reqId, blockParam, maybeVersionedHashParam, maybeParentBeaconBlockRoot);
final ValidationResult<RpcErrorType> forkValidationResult =
validateForkSupported(blockParam.getTimestamp());
if (!forkValidationResult.isValid()) {
return new JsonRpcErrorResponse(reqId, forkValidationResult);
}

final ValidationResult<RpcErrorType> parameterValidationResult =
validateParameters(blockParam, maybeVersionedHashParam, maybeParentBeaconBlockRootParam);
if (!parameterValidationResult.isValid()) {
return new JsonRpcErrorResponse(reqId, parameterValidationResult);
}

final Optional<List<VersionedHash>> maybeVersionedHashes;
try {
maybeVersionedHashes = extractVersionedHashes(maybeVersionedHashParam);
Expand All @@ -142,7 +145,7 @@ public JsonRpcResponse syncResponse(final JsonRpcRequestContext requestContext)
.map(ws -> ws.stream().map(WithdrawalParameter::toWithdrawal).collect(toList()));

if (!getWithdrawalsValidator(
protocolSchedule, blockParam.getTimestamp(), blockParam.getBlockNumber())
protocolSchedule.get(), blockParam.getTimestamp(), blockParam.getBlockNumber())
.validateWithdrawals(maybeWithdrawals)) {
return new JsonRpcErrorResponse(
reqId, new JsonRpcError(INVALID_PARAMS, "Invalid withdrawals"));
Expand All @@ -152,7 +155,7 @@ public JsonRpcResponse syncResponse(final JsonRpcRequestContext requestContext)
Optional.ofNullable(blockParam.getDeposits())
.map(ds -> ds.stream().map(DepositParameter::toDeposit).collect(toList()));
if (!getDepositsValidator(
protocolSchedule, blockParam.getTimestamp(), blockParam.getBlockNumber())
protocolSchedule.get(), blockParam.getTimestamp(), blockParam.getBlockNumber())
.validateDepositParameter(maybeDeposits)) {
return new JsonRpcErrorResponse(reqId, new JsonRpcError(INVALID_PARAMS, "Invalid deposits"));
}
Expand Down Expand Up @@ -236,7 +239,7 @@ public JsonRpcResponse syncResponse(final JsonRpcRequestContext requestContext)
newBlockHeader,
maybeParentHeader,
maybeVersionedHashes,
protocolSchedule.getByBlockHeader(newBlockHeader));
protocolSchedule.get().getByBlockHeader(newBlockHeader));
if (!blobValidationResult.isValid()) {
return respondWithInvalid(
reqId,
Expand Down Expand Up @@ -398,11 +401,10 @@ protected EngineStatus getInvalidBlockHashStatus() {
return INVALID;
}

protected ValidationResult<RpcErrorType> validateParamsAndForkSupported(
final Object id,
final EnginePayloadParameter payloadParameter,
protected ValidationResult<RpcErrorType> validateParameters(
final EnginePayloadParameter parameter,
final Optional<List<String>> maybeVersionedHashParam,
final Optional<Bytes32> parentBeaconBlockRoot) {
final Optional<String> maybeBeaconBlockRootParam) {
return ValidationResult.valid();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,23 +17,61 @@
import org.hyperledger.besu.consensus.merge.blockcreation.MergeMiningCoordinator;
import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.api.jsonrpc.RpcMethod;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.parameters.EngineForkchoiceUpdatedParameter;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.response.RpcErrorType;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
import org.hyperledger.besu.ethereum.mainnet.ScheduledProtocolSpec;
import org.hyperledger.besu.ethereum.mainnet.ValidationResult;

import java.util.Optional;

import io.vertx.core.Vertx;

public class EngineForkchoiceUpdatedV3 extends AbstractEngineForkchoiceUpdated {

private final Optional<ScheduledProtocolSpec.Hardfork> cancun;

public EngineForkchoiceUpdatedV3(
final Vertx vertx,
final ProtocolSchedule protocolSchedule,
final ProtocolContext protocolContext,
final MergeMiningCoordinator mergeCoordinator,
final EngineCallListener engineCallListener) {
super(vertx, protocolSchedule, protocolContext, mergeCoordinator, engineCallListener);
this.cancun = protocolSchedule.hardforkFor(s -> s.fork().name().equalsIgnoreCase("Cancun"));
}

@Override
public String getName() {
return RpcMethod.ENGINE_FORKCHOICE_UPDATED_V3.getMethodName();
}

@Override
protected ValidationResult<RpcErrorType> validateParameter(
final EngineForkchoiceUpdatedParameter fcuParameter) {
if (fcuParameter.getHeadBlockHash() == null) {
return ValidationResult.invalid(RpcErrorType.INVALID_PARAMS, "Missing head block hash");
} else if (fcuParameter.getSafeBlockHash() == null) {
return ValidationResult.invalid(RpcErrorType.INVALID_PARAMS, "Missing safe block hash");
} else if (fcuParameter.getFinalizedBlockHash() == null) {
return ValidationResult.invalid(RpcErrorType.INVALID_PARAMS, "Missing finalized block hash");
}
return ValidationResult.valid();
}

@Override
protected ValidationResult<RpcErrorType> validateForkSupported(final long blockTimestamp) {
if (protocolSchedule.isPresent()) {
if (cancun.isPresent() && blockTimestamp >= cancun.get().milestone()) {
return ValidationResult.valid();
} else {
return ValidationResult.invalid(
RpcErrorType.UNSUPPORTED_FORK,
"Cancun configured to start at timestamp: " + cancun.get().milestone());
}
} else {
return ValidationResult.invalid(
RpcErrorType.UNSUPPORTED_FORK, "Configuration error, no schedule for Cancun fork set");
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,25 +19,21 @@
import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.api.jsonrpc.RpcMethod;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.JsonRpcRequestContext;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.response.JsonRpcErrorResponse;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.response.JsonRpcResponse;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.response.JsonRpcSuccessResponse;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.response.RpcErrorType;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.results.BlockResultFactory;
import org.hyperledger.besu.ethereum.core.BlockWithReceipts;
import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule;
import org.hyperledger.besu.ethereum.mainnet.ScheduledProtocolSpec;
import org.hyperledger.besu.ethereum.mainnet.ValidationResult;

import java.util.Optional;

import io.vertx.core.Vertx;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

public class EngineGetPayloadV3 extends AbstractEngineGetPayload {

private static final Logger LOG = LoggerFactory.getLogger(EngineGetPayloadV3.class);
private final Optional<ScheduledProtocolSpec.Hardfork> shanghai;
private final Optional<ScheduledProtocolSpec.Hardfork> cancun;

public EngineGetPayloadV3(
Expand All @@ -47,8 +43,13 @@ public EngineGetPayloadV3(
final BlockResultFactory blockResultFactory,
final EngineCallListener engineCallListener,
final ProtocolSchedule schedule) {
super(vertx, protocolContext, mergeMiningCoordinator, blockResultFactory, engineCallListener);
this.shanghai = schedule.hardforkFor(s -> s.fork().name().equalsIgnoreCase("Shanghai"));
super(
vertx,
schedule,
protocolContext,
mergeMiningCoordinator,
blockResultFactory,
engineCallListener);
this.cancun = schedule.hardforkFor(s -> s.fork().name().equalsIgnoreCase("Cancun"));
}

Expand All @@ -63,29 +64,24 @@ protected JsonRpcResponse createResponse(
final PayloadIdentifier payloadId,
final BlockWithReceipts blockWithReceipts) {

try {
long builtAt = blockWithReceipts.getHeader().getTimestamp();
return new JsonRpcSuccessResponse(
request.getRequest().getId(),
blockResultFactory.payloadTransactionCompleteV3(blockWithReceipts));
}

if (this.shanghai.isPresent() && builtAt < this.shanghai.get().milestone()) {
return new JsonRpcSuccessResponse(
request.getRequest().getId(),
blockResultFactory.payloadTransactionCompleteV1(blockWithReceipts.getBlock()));
} else if (this.shanghai.isPresent()
&& builtAt >= this.shanghai.get().milestone()
&& this.cancun.isPresent()
&& builtAt < this.cancun.get().milestone()) {
return new JsonRpcSuccessResponse(
request.getRequest().getId(),
blockResultFactory.payloadTransactionCompleteV2(blockWithReceipts));
@Override
protected ValidationResult<RpcErrorType> validateForkSupported(final long blockTimestamp) {
if (protocolSchedule.isPresent()) {
if (cancun.isPresent() && blockTimestamp >= cancun.get().milestone()) {
return ValidationResult.valid();
} else {
return new JsonRpcSuccessResponse(
request.getRequest().getId(),
blockResultFactory.payloadTransactionCompleteV3(blockWithReceipts));
return ValidationResult.invalid(
RpcErrorType.UNSUPPORTED_FORK,
"Cancun configured to start at timestamp: " + cancun.get().milestone());
}

} catch (ClassCastException e) {
LOG.error("configuration error, can't call V3 endpoint with non-default protocol schedule");
return new JsonRpcErrorResponse(request.getRequest().getId(), RpcErrorType.INTERNAL_ERROR);
} else {
return ValidationResult.invalid(
RpcErrorType.UNSUPPORTED_FORK, "Configuration error, no schedule for Cancun fork set");
}
}
}
Loading

0 comments on commit dcbc04f

Please sign in to comment.