Less strict engine QoS timer (#4411)

* reset engine QoS timer with every call to the engine API, ExchangeTransitionConfiguration mismatch will only submit a debug log not a warning anymore

Signed-off-by: Daniel Lehrner <daniel.lehrner@consensys.net>
pull/4416/head
Daniel Lehrner 2 years ago committed by GitHub
parent 5e156253ab
commit bc9ff867c4
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 2
      CHANGELOG.md
  2. 8
      ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/ExecutionEngineJsonRpcMethod.java
  3. 19
      ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineCallListener.java
  4. 31
      ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineExchangeTransitionConfiguration.java
  5. 7
      ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineForkchoiceUpdated.java
  6. 7
      ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineGetPayload.java
  7. 7
      ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineNewPayload.java
  8. 50
      ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineQosTimer.java
  9. 20
      ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/methods/ExecutionEngineJsonRpcMethods.java
  10. 126
      ethereum/api/src/test/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineExchangeTransitionConfigurationTest.java
  11. 17
      ethereum/api/src/test/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineForkchoiceUpdatedTest.java
  12. 8
      ethereum/api/src/test/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineGetPayloadTest.java
  13. 22
      ethereum/api/src/test/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineNewPayloadTest.java
  14. 101
      ethereum/api/src/test/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/methods/engine/EngineQosTimerTest.java

@ -5,6 +5,8 @@
### Additions and Improvements
- Allow free gas networks in the London fee market [#4061](https://github.com/hyperledger/besu/issues/4061)
- Upgrade besu-native to 0.6.0 and use Blake2bf native implementation if available by default [#4264](https://github.com/hyperledger/besu/pull/4264)
- Resets engine QoS timer with every call to the engine API instead of only when ExchangeTransitionConfiguration is called [#4411](https://github.com/hyperledger/besu/issues/4411)
- ExchangeTransitionConfiguration mismatch will only submit a debug log not a warning anymore [#4411](https://github.com/hyperledger/besu/issues/4411)
### Bug Fixes
- Retry block creation if there is a transient error and we still have time, to mitigate empty block issue [#4407](https://github.com/hyperledger/besu/pull/4407)

@ -17,6 +17,7 @@ package org.hyperledger.besu.ethereum.api.jsonrpc.internal.methods;
import org.hyperledger.besu.consensus.merge.MergeContext;
import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.JsonRpcRequestContext;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.methods.engine.EngineCallListener;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.response.JsonRpcError;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.response.JsonRpcErrorResponse;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.response.JsonRpcResponse;
@ -45,12 +46,17 @@ public abstract class ExecutionEngineJsonRpcMethod implements JsonRpcMethod {
protected final Optional<MergeContext> mergeContextOptional;
protected final Supplier<MergeContext> mergeContext;
protected final ProtocolContext protocolContext;
protected final EngineCallListener engineCallListener;
protected ExecutionEngineJsonRpcMethod(final Vertx vertx, final ProtocolContext protocolContext) {
protected ExecutionEngineJsonRpcMethod(
final Vertx vertx,
final ProtocolContext protocolContext,
final EngineCallListener engineCallListener) {
this.syncVertx = vertx;
this.protocolContext = protocolContext;
this.mergeContextOptional = protocolContext.safeConsensusContext(MergeContext.class);
this.mergeContext = mergeContextOptional::orElseThrow;
this.engineCallListener = engineCallListener;
}
@Override

@ -0,0 +1,19 @@
/*
* Copyright Hyperledger Besu Contributors.
*
* 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.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.ethereum.api.jsonrpc.internal.methods.engine;
public interface EngineCallListener {
void executionEngineCalled();
}

@ -21,7 +21,6 @@ import org.hyperledger.besu.consensus.merge.MergeContext;
import org.hyperledger.besu.datatypes.Hash;
import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.JsonRpcRequestContext;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.QosTimer;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.methods.ExecutionEngineJsonRpcMethod;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.parameters.EngineExchangeTransitionConfigurationParameter;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.response.JsonRpcResponse;
@ -46,21 +45,11 @@ public class EngineExchangeTransitionConfiguration extends ExecutionEngineJsonRp
Difficulty.fromHexString(
"0xfffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffc00");
static final long QOS_TIMEOUT_MILLIS = 120000L;
private final QosTimer qosTimer;
public EngineExchangeTransitionConfiguration(
final Vertx vertx, final ProtocolContext protocolContext) {
super(vertx, protocolContext);
qosTimer =
new QosTimer(
vertx,
QOS_TIMEOUT_MILLIS,
lastCall ->
LOG.warn(
"not called in {} seconds, consensus client may not be connected",
QOS_TIMEOUT_MILLIS / 1000L));
final Vertx vertx,
final ProtocolContext protocolContext,
final EngineCallListener engineCallListener) {
super(vertx, protocolContext, engineCallListener);
}
@Override
@ -70,8 +59,7 @@ public class EngineExchangeTransitionConfiguration extends ExecutionEngineJsonRp
@Override
public JsonRpcResponse syncResponse(final JsonRpcRequestContext requestContext) {
// update our QoS "last call time"
getQosTimer().resetTimer();
engineCallListener.executionEngineCalled();
final EngineExchangeTransitionConfigurationParameter remoteTransitionConfiguration =
requestContext.getRequiredParameter(
@ -97,7 +85,7 @@ public class EngineExchangeTransitionConfiguration extends ExecutionEngineJsonRp
if (!localTransitionConfiguration
.getTerminalTotalDifficulty()
.equals(remoteTransitionConfiguration.getTerminalTotalDifficulty())) {
LOG.warn(
LOG.debug(
"Configured terminal total difficulty {} does not match value of consensus client {}",
localTransitionConfiguration.getTerminalTotalDifficulty(),
remoteTransitionConfiguration.getTerminalTotalDifficulty());
@ -106,7 +94,7 @@ public class EngineExchangeTransitionConfiguration extends ExecutionEngineJsonRp
if (!localTransitionConfiguration
.getTerminalBlockHash()
.equals(remoteTransitionConfiguration.getTerminalBlockHash())) {
LOG.warn(
LOG.debug(
"Configured terminal block hash {} does not match value of consensus client {}",
localTransitionConfiguration.getTerminalBlockHash(),
remoteTransitionConfiguration.getTerminalBlockHash());
@ -128,9 +116,4 @@ public class EngineExchangeTransitionConfiguration extends ExecutionEngineJsonRp
final EngineExchangeTransitionConfigurationResult transitionConfiguration) {
return new JsonRpcSuccessResponse(requestId, transitionConfiguration);
}
// QosTimer accessor for testing considerations
QosTimer getQosTimer() {
return qosTimer;
}
}

@ -50,8 +50,9 @@ public class EngineForkchoiceUpdated extends ExecutionEngineJsonRpcMethod {
public EngineForkchoiceUpdated(
final Vertx vertx,
final ProtocolContext protocolContext,
final MergeMiningCoordinator mergeCoordinator) {
super(vertx, protocolContext);
final MergeMiningCoordinator mergeCoordinator,
final EngineCallListener engineCallListener) {
super(vertx, protocolContext, engineCallListener);
this.mergeCoordinator = mergeCoordinator;
}
@ -62,6 +63,8 @@ public class EngineForkchoiceUpdated extends ExecutionEngineJsonRpcMethod {
@Override
public JsonRpcResponse syncResponse(final JsonRpcRequestContext requestContext) {
engineCallListener.executionEngineCalled();
final Object requestId = requestContext.getRequest().getId();
final EngineForkchoiceUpdatedParameter forkChoice =

@ -43,8 +43,9 @@ public class EngineGetPayload extends ExecutionEngineJsonRpcMethod {
public EngineGetPayload(
final Vertx vertx,
final ProtocolContext protocolContext,
final BlockResultFactory blockResultFactory) {
super(vertx, protocolContext);
final BlockResultFactory blockResultFactory,
final EngineCallListener engineCallListener) {
super(vertx, protocolContext, engineCallListener);
this.blockResultFactory = blockResultFactory;
}
@ -55,6 +56,8 @@ public class EngineGetPayload extends ExecutionEngineJsonRpcMethod {
@Override
public JsonRpcResponse syncResponse(final JsonRpcRequestContext request) {
engineCallListener.executionEngineCalled();
final PayloadIdentifier payloadId = request.getRequiredParameter(0, PayloadIdentifier.class);
final Optional<Block> block = mergeContext.get().retrieveBlockById(payloadId);

@ -73,8 +73,9 @@ public class EngineNewPayload extends ExecutionEngineJsonRpcMethod {
final Vertx vertx,
final ProtocolContext protocolContext,
final MergeMiningCoordinator mergeCoordinator,
final EthPeers ethPeers) {
super(vertx, protocolContext);
final EthPeers ethPeers,
final EngineCallListener engineCallListener) {
super(vertx, protocolContext, engineCallListener);
this.mergeCoordinator = mergeCoordinator;
this.ethPeers = ethPeers;
}
@ -86,6 +87,8 @@ public class EngineNewPayload extends ExecutionEngineJsonRpcMethod {
@Override
public JsonRpcResponse syncResponse(final JsonRpcRequestContext requestContext) {
engineCallListener.executionEngineCalled();
final EnginePayloadParameter blockParam =
requestContext.getRequiredParameter(0, EnginePayloadParameter.class);

@ -0,0 +1,50 @@
/*
* Copyright Hyperledger Besu Contributors.
*
* 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.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.ethereum.api.jsonrpc.internal.methods.engine;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.QosTimer;
import com.google.common.annotations.VisibleForTesting;
import io.vertx.core.Vertx;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class EngineQosTimer implements EngineCallListener {
static final long QOS_TIMEOUT_MILLIS = 120000L;
private static final Logger LOG = LoggerFactory.getLogger(EngineQosTimer.class);
private final QosTimer qosTimer;
public EngineQosTimer(final Vertx vertx) {
qosTimer = new QosTimer(vertx, QOS_TIMEOUT_MILLIS, lastCall -> logTimeoutWarning());
qosTimer.resetTimer();
}
@Override
public void executionEngineCalled() {
getQosTimer().resetTimer();
}
public void logTimeoutWarning() {
LOG.warn(
"Execution engine not called in {} seconds, consensus client may not be connected",
QOS_TIMEOUT_MILLIS / 1000L);
}
@VisibleForTesting
public QosTimer getQosTimer() {
return qosTimer;
}
}

@ -22,6 +22,7 @@ import org.hyperledger.besu.ethereum.api.jsonrpc.internal.methods.engine.EngineE
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.methods.engine.EngineForkchoiceUpdated;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.methods.engine.EngineGetPayload;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.methods.engine.EngineNewPayload;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.methods.engine.EngineQosTimer;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.results.BlockResultFactory;
import org.hyperledger.besu.ethereum.blockcreation.MiningCoordinator;
import org.hyperledger.besu.ethereum.eth.manager.EthPeers;
@ -62,17 +63,26 @@ public class ExecutionEngineJsonRpcMethods extends ApiGroupJsonRpcMethods {
@Override
protected Map<String, JsonRpcMethod> create() {
final EngineQosTimer engineQosTimer = new EngineQosTimer(consensusEngineServer);
if (mergeCoordinator.isPresent()) {
return mapOf(
new EngineGetPayload(consensusEngineServer, protocolContext, blockResultFactory),
new EngineGetPayload(
consensusEngineServer, protocolContext, blockResultFactory, engineQosTimer),
new EngineNewPayload(
consensusEngineServer, protocolContext, mergeCoordinator.get(), ethPeers),
consensusEngineServer,
protocolContext,
mergeCoordinator.get(),
ethPeers,
engineQosTimer),
new EngineForkchoiceUpdated(
consensusEngineServer, protocolContext, mergeCoordinator.get()),
new EngineExchangeTransitionConfiguration(consensusEngineServer, protocolContext));
consensusEngineServer, protocolContext, mergeCoordinator.get(), engineQosTimer),
new EngineExchangeTransitionConfiguration(
consensusEngineServer, protocolContext, engineQosTimer));
} else {
return mapOf(
new EngineExchangeTransitionConfiguration(consensusEngineServer, protocolContext));
new EngineExchangeTransitionConfiguration(
consensusEngineServer, protocolContext, engineQosTimer));
}
}
}

@ -15,8 +15,6 @@
package org.hyperledger.besu.ethereum.api.jsonrpc.internal.methods.engine;
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
@ -29,7 +27,6 @@ import org.hyperledger.besu.ethereum.ProtocolContext;
import org.hyperledger.besu.ethereum.api.jsonrpc.RpcMethod;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.JsonRpcRequest;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.JsonRpcRequestContext;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.QosTimer;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.parameters.EngineExchangeTransitionConfigurationParameter;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.parameters.UnsignedLongParameter;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.response.JsonRpcResponse;
@ -46,35 +43,37 @@ import org.hyperledger.besu.evm.log.LogsBloomFilter;
import java.math.BigInteger;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.atomic.AtomicInteger;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import io.vertx.core.Vertx;
import io.vertx.ext.unit.Async;
import io.vertx.ext.unit.TestContext;
import io.vertx.ext.unit.junit.VertxUnitRunner;
import org.apache.tuweni.bytes.Bytes;
import org.apache.tuweni.bytes.Bytes32;
import org.apache.tuweni.units.bigints.UInt256;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.MockitoJUnitRunner;
@RunWith(VertxUnitRunner.class)
@RunWith(MockitoJUnitRunner.class)
public class EngineExchangeTransitionConfigurationTest {
private EngineExchangeTransitionConfiguration method;
private static final Vertx vertx = Vertx.vertx();
private final ProtocolContext protocolContext = mock(ProtocolContext.class);
private final MergeContext mergeContext = mock(MergeContext.class);
@Mock private ProtocolContext protocolContext;
@Mock private MergeContext mergeContext;
@Mock private EngineCallListener engineCallListener;
@Before
public void setUp() {
when(protocolContext.getConsensusContext(Mockito.any())).thenReturn(mergeContext);
when(protocolContext.safeConsensusContext(Mockito.any())).thenReturn(Optional.of(mergeContext));
this.method = new EngineExchangeTransitionConfiguration(vertx, protocolContext);
this.method =
new EngineExchangeTransitionConfiguration(vertx, protocolContext, engineCallListener);
}
@Test
@ -99,6 +98,7 @@ public class EngineExchangeTransitionConfigurationTest {
assertThat(result.getTerminalTotalDifficulty()).isEqualTo(Difficulty.of(1337L));
assertThat(result.getTerminalBlockHash()).isEqualTo(mockBlockHeader.getHash());
assertThat(result.getTerminalBlockNumber()).isEqualTo(mockBlockHeader.getNumber());
verify(engineCallListener, times(1)).executionEngineCalled();
}
@Test
@ -115,12 +115,11 @@ public class EngineExchangeTransitionConfigurationTest {
assertThat(result.getTerminalTotalDifficulty()).isEqualTo(Difficulty.of(1337L));
assertThat(result.getTerminalBlockHash()).isEqualTo(Hash.ZERO);
assertThat(result.getTerminalBlockNumber()).isEqualTo(0L);
verify(engineCallListener, times(1)).executionEngineCalled();
}
@Test
public void shouldReturnDefaultOnNoTerminalTotalDifficultyConfigured() {
when(protocolContext.safeConsensusContext(Mockito.any())).thenReturn(Optional.empty());
var response =
resp(
new EngineExchangeTransitionConfigurationParameter(
@ -135,6 +134,7 @@ public class EngineExchangeTransitionConfigurationTest {
10)));
assertThat(result.getTerminalBlockHash()).isEqualTo(Hash.ZERO);
assertThat(result.getTerminalBlockNumber()).isEqualTo(0L);
verify(engineCallListener, times(1)).executionEngineCalled();
}
@Test
@ -154,6 +154,7 @@ public class EngineExchangeTransitionConfigurationTest {
assertThat(result.getTerminalTotalDifficulty()).isEqualTo(Difficulty.of(24));
assertThat(result.getTerminalBlockHash()).isEqualTo(Hash.fromHexStringLenient("0x01"));
assertThat(result.getTerminalBlockNumber()).isEqualTo(42);
verify(engineCallListener, times(1)).executionEngineCalled();
}
@Test
@ -173,6 +174,7 @@ public class EngineExchangeTransitionConfigurationTest {
assertThat(result.getTerminalTotalDifficulty()).isEqualTo(Difficulty.of(24));
assertThat(result.getTerminalBlockHash()).isEqualTo(Hash.fromHexStringLenient("0x01"));
assertThat(result.getTerminalBlockNumber()).isEqualTo(42);
verify(engineCallListener, times(1)).executionEngineCalled();
}
@Test
@ -211,102 +213,6 @@ public class EngineExchangeTransitionConfigurationTest {
assertThat(res.get("terminalTotalDifficulty")).isEqualTo("0x0");
}
@Test
public void shouldWarnWhenExchangeConfigNotCalledWithinTimeout(final TestContext ctx) {
final long TEST_QOS_TIMEOUT = 75L;
final Async async = ctx.async();
final AtomicInteger logCounter = new AtomicInteger(0);
final var spyMethod = spy(method);
final var spyTimer =
spy(new QosTimer(vertx, TEST_QOS_TIMEOUT, z -> logCounter.incrementAndGet()));
when(spyMethod.getQosTimer()).thenReturn(spyTimer);
spyTimer.resetTimer();
vertx.setTimer(
100L,
z -> {
try {
// just once on construction:
verify(spyTimer, times(1)).resetTimer();
} catch (Exception ex) {
ctx.fail(ex);
}
// assert one warn:
ctx.assertEquals(1, logCounter.get());
async.complete();
});
}
@Test
public void shouldNotWarnWhenTimerExecutesBeforeTimeout(final TestContext ctx) {
final long TEST_QOS_TIMEOUT = 500L;
final Async async = ctx.async();
final AtomicInteger logCounter = new AtomicInteger(0);
final var spyMethod = spy(method);
final var spyTimer =
spy(new QosTimer(vertx, TEST_QOS_TIMEOUT, z -> logCounter.incrementAndGet()));
when(spyMethod.getQosTimer()).thenReturn(spyTimer);
spyTimer.resetTimer();
vertx.setTimer(
50L,
z -> {
try {
// just once on construction:
verify(spyTimer, times(1)).resetTimer();
} catch (Exception ex) {
ctx.fail(ex);
}
// should not warn
ctx.assertEquals(0, logCounter.get());
async.complete();
});
}
@Test
public void shouldNotWarnWhenExchangeConfigurationCalledWithinTimeout(final TestContext ctx) {
final long TEST_QOS_TIMEOUT = 75L;
final Async async = ctx.async();
final AtomicInteger logCounter = new AtomicInteger(0);
final var spyMethod = spy(method);
final var spyTimer =
spy(new QosTimer(vertx, TEST_QOS_TIMEOUT, z -> logCounter.incrementAndGet()));
when(mergeContext.getTerminalPoWBlock()).thenReturn(Optional.empty());
when(mergeContext.getTerminalTotalDifficulty()).thenReturn(Difficulty.of(1337L));
when(spyMethod.getQosTimer()).thenReturn(spyTimer);
spyTimer.resetTimer();
// call exchangeTransitionConfiguration 50 milliseconds hence to reset our QoS timer
vertx.setTimer(
50L,
z ->
spyMethod.syncResponse(
new JsonRpcRequestContext(
new JsonRpcRequest(
"2.0",
RpcMethod.ENGINE_EXCHANGE_TRANSITION_CONFIGURATION.getMethodName(),
new Object[] {
new EngineExchangeTransitionConfigurationParameter(
"24",
Hash.fromHexStringLenient("0x01").toHexString(),
new UnsignedLongParameter(0))
}))));
vertx.setTimer(
100L,
z -> {
try {
// once on construction, once on call:
verify(spyTimer, times(2)).resetTimer();
} catch (Exception ex) {
ctx.fail(ex);
}
// should not warn
ctx.assertEquals(0, logCounter.get());
async.complete();
});
}
private JsonRpcResponse resp(final EngineExchangeTransitionConfigurationParameter param) {
return method.response(
new JsonRpcRequestContext(

@ -21,6 +21,7 @@ import static org.hyperledger.besu.ethereum.api.jsonrpc.internal.methods.Executi
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.never;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
@ -77,11 +78,14 @@ public class EngineForkchoiceUpdatedTest {
@Mock private MutableBlockchain blockchain;
@Mock private EngineCallListener engineCallListener;
@Before
public void before() {
when(protocolContext.safeConsensusContext(Mockito.any())).thenReturn(Optional.of(mergeContext));
when(protocolContext.getBlockchain()).thenReturn(blockchain);
this.method = new EngineForkchoiceUpdated(vertx, protocolContext, mergeCoordinator);
this.method =
new EngineForkchoiceUpdated(vertx, protocolContext, mergeCoordinator, engineCallListener);
}
@Test
@ -189,6 +193,7 @@ public class EngineForkchoiceUpdatedTest {
assertThat(resp.getPayloadStatus().getLatestValidHash().get()).isEqualTo(parent.getBlockHash());
assertThat(resp.getPayloadStatus().getError())
.isEqualTo("new head timestamp not greater than parent");
verify(engineCallListener, times(1)).executionEngineCalled();
}
@Test
@ -255,6 +260,7 @@ public class EngineForkchoiceUpdatedTest {
Optional.empty());
assertInvalidForkchoiceState(resp);
verify(engineCallListener, times(1)).executionEngineCalled();
}
@Test
@ -276,6 +282,7 @@ public class EngineForkchoiceUpdatedTest {
Optional.empty());
assertInvalidForkchoiceState(resp);
verify(engineCallListener, times(1)).executionEngineCalled();
}
@Test
@ -300,6 +307,7 @@ public class EngineForkchoiceUpdatedTest {
Optional.empty());
assertInvalidForkchoiceState(resp);
verify(engineCallListener, times(1)).executionEngineCalled();
}
@Test
@ -327,6 +335,7 @@ public class EngineForkchoiceUpdatedTest {
Optional.empty());
assertInvalidForkchoiceState(resp);
verify(engineCallListener, times(1)).executionEngineCalled();
}
@Test
@ -351,6 +360,7 @@ public class EngineForkchoiceUpdatedTest {
Optional.empty());
assertInvalidForkchoiceState(resp);
verify(engineCallListener, times(1)).executionEngineCalled();
}
@Test
@ -376,6 +386,7 @@ public class EngineForkchoiceUpdatedTest {
Optional.empty());
assertInvalidForkchoiceState(resp);
verify(engineCallListener, times(1)).executionEngineCalled();
}
@Test
@ -412,6 +423,8 @@ public class EngineForkchoiceUpdatedTest {
assertThat(forkchoiceRes.getPayloadStatus().getLatestValidHashAsString())
.isEqualTo(mockHeader.getHash().toHexString());
assertThat(forkchoiceRes.getPayloadId()).isNull();
assertThat(forkchoiceRes.getPayloadId()).isNull();
verify(engineCallListener, times(1)).executionEngineCalled();
}
private EngineUpdateForkchoiceResult assertSuccessWithPayloadForForkchoiceResult(
@ -464,6 +477,8 @@ public class EngineForkchoiceUpdatedTest {
: Optional.of(fcuParam.getFinalizedBlockHash()),
fcuParam.getSafeBlockHash());
verify(engineCallListener, times(1)).executionEngineCalled();
return res;
}

@ -15,6 +15,8 @@
package org.hyperledger.besu.ethereum.api.jsonrpc.internal.methods.engine;
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
import org.hyperledger.besu.consensus.merge.MergeContext;
@ -63,11 +65,13 @@ public class EngineGetPayloadTest {
@Mock private MergeContext mergeContext;
@Mock private EngineCallListener engineCallListener;
@Before
public void before() {
when(mergeContext.retrieveBlockById(mockPid)).thenReturn(Optional.of(mockBlock));
when(protocolContext.safeConsensusContext(Mockito.any())).thenReturn(Optional.of(mergeContext));
this.method = new EngineGetPayload(vertx, protocolContext, factory);
this.method = new EngineGetPayload(vertx, protocolContext, factory, engineCallListener);
}
@Test
@ -90,12 +94,14 @@ public class EngineGetPayloadTest {
assertThat(res.getPrevRandao())
.isEqualTo(mockHeader.getPrevRandao().map(Bytes32::toString).orElse(""));
});
verify(engineCallListener, times(1)).executionEngineCalled();
}
@Test
public void shouldFailForUnknownPayloadId() {
var resp = resp(PayloadIdentifier.forPayloadParams(Hash.ZERO, 0L));
assertThat(resp).isInstanceOf(JsonRpcErrorResponse.class);
verify(engineCallListener, times(1)).executionEngineCalled();
}
private JsonRpcResponse resp(final PayloadIdentifier pid) {

@ -24,6 +24,7 @@ import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.atLeastOnce;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
@ -83,12 +84,16 @@ public class EngineNewPayloadTest {
@Mock private EthPeers ethPeers;
@Mock private EngineCallListener engineCallListener;
@Before
public void before() {
when(protocolContext.safeConsensusContext(Mockito.any())).thenReturn(Optional.of(mergeContext));
when(protocolContext.getBlockchain()).thenReturn(blockchain);
when(ethPeers.peerCount()).thenReturn(1);
this.method = new EngineNewPayload(vertx, protocolContext, mergeCoordinator, ethPeers);
this.method =
new EngineNewPayload(
vertx, protocolContext, mergeCoordinator, ethPeers, engineCallListener);
}
@Test
@ -116,6 +121,7 @@ public class EngineNewPayloadTest {
assertThat(res.getLatestValidHash().get()).isEqualTo(mockHeader.getHash());
assertThat(res.getStatusAsString()).isEqualTo(VALID.name());
assertThat(res.getError()).isNull();
verify(engineCallListener, times(1)).executionEngineCalled();
}
@Test
@ -136,6 +142,7 @@ public class EngineNewPayloadTest {
assertThat(res.getLatestValidHash().get()).isEqualTo(mockHash);
assertThat(res.getStatusAsString()).isEqualTo(INVALID.name());
assertThat(res.getError()).isEqualTo("error 42");
verify(engineCallListener, times(1)).executionEngineCalled();
}
@Test
@ -155,6 +162,7 @@ public class EngineNewPayloadTest {
assertThat(res.getLatestValidHash()).isEmpty();
assertThat(res.getStatusAsString()).isEqualTo(ACCEPTED.name());
assertThat(res.getError()).isNull();
verify(engineCallListener, times(1)).executionEngineCalled();
}
@Test
@ -171,6 +179,7 @@ public class EngineNewPayloadTest {
assertThat(res.getLatestValidHash().get()).isEqualTo(mockHeader.getHash());
assertThat(res.getStatusAsString()).isEqualTo(VALID.name());
assertThat(res.getError()).isNull();
verify(engineCallListener, times(1)).executionEngineCalled();
}
@Test
@ -188,6 +197,7 @@ public class EngineNewPayloadTest {
assertThat(res.getLatestValidHash()).isEqualTo(Optional.of(Hash.ZERO));
assertThat(res.getStatusAsString()).isEqualTo(INVALID.name());
verify(mergeCoordinator, atLeastOnce()).addBadBlock(any());
verify(engineCallListener, times(1)).executionEngineCalled();
}
@Test
@ -205,6 +215,7 @@ public class EngineNewPayloadTest {
EnginePayloadStatusResult res = fromSuccessResp(resp);
assertThat(res.getLatestValidHash()).isEqualTo(Optional.of(latestValidHash));
assertThat(res.getStatusAsString()).isEqualTo(INVALID.name());
verify(engineCallListener, times(1)).executionEngineCalled();
}
@Test
@ -223,6 +234,7 @@ public class EngineNewPayloadTest {
var resp = resp(mockPayload(mockHeader, Collections.emptyList()));
fromErrorResp(resp);
verify(engineCallListener, times(1)).executionEngineCalled();
}
@Test
@ -234,6 +246,7 @@ public class EngineNewPayloadTest {
EnginePayloadStatusResult res = fromSuccessResp(resp);
assertThat(res.getLatestValidHash()).isEmpty();
assertThat(res.getStatusAsString()).isEqualTo(INVALID_BLOCK_HASH.name());
verify(engineCallListener, times(1)).executionEngineCalled();
}
@Test
@ -247,6 +260,7 @@ public class EngineNewPayloadTest {
EnginePayloadStatusResult res = fromSuccessResp(resp);
assertThat(res.getLatestValidHash()).isEmpty();
assertThat(res.getStatusAsString()).isEqualTo(INVALID_BLOCK_HASH.name());
verify(engineCallListener, times(1)).executionEngineCalled();
}
@Test
@ -261,6 +275,7 @@ public class EngineNewPayloadTest {
assertThat(res.getLatestValidHash().get()).isEqualTo(mockHash);
assertThat(res.getStatusAsString()).isEqualTo(INVALID.name());
assertThat(res.getError()).isEqualTo("Failed to decode transactions from block parameter");
verify(engineCallListener, times(1)).executionEngineCalled();
}
@Test
@ -282,6 +297,7 @@ public class EngineNewPayloadTest {
var payloadStatusResult = (EnginePayloadStatusResult) res;
assertThat(payloadStatusResult.getStatus()).isEqualTo(INVALID);
assertThat(payloadStatusResult.getError()).isEqualTo("block timestamp not greater than parent");
verify(engineCallListener, times(1)).executionEngineCalled();
}
@Test
@ -297,6 +313,7 @@ public class EngineNewPayloadTest {
assertThat(res.getError()).isNull();
assertThat(res.getStatusAsString()).isEqualTo(SYNCING.name());
assertThat(res.getLatestValidHash()).isEmpty();
verify(engineCallListener, times(1)).executionEngineCalled();
}
@Test
@ -310,6 +327,7 @@ public class EngineNewPayloadTest {
assertThat(res.getLatestValidHash()).isEmpty();
assertThat(res.getStatusAsString()).isEqualTo(SYNCING.name());
assertThat(res.getError()).isNull();
verify(engineCallListener, times(1)).executionEngineCalled();
}
@Test
@ -331,6 +349,7 @@ public class EngineNewPayloadTest {
assertThat(res.getLatestValidHash()).isEmpty();
assertThat(res.getStatusAsString()).isEqualTo(INVALID.name());
assertThat(res.getError()).isEqualTo("Field extraData must not be null");
verify(engineCallListener, times(1)).executionEngineCalled();
}
@Test
@ -344,6 +363,7 @@ public class EngineNewPayloadTest {
assertThat(res.getLatestValidHash()).contains(Hash.ZERO);
assertThat(res.getStatusAsString()).isEqualTo(INVALID.name());
assertThat(res.getError()).isNull();
verify(engineCallListener, times(1)).executionEngineCalled();
}
private JsonRpcResponse resp(final EnginePayloadParameter payload) {

@ -0,0 +1,101 @@
/*
* Copyright Hyperledger Besu Contributors.
*
* 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.
*
* SPDX-License-Identifier: Apache-2.0
*/
package org.hyperledger.besu.ethereum.api.jsonrpc.internal.methods.engine;
import static org.mockito.Mockito.atLeast;
import static org.mockito.Mockito.atLeastOnce;
import static org.mockito.Mockito.never;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
import org.hyperledger.besu.ethereum.api.jsonrpc.internal.QosTimer;
import io.vertx.core.Vertx;
import io.vertx.ext.unit.Async;
import io.vertx.ext.unit.TestContext;
import io.vertx.ext.unit.junit.VertxUnitRunner;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
@RunWith(VertxUnitRunner.class)
public class EngineQosTimerTest {
private EngineQosTimer engineQosTimer;
private Vertx vertx;
@Before
public void setUp() throws Exception {
vertx = Vertx.vertx();
engineQosTimer = new EngineQosTimer(vertx);
}
@After
public void cleanUp() {
vertx.close();
}
@Test
public void shouldNotWarnWhenCalledWithinTimeout(final TestContext ctx) {
final long TEST_QOS_TIMEOUT = 75L;
final Async async = ctx.async();
final var spyEngineQosTimer = spy(engineQosTimer);
final var spyTimer =
spy(new QosTimer(vertx, TEST_QOS_TIMEOUT, z -> spyEngineQosTimer.logTimeoutWarning()));
spyTimer.resetTimer();
when(spyEngineQosTimer.getQosTimer()).thenReturn(spyTimer);
// call executionEngineCalled() 50 milliseconds hence to reset our QoS timer
vertx.setTimer(50L, z -> spyEngineQosTimer.executionEngineCalled());
vertx.setTimer(
100L,
z -> {
try {
verify(spyTimer, atLeast(2)).resetTimer();
// should not warn
verify(spyEngineQosTimer, never()).logTimeoutWarning();
async.complete();
} catch (Exception ex) {
ctx.fail(ex);
}
});
}
@Test
public void shouldWarnWhenNotCalledWithinTimeout(final TestContext ctx) {
final long TEST_QOS_TIMEOUT = 75L;
final Async async = ctx.async();
final var spyEngineQosTimer = spy(engineQosTimer);
final var spyTimer =
spy(new QosTimer(vertx, TEST_QOS_TIMEOUT, z -> spyEngineQosTimer.logTimeoutWarning()));
spyTimer.resetTimer();
when(spyEngineQosTimer.getQosTimer()).thenReturn(spyTimer);
vertx.setTimer(
100L,
z -> {
try {
verify(spyTimer, atLeastOnce()).resetTimer();
// should warn
verify(spyEngineQosTimer, atLeastOnce()).logTimeoutWarning();
async.complete();
} catch (Exception ex) {
ctx.fail(ex);
}
});
}
}
Loading…
Cancel
Save