diff --git a/CHANGELOG.md b/CHANGELOG.md index 616ac3d1dc8..1629002cbd7 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -15,8 +15,10 @@ ### Additions and Improvements - Expose set finalized/safe block in plugin api BlockchainService. These method can be used by plugins to set finalized/safe block for a PoA network (such as QBFT, IBFT and Clique).[#7382](https://github.com/hyperledger/besu/pull/7382) - In process RPC service [#7395](https://github.com/hyperledger/besu/pull/7395) +- Added support for tracing private transactions using `priv_traceTransaction` API. [#6161](https://github.com/hyperledger/besu/pull/6161) - Wrap WorldUpdater into EVMWorldupdater [#7434](https://github.com/hyperledger/besu/pull/7434) + ### Bug fixes - Correct entrypoint in Docker evmtool [#7430](https://github.com/hyperledger/besu/pull/7430) - Fix protocol schedule check for devnets [#7429](https://github.com/hyperledger/besu/pull/7429) diff --git a/acceptance-tests/dsl/src/main/java/org/hyperledger/besu/tests/acceptance/dsl/privacy/transaction/PrivacyTransactions.java b/acceptance-tests/dsl/src/main/java/org/hyperledger/besu/tests/acceptance/dsl/privacy/transaction/PrivacyTransactions.java index 3763e40bb7c..d168108c975 100644 --- a/acceptance-tests/dsl/src/main/java/org/hyperledger/besu/tests/acceptance/dsl/privacy/transaction/PrivacyTransactions.java +++ b/acceptance-tests/dsl/src/main/java/org/hyperledger/besu/tests/acceptance/dsl/privacy/transaction/PrivacyTransactions.java @@ -15,6 +15,7 @@ package org.hyperledger.besu.tests.acceptance.dsl.privacy.transaction; import org.hyperledger.besu.datatypes.Address; +import org.hyperledger.besu.datatypes.Hash; import org.hyperledger.besu.ethereum.privacy.PrivacyGroupUtil; import org.hyperledger.besu.tests.acceptance.dsl.privacy.PrivacyNode; import org.hyperledger.besu.tests.acceptance.dsl.privacy.condition.PrivGetTransactionReceiptTransaction; @@ -25,6 +26,7 @@ import org.hyperledger.besu.tests.acceptance.dsl.transaction.privacy.PrivGetCodeTransaction; import org.hyperledger.besu.tests.acceptance.dsl.transaction.privacy.PrivGetLogsTransaction; import org.hyperledger.besu.tests.acceptance.dsl.transaction.privacy.PrivGetTransaction; +import org.hyperledger.besu.tests.acceptance.dsl.transaction.privacy.PrivTraceTransaction; import org.hyperledger.besu.tests.acceptance.dsl.transaction.privacy.filter.PrivGetFilterChangesTransaction; import org.hyperledger.besu.tests.acceptance.dsl.transaction.privacy.filter.PrivGetFilterLogsTransaction; import org.hyperledger.besu.tests.acceptance.dsl.transaction.privacy.filter.PrivNewFilterTransaction; @@ -115,6 +117,11 @@ public PrivGetLogsTransaction privGetLogs( return new PrivGetLogsTransaction(privacyGroupId, filterParameter); } + public PrivTraceTransaction privTraceTransaction( + final String privacyGroupId, final Hash transactionHash) { + return new PrivTraceTransaction(privacyGroupId, transactionHash); + } + public RemoveFromFlexiblePrivacyGroupTransaction removeFromPrivacyGroup( final String privacyGroupId, final String remover, diff --git a/acceptance-tests/dsl/src/main/java/org/hyperledger/besu/tests/acceptance/dsl/transaction/privacy/PrivTraceTransaction.java b/acceptance-tests/dsl/src/main/java/org/hyperledger/besu/tests/acceptance/dsl/transaction/privacy/PrivTraceTransaction.java new file mode 100644 index 00000000000..b971c4afde7 --- /dev/null +++ b/acceptance-tests/dsl/src/main/java/org/hyperledger/besu/tests/acceptance/dsl/transaction/privacy/PrivTraceTransaction.java @@ -0,0 +1,49 @@ +/* + * Copyright contributors to Hyperledger Besu. + * + * 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.tests.acceptance.dsl.transaction.privacy; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.hyperledger.besu.datatypes.Hash; +import org.hyperledger.besu.tests.acceptance.dsl.transaction.NodeRequests; +import org.hyperledger.besu.tests.acceptance.dsl.transaction.Transaction; + +import java.io.IOException; + +public class PrivTraceTransaction implements Transaction { + + private final String privacyGroupId; + private final Hash transactionHash; + + public PrivTraceTransaction(final String privacyGroupId, final Hash transactionHash) { + this.privacyGroupId = privacyGroupId; + this.transactionHash = transactionHash; + } + + @Override + public String execute(final NodeRequests node) { + try { + final PrivacyRequestFactory.PrivTraceTransaction response = + node.privacy().privTraceTransaction(privacyGroupId, transactionHash).send(); + + assertThat(response).as("check response is not null").isNotNull(); + assertThat(response.getResult()).as("check result in response is not null").isNotNull(); + + return response.getResult(); + } catch (final IOException e) { + throw new RuntimeException(e); + } + } +} diff --git a/acceptance-tests/dsl/src/main/java/org/hyperledger/besu/tests/acceptance/dsl/transaction/privacy/PrivacyRequestFactory.java b/acceptance-tests/dsl/src/main/java/org/hyperledger/besu/tests/acceptance/dsl/transaction/privacy/PrivacyRequestFactory.java index 056598a9d56..7c01c02c48e 100644 --- a/acceptance-tests/dsl/src/main/java/org/hyperledger/besu/tests/acceptance/dsl/transaction/privacy/PrivacyRequestFactory.java +++ b/acceptance-tests/dsl/src/main/java/org/hyperledger/besu/tests/acceptance/dsl/transaction/privacy/PrivacyRequestFactory.java @@ -109,6 +109,8 @@ public static class GetPrivacyPrecompileAddressResponse extends Response
{} + public static class PrivTraceTransaction extends Response {} + public static class CreatePrivacyGroupResponse extends Response {} public static class DeletePrivacyGroupResponse extends Response {} @@ -455,6 +457,16 @@ public Request privGetLogs( "priv_getLogs", Arrays.asList(privacyGroupId, filterParameter), web3jService, EthLog.class); } + public Request privTraceTransaction( + final String privacyGroupId, final Hash transactionHash) { + + return new Request<>( + "priv_traceTransaction", + Arrays.asList(privacyGroupId, transactionHash), + web3jService, + PrivTraceTransaction.class); + } + public Request privNewFilter( final String privacyGroupId, final LogFilterJsonParameter filterParameter) { return new Request<>( diff --git a/acceptance-tests/tests/src/test/java/org/hyperledger/besu/tests/acceptance/privacy/PrivTraceTransactionAcceptanceTest.java b/acceptance-tests/tests/src/test/java/org/hyperledger/besu/tests/acceptance/privacy/PrivTraceTransactionAcceptanceTest.java new file mode 100644 index 00000000000..9cac6d8bf59 --- /dev/null +++ b/acceptance-tests/tests/src/test/java/org/hyperledger/besu/tests/acceptance/privacy/PrivTraceTransactionAcceptanceTest.java @@ -0,0 +1,174 @@ +/* + * Copyright contributors to Hyperledger Besu. + * + * 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.tests.acceptance.privacy; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.web3j.utils.Restriction.UNRESTRICTED; + +import org.hyperledger.besu.datatypes.Hash; +import org.hyperledger.besu.tests.acceptance.dsl.privacy.ParameterizedEnclaveTestBase; +import org.hyperledger.besu.tests.acceptance.dsl.privacy.PrivacyNode; +import org.hyperledger.besu.tests.acceptance.dsl.privacy.account.PrivacyAccountResolver; +import org.hyperledger.besu.tests.web3j.generated.SimpleStorage; +import org.hyperledger.enclave.testutil.EnclaveEncryptorType; +import org.hyperledger.enclave.testutil.EnclaveType; + +import java.io.IOException; +import java.math.BigInteger; +import java.util.Optional; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.junit.Test; +import org.web3j.utils.Restriction; + +public class PrivTraceTransactionAcceptanceTest extends ParameterizedEnclaveTestBase { + + private final PrivacyNode node; + + private final PrivacyNode wrongNode; + + public PrivTraceTransactionAcceptanceTest( + final Restriction restriction, + final EnclaveType enclaveType, + final EnclaveEncryptorType enclaveEncryptorType) + throws IOException { + + super(restriction, enclaveType, enclaveEncryptorType); + + node = + privacyBesu.createPrivateTransactionEnabledMinerNode( + restriction + "-node", + PrivacyAccountResolver.ALICE.resolve(enclaveEncryptorType), + enclaveType, + Optional.empty(), + false, + false, + restriction == UNRESTRICTED); + + wrongNode = + privacyBesu.createPrivateTransactionEnabledMinerNode( + restriction + "-node", + PrivacyAccountResolver.BOB.resolve(enclaveEncryptorType), + enclaveType, + Optional.empty(), + false, + false, + restriction == UNRESTRICTED); + + privacyCluster.start(node); + privacyCluster.start(wrongNode); + } + + @Test + public void getTransactionTrace() throws JsonProcessingException { + final String privacyGroupId = createPrivacyGroup(); + final SimpleStorage simpleStorageContract = deploySimpleStorageContract(privacyGroupId); + + Hash transactionHash = + Hash.fromHexString(doTransaction(privacyGroupId, simpleStorageContract, 0)); + + final String result = + node.execute(privacyTransactions.privTraceTransaction(privacyGroupId, transactionHash)); + + assertThat(result).isNotNull(); + ObjectMapper mapper = new ObjectMapper(); + + JsonNode rootNode = mapper.readTree(result); + JsonNode resultNode = rootNode.get("result"); + + assertThat(resultNode).isNotNull(); + assertThat(resultNode.isArray()).isTrue(); + assertThat(resultNode.size()).isGreaterThan(0); + + JsonNode trace = resultNode.get(0); + assertThat(trace.get("action").get("callType").asText()).isEqualTo("call"); + assertThat(trace.get("action").get("from").asText()).isEqualTo(node.getAddress().toString()); + assertThat(trace.get("action").get("input").asText()).startsWith("0x60fe47b1"); + assertThat(trace.get("action").get("to").asText()) + .isEqualTo(simpleStorageContract.getContractAddress()); + assertThat(trace.get("action").get("value").asText()).isEqualTo("0x0"); + assertThat(trace.get("blockHash").asText()).isNotEmpty(); + assertThat(trace.get("blockNumber").asInt()).isGreaterThan(0); + assertThat(trace.get("transactionHash").asText()).isEqualTo(transactionHash.toString()); + assertThat(trace.get("type").asText()).isEqualTo("call"); + + final String wrongPrivacyGroupId = createWrongPrivacyGroup(); + + final String resultEmpty = + wrongNode.execute( + privacyTransactions.privTraceTransaction(wrongPrivacyGroupId, transactionHash)); + + ObjectMapper mapperEmpty = new ObjectMapper(); + + JsonNode rootNodeEmpty = mapperEmpty.readTree(resultEmpty); + JsonNode resultNodeEmpty = rootNodeEmpty.get("result"); + + assertThat(resultNodeEmpty).isNotNull(); + assertThat(resultNodeEmpty.isArray()).isTrue(); + assertThat(resultNodeEmpty.isEmpty()).isTrue(); + + final String resultWrongHash = + wrongNode.execute(privacyTransactions.privTraceTransaction(privacyGroupId, Hash.EMPTY)); + + ObjectMapper mapperWrongHash = new ObjectMapper(); + + JsonNode rootNodeWrongHash = mapperWrongHash.readTree(resultWrongHash); + JsonNode resultNodeWrongHash = rootNodeWrongHash.get("result"); + + assertThat(resultNodeWrongHash).isNotNull(); + assertThat(resultNodeWrongHash.isArray()).isTrue(); + assertThat(resultNodeWrongHash.isEmpty()).isTrue(); + } + + private String createPrivacyGroup() { + return node.execute(createPrivacyGroup("myGroupName", "my group description", node)); + } + + private String createWrongPrivacyGroup() { + return wrongNode.execute(createPrivacyGroup("myGroupName", "my group description", wrongNode)); + } + + private SimpleStorage deploySimpleStorageContract(final String privacyGroupId) { + final SimpleStorage simpleStorage = + node.execute( + privateContractTransactions.createSmartContractWithPrivacyGroupId( + SimpleStorage.class, + node.getTransactionSigningKey(), + restriction, + node.getEnclaveKey(), + privacyGroupId)); + + privateContractVerifier + .validPrivateContractDeployed( + simpleStorage.getContractAddress(), node.getAddress().toString()) + .verify(simpleStorage); + + return simpleStorage; + } + + private String doTransaction( + final String privacyGroupId, final SimpleStorage simpleStorageContract, final int value) { + return node.execute( + privateContractTransactions.callSmartContractWithPrivacyGroupId( + simpleStorageContract.getContractAddress(), + simpleStorageContract.set(BigInteger.valueOf(value)).encodeFunctionCall(), + node.getTransactionSigningKey(), + restriction, + node.getEnclaveKey(), + privacyGroupId)); + } +} diff --git a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/RpcMethod.java b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/RpcMethod.java index 4174fdf8313..2c883337d34 100644 --- a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/RpcMethod.java +++ b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/RpcMethod.java @@ -86,6 +86,7 @@ public enum RpcMethod { PRIV_GET_FILTER_LOGS("priv_getFilterLogs"), PRIV_SUBSCRIBE("priv_subscribe"), PRIV_UNSUBSCRIBE("priv_unsubscribe"), + PRIV_TRACE_TRANSACTION("priv_traceTransaction"), PRIVX_FIND_PRIVACY_GROUP_OLD("privx_findOnchainPrivacyGroup"), PRIVX_FIND_PRIVACY_GROUP("privx_findFlexiblePrivacyGroup"), EEA_SEND_RAW_TRANSACTION("eea_sendRawTransaction"), diff --git a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/privacy/methods/priv/AbstractPrivateTraceByHash.java b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/privacy/methods/priv/AbstractPrivateTraceByHash.java new file mode 100644 index 00000000000..01398868624 --- /dev/null +++ b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/privacy/methods/priv/AbstractPrivateTraceByHash.java @@ -0,0 +1,167 @@ +/* + * Copyright contributors to Hyperledger Besu. + * + * 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.privacy.methods.priv; + +import org.hyperledger.besu.datatypes.Hash; +import org.hyperledger.besu.ethereum.api.jsonrpc.internal.JsonRpcRequestContext; +import org.hyperledger.besu.ethereum.api.jsonrpc.internal.methods.JsonRpcMethod; +import org.hyperledger.besu.ethereum.api.jsonrpc.internal.privacy.methods.PrivacyIdProvider; +import org.hyperledger.besu.ethereum.api.jsonrpc.internal.processor.privateProcessor.PrivateBlockTrace; +import org.hyperledger.besu.ethereum.api.jsonrpc.internal.processor.privateProcessor.PrivateBlockTracer; +import org.hyperledger.besu.ethereum.api.jsonrpc.internal.processor.privateProcessor.PrivateTracer; +import org.hyperledger.besu.ethereum.api.jsonrpc.internal.processor.privateProcessor.PrivateTransactionTrace; +import org.hyperledger.besu.ethereum.api.jsonrpc.internal.results.privacy.privateTracing.PrivateFlatTrace; +import org.hyperledger.besu.ethereum.api.jsonrpc.internal.results.privacy.privateTracing.PrivateTraceGenerator; +import org.hyperledger.besu.ethereum.api.query.BlockchainQueries; +import org.hyperledger.besu.ethereum.api.query.PrivacyQueries; +import org.hyperledger.besu.ethereum.core.Block; +import org.hyperledger.besu.ethereum.core.PrivacyParameters; +import org.hyperledger.besu.ethereum.debug.TraceOptions; +import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule; +import org.hyperledger.besu.ethereum.privacy.ExecutedPrivateTransaction; +import org.hyperledger.besu.ethereum.privacy.MultiTenancyPrivacyController; +import org.hyperledger.besu.ethereum.privacy.PrivacyController; +import org.hyperledger.besu.ethereum.privacy.storage.PrivateBlockMetadata; +import org.hyperledger.besu.ethereum.vm.DebugOperationTracer; + +import java.util.Collections; +import java.util.Optional; +import java.util.function.Supplier; +import java.util.stream.Stream; + +public abstract class AbstractPrivateTraceByHash implements JsonRpcMethod { + + protected final Supplier blockTracerSupplier; + protected final BlockchainQueries blockchainQueries; + protected final PrivacyQueries privacyQueries; + protected final ProtocolSchedule protocolSchedule; + protected final PrivacyController privacyController; + protected final PrivacyParameters privacyParameters; + protected final PrivacyIdProvider privacyIdProvider; + + protected AbstractPrivateTraceByHash( + final Supplier blockTracerSupplier, + final BlockchainQueries blockchainQueries, + final PrivacyQueries privacyQueries, + final ProtocolSchedule protocolSchedule, + final PrivacyController privacyController, + final PrivacyParameters privacyParameters, + final PrivacyIdProvider privacyIdProvider) { + this.blockTracerSupplier = blockTracerSupplier; + this.blockchainQueries = blockchainQueries; + this.privacyQueries = privacyQueries; + this.protocolSchedule = protocolSchedule; + this.privacyController = privacyController; + this.privacyParameters = privacyParameters; + this.privacyIdProvider = privacyIdProvider; + } + + public Stream resultByTransactionHash( + final String privacyGroupId, + final Hash transactionHash, + final JsonRpcRequestContext requestContext) { + + final String enclaveKey = privacyIdProvider.getPrivacyUserId(requestContext.getUser()); + if (privacyController instanceof MultiTenancyPrivacyController) { + verifyPrivacyGroupMatchesAuthenticatedEnclaveKey( + requestContext, privacyGroupId, Optional.empty()); + } + + return privacyController + .findPrivateTransactionByPmtHash(transactionHash, enclaveKey) + .map(ExecutedPrivateTransaction::getBlockNumber) + .flatMap(blockNumber -> blockchainQueries.getBlockchain().getBlockHashByNumber(blockNumber)) + .map(blockHash -> getTraceBlock(blockHash, transactionHash, enclaveKey, privacyGroupId)) + .orElse(Stream.empty()); + } + + private Stream getTraceBlock( + final Hash blockHash, + final Hash transactionHash, + final String enclaveKey, + final String privacyGroupId) { + final Block block = blockchainQueries.getBlockchain().getBlockByHash(blockHash).orElse(null); + final PrivateBlockMetadata privateBlockMetadata = + privacyQueries.getPrivateBlockMetaData(privacyGroupId, blockHash).orElse(null); + + if (privateBlockMetadata == null || block == null) { + return Stream.empty(); + } + return PrivateTracer.processTracing( + blockchainQueries, + Optional.of(block.getHeader()), + privacyGroupId, + enclaveKey, + privacyParameters, + privacyController, + mutableWorldState -> { + final PrivateTransactionTrace privateTransactionTrace = + getTransactionTrace( + block, transactionHash, enclaveKey, privateBlockMetadata, privacyGroupId); + return Optional.ofNullable(getTraceStream(privateTransactionTrace, block)); + }) + .orElse(Stream.empty()); + } + + private PrivateTransactionTrace getTransactionTrace( + final Block block, + final Hash transactionHash, + final String enclaveKey, + final PrivateBlockMetadata privateBlockMetadata, + final String privacyGroupId) { + return PrivateTracer.processTracing( + blockchainQueries, + Optional.of(block.getHeader()), + privacyGroupId, + enclaveKey, + privacyParameters, + privacyController, + mutableWorldState -> + blockTracerSupplier + .get() + .trace( + mutableWorldState, + block, + new DebugOperationTracer(new TraceOptions(false, false, true), false), + enclaveKey, + privacyGroupId, + privateBlockMetadata) + .map(PrivateBlockTrace::getTransactionTraces) + .orElse(Collections.emptyList()) + .stream() + .filter( + trxTrace -> + trxTrace.getPrivateTransaction().getPmtHash().equals(transactionHash)) + .findFirst()) + .orElseThrow(); + } + + private Stream getTraceStream( + final PrivateTransactionTrace transactionTrace, final Block block) { + + return PrivateTraceGenerator.generateFromTransactionTraceAndBlock( + this.protocolSchedule, transactionTrace, block) + .map(PrivateFlatTrace.class::cast); + } + + private void verifyPrivacyGroupMatchesAuthenticatedEnclaveKey( + final JsonRpcRequestContext request, + final String privacyGroupId, + final Optional toBlock) { + final String privacyUserId = privacyIdProvider.getPrivacyUserId(request.getUser()); + privacyController.verifyPrivacyGroupContainsPrivacyUserId( + privacyGroupId, privacyUserId, toBlock); + } +} diff --git a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/privacy/methods/priv/PrivTraceTransaction.java b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/privacy/methods/priv/PrivTraceTransaction.java new file mode 100644 index 00000000000..b1802b07eb7 --- /dev/null +++ b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/privacy/methods/priv/PrivTraceTransaction.java @@ -0,0 +1,93 @@ +/* + * Copyright contributors to Hyperledger Besu. + * + * 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.privacy.methods.priv; + +import org.hyperledger.besu.datatypes.Hash; +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.methods.JsonRpcMethod; +import org.hyperledger.besu.ethereum.api.jsonrpc.internal.methods.TraceTransaction; +import org.hyperledger.besu.ethereum.api.jsonrpc.internal.privacy.methods.PrivacyIdProvider; +import org.hyperledger.besu.ethereum.api.jsonrpc.internal.processor.privateProcessor.PrivateBlockTracer; +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.privacy.privateTracing.PrivateFlatTrace; +import org.hyperledger.besu.ethereum.api.query.BlockchainQueries; +import org.hyperledger.besu.ethereum.api.query.PrivacyQueries; +import org.hyperledger.besu.ethereum.core.PrivacyParameters; +import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule; +import org.hyperledger.besu.ethereum.privacy.PrivacyController; + +import java.util.function.Supplier; +import java.util.stream.Stream; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ArrayNode; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class PrivTraceTransaction extends AbstractPrivateTraceByHash implements JsonRpcMethod { + private static final Logger LOG = LoggerFactory.getLogger(TraceTransaction.class); + + public PrivTraceTransaction( + final Supplier blockTracerSupplier, + final BlockchainQueries blockchainQueries, + final ProtocolSchedule protocolSchedule, + final PrivacyQueries privacyQueries, + final PrivacyController privacyController, + final PrivacyParameters privacyParameters, + final PrivacyIdProvider privacyIdProvider) { + super( + blockTracerSupplier, + blockchainQueries, + privacyQueries, + protocolSchedule, + privacyController, + privacyParameters, + privacyIdProvider); + } + + @Override + public String getName() { + return RpcMethod.PRIV_TRACE_TRANSACTION.getMethodName(); + } + + @Override + public JsonRpcResponse response(final JsonRpcRequestContext requestContext) { + final String privacyGroupId = requestContext.getRequiredParameter(0, String.class); + final Hash transactionHash = requestContext.getRequiredParameter(1, Hash.class); + LOG.trace("Received RPC rpcName={} txHash={}", getName(), transactionHash); + + if (privacyGroupId.isEmpty() || transactionHash.isEmpty()) { + return new JsonRpcErrorResponse( + requestContext.getRequest().getId(), RpcErrorType.INVALID_PARAMS); + } + + return new JsonRpcSuccessResponse( + requestContext.getRequest().getId(), + arrayNodeFromTraceStream( + resultByTransactionHash(privacyGroupId, transactionHash, requestContext))); + } + + protected JsonNode arrayNodeFromTraceStream(final Stream traceStream) { + final ObjectMapper mapper = new ObjectMapper(); + final ArrayNode resultArrayNode = mapper.createArrayNode(); + traceStream.forEachOrdered(resultArrayNode::addPOJO); + return resultArrayNode; + } +} diff --git a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/processor/privateProcessor/PrivateBlockReplay.java b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/processor/privateProcessor/PrivateBlockReplay.java new file mode 100644 index 00000000000..de5c11940b5 --- /dev/null +++ b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/processor/privateProcessor/PrivateBlockReplay.java @@ -0,0 +1,110 @@ +/* + * Copyright contributors to Hyperledger Besu. + * + * 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.processor.privateProcessor; + +import org.hyperledger.besu.ethereum.chain.Blockchain; +import org.hyperledger.besu.ethereum.core.Block; +import org.hyperledger.besu.ethereum.core.BlockBody; +import org.hyperledger.besu.ethereum.core.BlockHeader; +import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule; +import org.hyperledger.besu.ethereum.mainnet.ProtocolSpec; +import org.hyperledger.besu.ethereum.privacy.ExecutedPrivateTransaction; +import org.hyperledger.besu.ethereum.privacy.PrivacyController; +import org.hyperledger.besu.ethereum.privacy.PrivateTransactionProcessor; +import org.hyperledger.besu.ethereum.privacy.storage.PrivateBlockMetadata; + +import java.util.List; +import java.util.Optional; + +public class PrivateBlockReplay { + + private final ProtocolSchedule protocolSchedule; + private final Blockchain blockchain; + private final PrivacyController privacyController; + + public PrivateBlockReplay( + final ProtocolSchedule protocolSchedule, + final Blockchain blockchain, + final PrivacyController privacyController) { + this.protocolSchedule = protocolSchedule; + this.blockchain = blockchain; + this.privacyController = privacyController; + } + + public Optional block( + final Block block, + final PrivateBlockMetadata privateBlockMetadata, + final String enclaveKey, + final TransactionAction action) { + return performActionWithBlock( + block.getHeader(), + block.getBody(), + (body, header, blockchain, transactionProcessor, protocolSpec) -> { + final List transactionTraces = + privateBlockMetadata.getPrivateTransactionMetadataList().stream() + .map( + privateTransactionMetadata -> + privacyController + .findPrivateTransactionByPmtHash( + privateTransactionMetadata.getPrivateMarkerTransactionHash(), + enclaveKey) + .map( + executedPrivateTransaction -> + action.performAction( + executedPrivateTransaction, + header, + blockchain, + transactionProcessor)) + .orElse(null)) + .toList(); + + return Optional.of(new PrivateBlockTrace(transactionTraces)); + }); + } + + private Optional performActionWithBlock( + final BlockHeader header, final BlockBody body, final BlockAction action) { + if (header == null) { + return Optional.empty(); + } + if (body == null) { + return Optional.empty(); + } + final ProtocolSpec protocolSpec = protocolSchedule.getByBlockHeader(header); + final PrivateTransactionProcessor transactionProcessor = + protocolSpec.getPrivateTransactionProcessor(); + + return action.perform(body, header, blockchain, transactionProcessor, protocolSpec); + } + + @FunctionalInterface + public interface BlockAction { + Optional perform( + BlockBody body, + BlockHeader blockHeader, + Blockchain blockchain, + PrivateTransactionProcessor transactionProcessor, + ProtocolSpec protocolSpec); + } + + @FunctionalInterface + public interface TransactionAction { + T performAction( + ExecutedPrivateTransaction transaction, + BlockHeader blockHeader, + Blockchain blockchain, + PrivateTransactionProcessor transactionProcessor); + } +} diff --git a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/processor/privateProcessor/PrivateBlockTrace.java b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/processor/privateProcessor/PrivateBlockTrace.java new file mode 100644 index 00000000000..4deebed2617 --- /dev/null +++ b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/processor/privateProcessor/PrivateBlockTrace.java @@ -0,0 +1,30 @@ +/* + * Copyright contributors to Hyperledger Besu. + * + * 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.processor.privateProcessor; + +import java.util.List; + +public class PrivateBlockTrace { + + private final List transactionTraces; + + public PrivateBlockTrace(final List transactionTraces) { + this.transactionTraces = transactionTraces; + } + + public List getTransactionTraces() { + return transactionTraces; + } +} diff --git a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/processor/privateProcessor/PrivateBlockTracer.java b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/processor/privateProcessor/PrivateBlockTracer.java new file mode 100644 index 00000000000..c58af8fc3fc --- /dev/null +++ b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/processor/privateProcessor/PrivateBlockTracer.java @@ -0,0 +1,87 @@ +/* + * Copyright contributors to Hyperledger Besu. + * + * 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.processor.privateProcessor; + +import org.hyperledger.besu.ethereum.core.Block; +import org.hyperledger.besu.ethereum.debug.TraceFrame; +import org.hyperledger.besu.ethereum.privacy.storage.PrivateBlockMetadata; +import org.hyperledger.besu.ethereum.processing.TransactionProcessingResult; +import org.hyperledger.besu.ethereum.vm.CachingBlockHashLookup; +import org.hyperledger.besu.ethereum.vm.DebugOperationTracer; +import org.hyperledger.besu.evm.worldstate.StackedUpdater; +import org.hyperledger.besu.evm.worldstate.WorldUpdater; + +import java.util.List; +import java.util.Optional; + +import org.apache.tuweni.bytes.Bytes; +import org.apache.tuweni.bytes.Bytes32; + +public class PrivateBlockTracer { + private final PrivateBlockReplay blockReplay; + // Either the initial block state or the state of the prior TX, including miner rewards. + private WorldUpdater chainedUpdater; + + public PrivateBlockTracer(final PrivateBlockReplay blockReplay) { + this.blockReplay = blockReplay; + } + + public Optional trace( + final PrivateTracer.TraceableState mutableWorldState, + final Block block, + final DebugOperationTracer tracer, + final String enclaveKey, + final String privacyGroupId, + final PrivateBlockMetadata privateBlockMetadata) { + return blockReplay.block( + block, + privateBlockMetadata, + enclaveKey, + prepareReplayAction(mutableWorldState, tracer, privacyGroupId)); + } + + private PrivateBlockReplay.TransactionAction prepareReplayAction( + final PrivateTracer.TraceableState mutableWorldState, + final DebugOperationTracer tracer, + final String privacyGroupId) { + return (transaction, header, blockchain, transactionProcessor) -> { + // if we have no prior updater, it must be the first TX, so use the block's initial state + if (chainedUpdater == null) { + chainedUpdater = mutableWorldState.updater(); + + } else if (chainedUpdater instanceof StackedUpdater stackedUpdater) { + stackedUpdater.markTransactionBoundary(); + } + // create an updater for just this tx + chainedUpdater = chainedUpdater.updater(); + WorldUpdater privateChainedUpdater = mutableWorldState.privateUpdater(); + final TransactionProcessingResult result = + transactionProcessor.processTransaction( + chainedUpdater, + privateChainedUpdater, + header, + transaction.getPmtHash(), + transaction, + header.getCoinbase(), + tracer, + new CachingBlockHashLookup(header, blockchain), + Bytes32.wrap(Bytes.fromBase64String(privacyGroupId))); + + final List traceFrames = tracer.copyTraceFrames(); + tracer.reset(); + return new PrivateTransactionTrace(transaction, result, traceFrames); + }; + } +} diff --git a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/processor/privateProcessor/PrivateTracer.java b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/processor/privateProcessor/PrivateTracer.java new file mode 100644 index 00000000000..c79feead167 --- /dev/null +++ b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/processor/privateProcessor/PrivateTracer.java @@ -0,0 +1,118 @@ +/* + * Copyright contributors to Hyperledger Besu. + * + * 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.processor.privateProcessor; + +import org.hyperledger.besu.datatypes.Address; +import org.hyperledger.besu.datatypes.Hash; +import org.hyperledger.besu.ethereum.api.query.BlockchainQueries; +import org.hyperledger.besu.ethereum.core.BlockHeader; +import org.hyperledger.besu.ethereum.core.MutableWorldState; +import org.hyperledger.besu.ethereum.core.PrivacyParameters; +import org.hyperledger.besu.ethereum.privacy.PrivacyController; +import org.hyperledger.besu.evm.account.Account; +import org.hyperledger.besu.evm.worldstate.WorldUpdater; + +import java.util.Optional; +import java.util.function.Function; +import java.util.stream.Stream; + +import org.apache.tuweni.bytes.Bytes32; + +public class PrivateTracer { + + public static Optional processTracing( + final BlockchainQueries blockchainQueries, + final Optional blockHeader, + final String privacyGroupId, + final String enclaveKey, + final PrivacyParameters privacyParameters, + final PrivacyController privacyController, + final Function> mapper) { + + return blockHeader.flatMap( + header -> { + final long blockNumber = header.getNumber(); + final Hash parentHash = header.getParentHash(); + + final MutableWorldState disposablePrivateState = + privacyParameters + .getPrivateWorldStateArchive() + .getMutable( + privacyController + .getStateRootByBlockNumber(privacyGroupId, enclaveKey, blockNumber) + .get(), + parentHash) + .get(); + + return blockchainQueries.getAndMapWorldState( + parentHash, + mutableWorldState -> + mapper.apply( + new PrivateTracer.TraceableState(mutableWorldState, disposablePrivateState))); + }); + } + + /** + * This class force the use of the processTracing method to do tracing. processTracing allows you + * to cleanly manage the worldstate, to close it etc + */ + public static class TraceableState implements MutableWorldState { + private final MutableWorldState mutableWorldState; + private final MutableWorldState disposableWorldState; + + private TraceableState( + final MutableWorldState mutableWorldState, final MutableWorldState disposableWorldState) { + this.mutableWorldState = mutableWorldState; + this.disposableWorldState = disposableWorldState; + } + + @Override + public WorldUpdater updater() { + return mutableWorldState.updater(); + } + + public WorldUpdater privateUpdater() { + return disposableWorldState.updater(); + } + + @Override + public Hash rootHash() { + return mutableWorldState.rootHash(); + } + + @Override + public Hash frontierRootHash() { + return mutableWorldState.rootHash(); + } + + @Override + public Stream streamAccounts(final Bytes32 startKeyHash, final int limit) { + return mutableWorldState.streamAccounts(startKeyHash, limit); + } + + @Override + public Account get(final Address address) { + return mutableWorldState.get(address); + } + + @Override + public void close() throws Exception { + mutableWorldState.close(); + } + + @Override + public void persist(final BlockHeader blockHeader) {} + } +} diff --git a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/processor/privateProcessor/PrivateTransactionTrace.java b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/processor/privateProcessor/PrivateTransactionTrace.java new file mode 100644 index 00000000000..45e41d068b6 --- /dev/null +++ b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/processor/privateProcessor/PrivateTransactionTrace.java @@ -0,0 +1,91 @@ +/* + * Copyright contributors to Hyperledger Besu. + * + * 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.processor.privateProcessor; + +import org.hyperledger.besu.ethereum.core.Block; +import org.hyperledger.besu.ethereum.debug.TraceFrame; +import org.hyperledger.besu.ethereum.privacy.ExecutedPrivateTransaction; +import org.hyperledger.besu.ethereum.processing.TransactionProcessingResult; + +import java.util.List; +import java.util.Optional; + +public class PrivateTransactionTrace { + + private final ExecutedPrivateTransaction privateTransaction; + private final TransactionProcessingResult result; + private final List traceFrames; + private final Optional block; + + public PrivateTransactionTrace(final Optional block) { + this.privateTransaction = null; + this.result = null; + this.traceFrames = null; + this.block = block; + } + + public PrivateTransactionTrace( + final ExecutedPrivateTransaction privateTransaction, + final TransactionProcessingResult result, + final List traceFrames) { + this.privateTransaction = privateTransaction; + this.result = result; + this.traceFrames = traceFrames; + this.block = Optional.empty(); + } + + public PrivateTransactionTrace( + final ExecutedPrivateTransaction privateTransaction, + final TransactionProcessingResult result, + final List traceFrames, + final Optional block) { + this.privateTransaction = privateTransaction; + this.result = result; + this.traceFrames = traceFrames; + this.block = block; + } + + public PrivateTransactionTrace( + final ExecutedPrivateTransaction privateTransaction, final Optional block) { + this.privateTransaction = privateTransaction; + this.result = null; + this.traceFrames = null; + this.block = block; + } + + public ExecutedPrivateTransaction getPrivateTransaction() { + return privateTransaction; + } + + public long getGas() { + return privateTransaction.getGasLimit() - result.getGasRemaining(); + } + + public long getGasLimit() { + return privateTransaction.getGasLimit(); + } + + public TransactionProcessingResult getResult() { + return result; + } + + public List getTraceFrames() { + return traceFrames; + } + + public Optional getBlock() { + return block; + } +} diff --git a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/results/privacy/privateTracing/PrivateFlatTrace.java b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/results/privacy/privateTracing/PrivateFlatTrace.java new file mode 100644 index 00000000000..805e1c5ca37 --- /dev/null +++ b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/results/privacy/privateTracing/PrivateFlatTrace.java @@ -0,0 +1,377 @@ +/* + * Copyright contributors to Hyperledger Besu. + * + * 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.results.privacy.privateTracing; + +import static com.fasterxml.jackson.annotation.JsonInclude.Include.NON_NULL; + +import org.hyperledger.besu.ethereum.api.jsonrpc.internal.processor.privateProcessor.PrivateTransactionTrace; +import org.hyperledger.besu.ethereum.api.jsonrpc.internal.results.Quantity; +import org.hyperledger.besu.ethereum.api.jsonrpc.internal.results.tracing.Trace; +import org.hyperledger.besu.ethereum.api.jsonrpc.internal.results.tracing.flat.Action; +import org.hyperledger.besu.ethereum.api.jsonrpc.internal.results.tracing.flat.Result; +import org.hyperledger.besu.ethereum.debug.TraceFrame; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicReference; + +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; + +@JsonPropertyOrder({ + "action", + "blockHash", + "blockNumber", + "result", + "error", + "revertReason", + "subtraces", + "traceAddress", + "transactionHash", + "transactionPosition", + "type" +}) +public class PrivateFlatTrace implements Trace { + + private final Action action; + private final Result result; + private final Long blockNumber; + private final String blockHash; + private final Integer transactionPosition; + private final String transactionHash; + private final Optional error; + private final String revertReason; + private final int subtraces; + private final List traceAddress; + private final String type; + + protected PrivateFlatTrace( + final Action.Builder actionBuilder, + final Result.Builder resultBuilder, + final int subtraces, + final List traceAddress, + final String type, + final Long blockNumber, + final String blockHash, + final Integer transactionPosition, + final String transactionHash, + final Optional error) { + this( + actionBuilder != null ? actionBuilder.build() : null, + resultBuilder != null ? resultBuilder.build() : null, + subtraces, + traceAddress, + type, + blockNumber, + blockHash, + transactionPosition, + transactionHash, + error, + null); + } + + protected PrivateFlatTrace( + final Action.Builder actionBuilder, + final Result.Builder resultBuilder, + final int subtraces, + final List traceAddress, + final String type, + final Long blockNumber, + final String blockHash, + final Integer transactionPosition, + final String transactionHash, + final Optional error, + final String revertReason) { + this( + actionBuilder != null ? actionBuilder.build() : null, + resultBuilder != null ? resultBuilder.build() : null, + subtraces, + traceAddress, + type, + blockNumber, + blockHash, + transactionPosition, + transactionHash, + error, + revertReason); + } + + protected PrivateFlatTrace( + final Action action, + final Result result, + final int subtraces, + final List traceAddress, + final String type, + final Long blockNumber, + final String blockHash, + final Integer transactionPosition, + final String transactionHash, + final Optional error, + final String revertReason) { + this.action = action; + this.result = result; + this.subtraces = subtraces; + this.traceAddress = traceAddress; + this.type = type; + this.blockNumber = blockNumber; + this.blockHash = blockHash; + this.transactionPosition = transactionPosition; + this.transactionHash = transactionHash; + this.error = error; + this.revertReason = revertReason; + } + + static PrivateFlatTrace.Builder freshBuilder(final PrivateTransactionTrace transactionTrace) { + return PrivateFlatTrace.builder() + .resultBuilder(Result.builder()) + .actionBuilder(from(transactionTrace)); + } + + public static Action.Builder from(final PrivateTransactionTrace trace) { + final Action.Builder builder = + Action.builder() + .from(trace.getPrivateTransaction().getSender().toHexString()) + .value(Quantity.create(trace.getPrivateTransaction().getValue())); + if (!trace.getTraceFrames().isEmpty()) { + final TraceFrame traceFrame = trace.getTraceFrames().get(0); + builder.gas( + "0x" + + Long.toHexString( + traceFrame.getGasRemaining() + (traceFrame.getPrecompiledGasCost().orElse(0L)))); + } + return builder; + } + + public Action getAction() { + return action; + } + + @JsonInclude(NON_NULL) + public Long getBlockNumber() { + return blockNumber; + } + + @JsonInclude(NON_NULL) + public String getBlockHash() { + return blockHash; + } + + @JsonInclude(NON_NULL) + public String getTransactionHash() { + return transactionHash; + } + + @JsonInclude(NON_NULL) + public Integer getTransactionPosition() { + return transactionPosition; + } + + @JsonInclude(NON_NULL) + public String getError() { + return error.orElse(null); + } + + @JsonInclude(NON_NULL) + public String getRevertReason() { + return revertReason; + } + + @JsonInclude(NON_NULL) + public AtomicReference getResult() { + return (error.isPresent()) ? null : new AtomicReference<>(result); + } + + public int getSubtraces() { + return subtraces; + } + + public List getTraceAddress() { + return traceAddress; + } + + public String getType() { + return type; + } + + public static PrivateFlatTrace.Builder builder() { + return new PrivateFlatTrace.Builder(); + } + + public static class Context { + + private final PrivateFlatTrace.Builder builder; + private long gasUsed = 0; + private boolean createOp; + + Context(final PrivateFlatTrace.Builder builder) { + this.builder = builder; + } + + public PrivateFlatTrace.Builder getBuilder() { + return builder; + } + + void incGasUsed(final long gas) { + setGasUsed(gasUsed + gas); + } + + void decGasUsed(final long gas) { + setGasUsed(gasUsed - gas); + } + + public long getGasUsed() { + return gasUsed; + } + + public void setGasUsed(final long gasUsed) { + this.gasUsed = gasUsed; + builder.getResultBuilder().gasUsed("0x" + Long.toHexString(gasUsed)); + } + + boolean isCreateOp() { + return createOp; + } + + void setCreateOp(final boolean createOp) { + this.createOp = createOp; + } + } + + public static class Builder { + + private Action.Builder actionBuilder; + private Result.Builder resultBuilder; + private int subtraces; + private List traceAddress = new ArrayList<>(); + private String type = "call"; + private Long blockNumber; + private String blockHash; + private String transactionHash; + private Integer transactionPosition; + private Optional error = Optional.empty(); + private String revertReason; + + protected Builder() {} + + PrivateFlatTrace.Builder resultBuilder(final Result.Builder resultBuilder) { + this.resultBuilder = resultBuilder; + return this; + } + + PrivateFlatTrace.Builder actionBuilder(final Action.Builder actionBuilder) { + this.actionBuilder = actionBuilder; + return this; + } + + public PrivateFlatTrace.Builder traceAddress(final List traceAddress) { + this.traceAddress = traceAddress; + return this; + } + + public PrivateFlatTrace.Builder type(final String type) { + this.type = type; + return this; + } + + public PrivateFlatTrace.Builder blockNumber(final Long blockNumber) { + this.blockNumber = blockNumber; + return this; + } + + public PrivateFlatTrace.Builder blockHash(final String blockHash) { + this.blockHash = blockHash; + return this; + } + + public PrivateFlatTrace.Builder transactionHash(final String transactionHash) { + this.transactionHash = transactionHash; + return this; + } + + public PrivateFlatTrace.Builder error(final Optional error) { + this.error = error; + return this; + } + + public PrivateFlatTrace.Builder revertReason(final String revertReason) { + this.revertReason = revertReason; + return this; + } + + public String getType() { + return type; + } + + public int getSubtraces() { + return subtraces; + } + + public List getTraceAddress() { + return traceAddress; + } + + public Long getBlockNumber() { + return blockNumber; + } + + public String getBlockHash() { + return blockHash; + } + + public String getTransactionHash() { + return transactionHash; + } + + public Integer getTransactionPosition() { + return transactionPosition; + } + + public Optional getError() { + return error; + } + + public String getRevertReason() { + return revertReason; + } + + void incSubTraces() { + this.subtraces++; + } + + public PrivateFlatTrace build() { + return new PrivateFlatTrace( + actionBuilder, + resultBuilder, + subtraces, + traceAddress, + type, + blockNumber, + blockHash, + transactionPosition, + transactionHash, + error, + revertReason); + } + + public Result.Builder getResultBuilder() { + return resultBuilder; + } + + public Action.Builder getActionBuilder() { + return actionBuilder; + } + } +} diff --git a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/results/privacy/privateTracing/PrivateTraceGenerator.java b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/results/privacy/privateTracing/PrivateTraceGenerator.java new file mode 100644 index 00000000000..3f78ffe592f --- /dev/null +++ b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/results/privacy/privateTracing/PrivateTraceGenerator.java @@ -0,0 +1,598 @@ +/* + * Copyright contributors to Hyperledger Besu. + * + * 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.results.privacy.privateTracing; + +import static org.hyperledger.besu.evm.internal.Words.toAddress; + +import org.hyperledger.besu.datatypes.Address; +import org.hyperledger.besu.datatypes.Wei; +import org.hyperledger.besu.ethereum.api.jsonrpc.internal.processor.privateProcessor.PrivateTransactionTrace; +import org.hyperledger.besu.ethereum.api.jsonrpc.internal.results.Quantity; +import org.hyperledger.besu.ethereum.api.jsonrpc.internal.results.tracing.Trace; +import org.hyperledger.besu.ethereum.api.jsonrpc.internal.results.tracing.TracingUtils; +import org.hyperledger.besu.ethereum.api.jsonrpc.internal.results.tracing.flat.Action; +import org.hyperledger.besu.ethereum.api.jsonrpc.internal.results.tracing.flat.Result; +import org.hyperledger.besu.ethereum.core.Block; +import org.hyperledger.besu.ethereum.debug.TraceFrame; +import org.hyperledger.besu.ethereum.mainnet.ProtocolSchedule; +import org.hyperledger.besu.ethereum.privacy.ExecutedPrivateTransaction; +import org.hyperledger.besu.evm.Code; +import org.hyperledger.besu.evm.frame.ExceptionalHaltReason; + +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Deque; +import java.util.Iterator; +import java.util.List; +import java.util.Locale; +import java.util.Optional; +import java.util.OptionalLong; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import com.google.common.collect.Streams; +import com.google.common.util.concurrent.Atomics; +import org.apache.tuweni.bytes.Bytes; + +public class PrivateTraceGenerator { + + private static final String ZERO_ADDRESS_STRING = Address.ZERO.toHexString(); + private static final int EIP_150_DIVISOR = 64; + + public static Stream generateFromTransactionTrace( + final ProtocolSchedule protocolSchedule, + final PrivateTransactionTrace transactionTrace, + final Block block, + final AtomicInteger traceCounter, + final Consumer consumer) { + + final PrivateFlatTrace.Builder firstFlatTraceBuilder = + PrivateFlatTrace.freshBuilder(transactionTrace); + + final ExecutedPrivateTransaction tx = transactionTrace.getPrivateTransaction(); + + final Optional smartContractCode = + tx.getInit().map(__ -> transactionTrace.getResult().getOutput().toString()); + final Optional smartContractAddress = + smartContractCode.map( + __ -> Address.contractAddress(tx.getSender(), tx.getNonce()).toHexString()); + final Optional revertReason = transactionTrace.getResult().getRevertReason(); + + // set code field in result node + smartContractCode.ifPresent(firstFlatTraceBuilder.getResultBuilder()::code); + revertReason.ifPresent(r -> firstFlatTraceBuilder.revertReason(r.toHexString())); + + // set init field if transaction is a smart contract deployment + tx.getInit().map(Bytes::toHexString).ifPresent(firstFlatTraceBuilder.getActionBuilder()::init); + + // set to, input and callType fields if not a smart contract + if (tx.getTo().isPresent()) { + final Bytes payload = tx.getPayload(); + firstFlatTraceBuilder + .getActionBuilder() + .to(tx.getTo().map(Bytes::toHexString).orElse(null)) + .callType("call") + .input(payload == null ? "0x" : payload.toHexString()); + + if (!transactionTrace.getTraceFrames().isEmpty() + && hasRevertInSubCall(transactionTrace, transactionTrace.getTraceFrames().get(0))) { + firstFlatTraceBuilder.error(Optional.of("Reverted")); + } + + } else { + firstFlatTraceBuilder + .type("create") + .getResultBuilder() + .address(smartContractAddress.orElse(null)); + } + + if (!transactionTrace.getTraceFrames().isEmpty()) { + final OptionalLong precompiledGasCost = + transactionTrace.getTraceFrames().get(0).getPrecompiledGasCost(); + if (precompiledGasCost.isPresent()) { + firstFlatTraceBuilder + .getResultBuilder() + .gasUsed("0x" + Long.toHexString(precompiledGasCost.getAsLong())); + } + } + + final List flatTraces = new ArrayList<>(); + + // stack of previous contexts + final Deque tracesContexts = new ArrayDeque<>(); + + // add the first transactionTrace context to the queue of transactionTrace contexts + PrivateFlatTrace.Context currentContext = new PrivateFlatTrace.Context(firstFlatTraceBuilder); + tracesContexts.addLast(currentContext); + flatTraces.add(currentContext.getBuilder()); + // declare the first transactionTrace context as the previous transactionTrace context + long cumulativeGasCost = 0; + + final Iterator iter = transactionTrace.getTraceFrames().iterator(); + Optional nextTraceFrame = + iter.hasNext() ? Optional.of(iter.next()) : Optional.empty(); + while (nextTraceFrame.isPresent()) { + final TraceFrame traceFrame = nextTraceFrame.get(); + nextTraceFrame = iter.hasNext() ? Optional.of(iter.next()) : Optional.empty(); + cumulativeGasCost += + traceFrame.getGasCost().orElse(0L) + traceFrame.getPrecompiledGasCost().orElse(0L); + + final String opcodeString = traceFrame.getOpcode(); + if ("CALL".equals(opcodeString) + || "CALLCODE".equals(opcodeString) + || "DELEGATECALL".equals(opcodeString) + || "STATICCALL".equals(opcodeString)) { + + currentContext = + handleCall( + transactionTrace, + traceFrame, + nextTraceFrame, + flatTraces, + cumulativeGasCost, + tracesContexts, + opcodeString.toLowerCase(Locale.US)); + + } else if ("CALLDATALOAD".equals(opcodeString)) { + currentContext = handleCallDataLoad(currentContext, traceFrame); + } else if ("RETURN".equals(opcodeString) || "STOP".equals(opcodeString)) { + currentContext = + handleReturn( + protocolSchedule, + transactionTrace, + block, + traceFrame, + tracesContexts, + currentContext); + } else if ("SELFDESTRUCT".equals(opcodeString)) { + if (traceFrame.getExceptionalHaltReason().isPresent()) { + currentContext = + handleCall( + transactionTrace, + traceFrame, + nextTraceFrame, + flatTraces, + cumulativeGasCost, + tracesContexts, + opcodeString.toLowerCase(Locale.US)); + } else { + currentContext = + handleSelfDestruct(traceFrame, tracesContexts, currentContext, flatTraces); + } + } else if (("CREATE".equals(opcodeString) || "CREATE2".equals(opcodeString)) + && (traceFrame.getExceptionalHaltReason().isEmpty() || traceFrame.getDepth() == 0)) { + currentContext = + handleCreateOperation( + traceFrame, + nextTraceFrame, + flatTraces, + cumulativeGasCost, + tracesContexts, + smartContractAddress); + } else if ("REVERT".equals(opcodeString)) { + currentContext = handleRevert(tracesContexts, currentContext); + } + + if (traceFrame.getExceptionalHaltReason().isPresent()) { + currentContext = handleHalt(flatTraces, tracesContexts, currentContext, traceFrame); + } + + if (currentContext == null) { + break; + } + } + + return flatTraces.stream().peek(consumer).map(PrivateFlatTrace.Builder::build); + } + + public static Stream generateFromTransactionTraceAndBlock( + final ProtocolSchedule protocolSchedule, + final PrivateTransactionTrace transactionTrace, + final Block block) { + return generateFromTransactionTrace( + protocolSchedule, + transactionTrace, + block, + new AtomicInteger(), + builder -> + addAdditionalTransactionInformationToFlatTrace(builder, transactionTrace, block)); + } + + private static long computeGasUsed( + final Deque tracesContexts, + final PrivateFlatTrace.Context currentContext, + final PrivateTransactionTrace transactionTrace, + final TraceFrame traceFrame) { + + final long gasRemainingBeforeProcessed; + final long gasRemainingAfterProcessed; + long gasRefund = 0; + if (tracesContexts.size() == 1) { + gasRemainingBeforeProcessed = transactionTrace.getTraceFrames().get(0).getGasRemaining(); + gasRemainingAfterProcessed = transactionTrace.getResult().getGasRemaining(); + if (gasRemainingAfterProcessed > traceFrame.getGasRemaining()) { + gasRefund = gasRemainingAfterProcessed - traceFrame.getGasRemaining(); + } else { + gasRefund = traceFrame.getGasRefund(); + } + } else { + final Action.Builder actionBuilder = currentContext.getBuilder().getActionBuilder(); + gasRemainingBeforeProcessed = Long.decode(actionBuilder.getGas()); + gasRemainingAfterProcessed = traceFrame.getGasRemaining(); + } + return gasRemainingBeforeProcessed - gasRemainingAfterProcessed + gasRefund; + } + + private static long computeGas( + final TraceFrame traceFrame, final Optional nextTraceFrame) { + if (traceFrame.getGasCost().isPresent()) { + final long gasNeeded = traceFrame.getGasCost().getAsLong(); + final long currentGas = traceFrame.getGasRemaining(); + if (currentGas >= gasNeeded) { + final long gasRemaining = currentGas - gasNeeded; + return gasRemaining - Math.floorDiv(gasRemaining, EIP_150_DIVISOR); + } + } + return nextTraceFrame.map(TraceFrame::getGasRemaining).orElse(0L); + } + + private static String calculateCallingAddress(final PrivateFlatTrace.Context lastContext) { + final PrivateFlatTrace.Builder lastContextBuilder = lastContext.getBuilder(); + final Action.Builder lastActionBuilder = lastContextBuilder.getActionBuilder(); + if (lastActionBuilder.getCallType() == null) { + if ("create".equals(lastContextBuilder.getType())) { + return lastContextBuilder.getResultBuilder().getAddress(); + } else { + return ZERO_ADDRESS_STRING; + } + } + + switch (lastActionBuilder.getCallType()) { + case "call": + case "staticcall": + return lastActionBuilder.getTo(); + case "delegatecall": + case "callcode": + return lastActionBuilder.getFrom(); + case "create": + case "create2": + return lastContextBuilder.getResultBuilder().getAddress(); + default: + return ZERO_ADDRESS_STRING; + } + } + + private static List calculateTraceAddress( + final Deque contexts) { + return contexts.stream() + .map(context -> context.getBuilder().getSubtraces()) + .collect(Collectors.toList()); + } + + private static List calculateSelfDescructAddress( + final Deque contexts) { + return Streams.concat( + contexts.stream() + .map(context -> context.getBuilder().getSubtraces())) // , Stream.of(0)) + .collect(Collectors.toList()); + } + + private static String getActionAddress( + final Action.Builder callingAction, final String recipient) { + if (callingAction.getCallType() != null) { + return callingAction.getCallType().equals("call") + ? callingAction.getTo() + : callingAction.getFrom(); + } + return firstNonNull("", recipient, callingAction.getFrom(), callingAction.getTo()); + } + + private static String firstNonNull(final String defaultValue, final String... values) { + for (final String value : values) { + if (value != null) { + return value; + } + } + return defaultValue; + } + + private static PrivateFlatTrace.Context handleCreateOperation( + final TraceFrame traceFrame, + final Optional nextTraceFrame, + final List flatTraces, + final long cumulativeGasCost, + final Deque tracesContexts, + final Optional smartContractAddress) { + final PrivateFlatTrace.Context lastContext = tracesContexts.peekLast(); + + final String callingAddress = calculateCallingAddress(lastContext); + + final PrivateFlatTrace.Builder subTraceBuilder = + PrivateFlatTrace.builder() + .type("create") + .traceAddress(calculateTraceAddress(tracesContexts)) + .resultBuilder(Result.builder()); + + final Action.Builder subTraceActionBuilder = + Action.builder() + .from(smartContractAddress.orElse(callingAddress)) + .gas("0x" + Long.toHexString(computeGas(traceFrame, nextTraceFrame))) + .value(Quantity.create(nextTraceFrame.map(TraceFrame::getValue).orElse(Wei.ZERO))); + + traceFrame + .getMaybeCode() + .map(Code::getBytes) + .map(Bytes::toHexString) + .ifPresent(subTraceActionBuilder::init); + + final PrivateFlatTrace.Context currentContext = + new PrivateFlatTrace.Context(subTraceBuilder.actionBuilder(subTraceActionBuilder)); + + currentContext + .getBuilder() + .getResultBuilder() + .address(nextTraceFrame.map(TraceFrame::getRecipient).orElse(Address.ZERO).toHexString()); + currentContext.setCreateOp(true); + currentContext.decGasUsed(cumulativeGasCost); + tracesContexts.addLast(currentContext); + flatTraces.add(currentContext.getBuilder()); + return currentContext; + } + + private static PrivateFlatTrace.Context handleHalt( + final List flatTraces, + final Deque tracesContexts, + final PrivateFlatTrace.Context currentContext, + final TraceFrame traceFrame) { + final PrivateFlatTrace.Builder traceFrameBuilder; + if (currentContext == null) { + traceFrameBuilder = flatTraces.get(flatTraces.size() - 1); + } else { + traceFrameBuilder = currentContext.getBuilder(); + } + traceFrameBuilder.error( + traceFrame.getExceptionalHaltReason().map(ExceptionalHaltReason::getDescription)); + if (currentContext != null) { + final Action.Builder actionBuilder = traceFrameBuilder.getActionBuilder(); + actionBuilder.value(Quantity.create(traceFrame.getValue())); + tracesContexts.removeLast(); + final PrivateFlatTrace.Context nextContext = tracesContexts.peekLast(); + if (nextContext != null) { + nextContext.getBuilder().incSubTraces(); + } + return nextContext; + } + return currentContext; + } + + private static PrivateFlatTrace.Context handleRevert( + final Deque tracesContexts, + final PrivateFlatTrace.Context currentContext) { + currentContext.getBuilder().error(Optional.of("Reverted")); + tracesContexts.removeLast(); + final PrivateFlatTrace.Context nextContext = tracesContexts.peekLast(); + if (nextContext != null) { + nextContext.getBuilder().incSubTraces(); + } + return nextContext; + } + + private static PrivateFlatTrace.Context handleSelfDestruct( + final TraceFrame traceFrame, + final Deque tracesContexts, + final PrivateFlatTrace.Context currentContext, + final List flatTraces) { + + final Action.Builder actionBuilder = currentContext.getBuilder().getActionBuilder(); + final long gasUsed = + Long.decode(actionBuilder.getGas()) + - traceFrame.getGasRemaining() + + (traceFrame.getGasCost().orElse(0L)); + + currentContext.setGasUsed(gasUsed); + + final Bytes[] stack = traceFrame.getStack().orElseThrow(); + final Address refundAddress = toAddress(stack[stack.length - 1]); + final PrivateFlatTrace.Builder subTraceBuilder = + PrivateFlatTrace.builder() + .type("suicide") + .traceAddress(calculateSelfDescructAddress(tracesContexts)); + + final AtomicReference weiBalance = Atomics.newReference(Wei.ZERO); + traceFrame + .getMaybeRefunds() + .ifPresent(refunds -> weiBalance.set(refunds.getOrDefault(refundAddress, Wei.ZERO))); + + final Action.Builder callingAction = tracesContexts.peekLast().getBuilder().getActionBuilder(); + final String actionAddress = + getActionAddress(callingAction, traceFrame.getRecipient().toHexString()); + final Action.Builder subTraceActionBuilder = + Action.builder() + .address(actionAddress) + .refundAddress(refundAddress.toString()) + .balance(TracingUtils.weiAsHex(weiBalance.get())); + + flatTraces.add( + new PrivateFlatTrace.Context(subTraceBuilder.actionBuilder(subTraceActionBuilder)) + .getBuilder()); + final PrivateFlatTrace.Context lastContext = tracesContexts.removeLast(); + lastContext.getBuilder().incSubTraces(); + final PrivateFlatTrace.Context nextContext = tracesContexts.peekLast(); + if (nextContext != null) { + nextContext.getBuilder().incSubTraces(); + } + return nextContext; + } + + private static PrivateFlatTrace.Context handleReturn( + final ProtocolSchedule protocolSchedule, + final PrivateTransactionTrace transactionTrace, + final Block block, + final TraceFrame traceFrame, + final Deque tracesContexts, + final PrivateFlatTrace.Context currentContext) { + + final PrivateFlatTrace.Builder traceFrameBuilder = currentContext.getBuilder(); + final Result.Builder resultBuilder = traceFrameBuilder.getResultBuilder(); + final Action.Builder actionBuilder = traceFrameBuilder.getActionBuilder(); + actionBuilder.value(Quantity.create(traceFrame.getValue())); + + currentContext.setGasUsed( + computeGasUsed(tracesContexts, currentContext, transactionTrace, traceFrame)); + + if ("STOP".equals(traceFrame.getOpcode()) && resultBuilder.isGasUsedEmpty()) { + final long callStipend = + protocolSchedule + .getByBlockHeader(block.getHeader()) + .getGasCalculator() + .getAdditionalCallStipend(); + tracesContexts.stream() + .filter( + context -> + !tracesContexts.getFirst().equals(context) + && !tracesContexts.getLast().equals(context)) + .forEach(context -> context.decGasUsed(callStipend)); + } + + final Bytes outputData = traceFrame.getOutputData(); + if (resultBuilder.getCode() == null) { + resultBuilder.output(outputData.toHexString()); + } + + // set value for contract creation TXes, CREATE, and CREATE2 + if (actionBuilder.getCallType() == null && traceFrame.getMaybeCode().isPresent()) { + actionBuilder.init(traceFrame.getMaybeCode().get().getBytes().toHexString()); + resultBuilder.code(outputData.toHexString()); + if (currentContext.isCreateOp()) { + // this is from a CREATE/CREATE2, so add code deposit cost. + currentContext.incGasUsed(outputData.size() * 200L); + } + } + + tracesContexts.removeLast(); + final PrivateFlatTrace.Context nextContext = tracesContexts.peekLast(); + if (nextContext != null) { + nextContext.getBuilder().incSubTraces(); + } + return nextContext; + } + + private static PrivateFlatTrace.Context handleCallDataLoad( + final PrivateFlatTrace.Context currentContext, final TraceFrame traceFrame) { + if (!traceFrame.getValue().isZero()) { + currentContext + .getBuilder() + .getActionBuilder() + .value(traceFrame.getValue().toShortHexString()); + } else { + currentContext.getBuilder().getActionBuilder().value("0x0"); + } + return currentContext; + } + + private static PrivateFlatTrace.Context handleCall( + final PrivateTransactionTrace transactionTrace, + final TraceFrame traceFrame, + final Optional nextTraceFrame, + final List flatTraces, + final long cumulativeGasCost, + final Deque tracesContexts, + final String opcodeString) { + final Bytes[] stack = traceFrame.getStack().orElseThrow(); + final PrivateFlatTrace.Context lastContext = tracesContexts.peekLast(); + + final String callingAddress = calculateCallingAddress(lastContext); + + if (traceFrame.getDepth() >= nextTraceFrame.map(TraceFrame::getDepth).orElse(0)) { + // don't log calls to calls that don't execute, such as insufficient value and precompiles + return tracesContexts.peekLast(); + } + + final PrivateFlatTrace.Builder subTraceBuilder = + PrivateFlatTrace.builder() + .traceAddress(calculateTraceAddress(tracesContexts)) + .resultBuilder(Result.builder()); + final Action.Builder subTraceActionBuilder = + Action.builder() + .from(callingAddress) + .input( + nextTraceFrame.map(TraceFrame::getInputData).map(Bytes::toHexString).orElse(null)) + .gas( + "0x" + Long.toHexString(nextTraceFrame.map(TraceFrame::getGasRemaining).orElse(0L))) + .callType(opcodeString.toLowerCase(Locale.US)) + .value(Quantity.create(traceFrame.getValue())); + + if (stack.length > 1) { + subTraceActionBuilder.to(toAddress(stack[stack.length - 2]).toString()); + } + + nextTraceFrame.ifPresent( + nextFrame -> { + if (hasRevertInSubCall(transactionTrace, nextFrame)) { + subTraceBuilder.error(Optional.of("Reverted")); + } + }); + + final PrivateFlatTrace.Context currentContext = + new PrivateFlatTrace.Context(subTraceBuilder.actionBuilder(subTraceActionBuilder)); + currentContext.decGasUsed(cumulativeGasCost); + + tracesContexts.addLast(currentContext); + flatTraces.add(currentContext.getBuilder()); + return currentContext; + } + + private static boolean hasRevertInSubCall( + final PrivateTransactionTrace transactionTrace, final TraceFrame callFrame) { + return transactionTrace.getTraceFrames().stream() + .filter(traceFrame -> !traceFrame.equals(callFrame)) + .takeWhile(traceFrame -> !traceFrame.getOpcode().equals("RETURN")) + .filter(traceFrame -> traceFrame.getOpcode().equals("REVERT")) + .anyMatch(traceFrame -> traceFrame.getDepth() == callFrame.getDepth()); + } + + private static void addAdditionalTransactionInformationToFlatTrace( + final PrivateFlatTrace.Builder builder, + final PrivateTransactionTrace transactionTrace, + final Block block) { + // add block information (hash and number) + builder.blockHash(block.getHash().toHexString()).blockNumber(block.getHeader().getNumber()); + // add transaction information (position and hash) + builder.transactionHash(transactionTrace.getPrivateTransaction().getPmtHash().toHexString()); + + addContractCreationMethodToTrace(transactionTrace, builder); + } + + private static void addContractCreationMethodToTrace( + final PrivateTransactionTrace transactionTrace, final PrivateFlatTrace.Builder builder) { + // add creationMethod for create action + Optional.ofNullable(builder.getType()) + .filter(type -> type.equals("create")) + .ifPresent( + __ -> + builder + .getActionBuilder() + .creationMethod( + transactionTrace.getTraceFrames().stream() + .filter(frame -> "CREATE2".equals(frame.getOpcode())) + .findFirst() + .map(TraceFrame::getOpcode) + .orElse("CREATE") + .toLowerCase(Locale.US))); + } +} diff --git a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/results/tracing/flat/Action.java b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/results/tracing/flat/Action.java index 03bafbe859f..de7ae2c6f0e 100644 --- a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/results/tracing/flat/Action.java +++ b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/results/tracing/flat/Action.java @@ -263,7 +263,7 @@ public Builder balance(final String balance) { return this; } - Builder refundAddress(final String refundAddress) { + public Builder refundAddress(final String refundAddress) { this.refundAddress = refundAddress; return this; } diff --git a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/results/tracing/flat/FlatTraceGenerator.java b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/results/tracing/flat/FlatTraceGenerator.java index ad39330fa53..a253c6e323f 100644 --- a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/results/tracing/flat/FlatTraceGenerator.java +++ b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/internal/results/tracing/flat/FlatTraceGenerator.java @@ -624,7 +624,7 @@ private static List calculateSelfDescructAddress( .collect(Collectors.toList()); } - private static void addAdditionalTransactionInformationToFlatTrace( + protected static void addAdditionalTransactionInformationToFlatTrace( final FlatTrace.Builder builder, final TransactionTrace transactionTrace, final Block block) { // add block information (hash and number) builder.blockHash(block.getHash().toHexString()).blockNumber(block.getHeader().getNumber()); diff --git a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/methods/PrivJsonRpcMethods.java b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/methods/PrivJsonRpcMethods.java index 0f6df40d143..8ae62741b94 100644 --- a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/methods/PrivJsonRpcMethods.java +++ b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/jsonrpc/methods/PrivJsonRpcMethods.java @@ -34,6 +34,9 @@ import org.hyperledger.besu.ethereum.api.jsonrpc.internal.privacy.methods.priv.PrivGetTransactionCount; import org.hyperledger.besu.ethereum.api.jsonrpc.internal.privacy.methods.priv.PrivGetTransactionReceipt; import org.hyperledger.besu.ethereum.api.jsonrpc.internal.privacy.methods.priv.PrivNewFilter; +import org.hyperledger.besu.ethereum.api.jsonrpc.internal.privacy.methods.priv.PrivTraceTransaction; +import org.hyperledger.besu.ethereum.api.jsonrpc.internal.processor.privateProcessor.PrivateBlockReplay; +import org.hyperledger.besu.ethereum.api.jsonrpc.internal.processor.privateProcessor.PrivateBlockTracer; import org.hyperledger.besu.ethereum.api.query.BlockchainQueries; import org.hyperledger.besu.ethereum.core.PrivacyParameters; import org.hyperledger.besu.ethereum.eth.transactions.TransactionPool; @@ -68,6 +71,9 @@ protected Map create( final PrivacyIdProvider privacyIdProvider, final PrivateMarkerTransactionFactory privateMarkerTransactionFactory) { + final PrivateBlockReplay blockReplay = + new PrivateBlockReplay( + getProtocolSchedule(), getBlockchainQueries().getBlockchain(), privacyController); final Map RPC_METHODS = mapOf( new PrivCall(getBlockchainQueries(), privacyController, privacyIdProvider), @@ -89,7 +95,15 @@ protected Map create( new PrivGetFilterLogs(filterManager, privacyController, privacyIdProvider), new PrivGetFilterChanges(filterManager, privacyController, privacyIdProvider), new PrivNewFilter(filterManager, privacyController, privacyIdProvider), - new PrivUninstallFilter(filterManager, privacyController, privacyIdProvider)); + new PrivUninstallFilter(filterManager, privacyController, privacyIdProvider), + new PrivTraceTransaction( + () -> new PrivateBlockTracer(blockReplay), + getBlockchainQueries(), + getProtocolSchedule(), + getPrivacyQueries(), + privacyController, + getPrivacyParameters(), + privacyIdProvider)); if (!getPrivacyParameters().isFlexiblePrivacyGroupsEnabled()) { final Map OFFCHAIN_METHODS = diff --git a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/query/PrivacyQueries.java b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/query/PrivacyQueries.java index 604bad0c9bc..9768ead9dc5 100644 --- a/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/query/PrivacyQueries.java +++ b/ethereum/api/src/main/java/org/hyperledger/besu/ethereum/api/query/PrivacyQueries.java @@ -20,6 +20,7 @@ import org.hyperledger.besu.ethereum.core.LogWithMetadata; import org.hyperledger.besu.ethereum.privacy.PrivateTransactionReceipt; import org.hyperledger.besu.ethereum.privacy.PrivateWorldStateReader; +import org.hyperledger.besu.ethereum.privacy.storage.PrivateBlockMetadata; import org.hyperledger.besu.ethereum.privacy.storage.PrivateTransactionMetadata; import java.util.Collection; @@ -43,6 +44,11 @@ public PrivacyQueries( this.privateWorldStateReader = privateWorldStateReader; } + public Optional getPrivateBlockMetaData( + final String privacyGroupId, final Hash blockHash) { + return privateWorldStateReader.getPrivateBlockMetadata(privacyGroupId, blockHash); + } + public List matchingLogs( final String privacyGroupId, final long fromBlockNumber, diff --git a/ethereum/api/src/test/java/org/hyperledger/besu/ethereum/api/jsonrpc/methods/PrivJsonRpcMethodsTest.java b/ethereum/api/src/test/java/org/hyperledger/besu/ethereum/api/jsonrpc/methods/PrivJsonRpcMethodsTest.java index 249b2e7077e..b9024db9984 100644 --- a/ethereum/api/src/test/java/org/hyperledger/besu/ethereum/api/jsonrpc/methods/PrivJsonRpcMethodsTest.java +++ b/ethereum/api/src/test/java/org/hyperledger/besu/ethereum/api/jsonrpc/methods/PrivJsonRpcMethodsTest.java @@ -44,7 +44,6 @@ public class PrivJsonRpcMethodsTest { @Mock private TransactionPool transactionPool; @Mock private PrivacyParameters privacyParameters; @Mock private FilterManager filterManager; - private PrivJsonRpcMethods privJsonRpcMethods; @BeforeEach diff --git a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/privacy/PrivateTransaction.java b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/privacy/PrivateTransaction.java index c108d27626f..42672ddbf06 100644 --- a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/privacy/PrivateTransaction.java +++ b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/privacy/PrivateTransaction.java @@ -355,6 +355,16 @@ public Bytes getPayload() { return payload; } + /** + * Returns the payload if this is a contract creation transaction. + * + * @return if present the init code + */ + @Override + public Optional getInit() { + return getTo().isPresent() ? Optional.empty() : Optional.of(payload); + } + /** * Return the transaction chain id (if it exists) * diff --git a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/privacy/PrivateWorldStateReader.java b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/privacy/PrivateWorldStateReader.java index af32e76513c..255f80b5871 100644 --- a/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/privacy/PrivateWorldStateReader.java +++ b/ethereum/core/src/main/java/org/hyperledger/besu/ethereum/privacy/PrivateWorldStateReader.java @@ -66,6 +66,12 @@ public List getPrivateTransactionMetadataList( .orElse(Collections.emptyList()); } + public Optional getPrivateBlockMetadata( + final String privacyGroupId, final Hash blockHash) { + final Bytes32 privacyGroupIdBytes = Bytes32.wrap(Bytes.fromBase64String(privacyGroupId)); + return privateStateStorage.getPrivateBlockMetadata(blockHash, privacyGroupIdBytes); + } + public Optional getPrivateTransactionReceipt( final Hash blockHash, final Hash transactionHash) { return privateStateStorage.getTransactionReceipt(blockHash, transactionHash); diff --git a/plugin-api/build.gradle b/plugin-api/build.gradle index e49297ab7b3..fe7308993b8 100644 --- a/plugin-api/build.gradle +++ b/plugin-api/build.gradle @@ -70,7 +70,7 @@ Calculated : ${currentHash} tasks.register('checkAPIChanges', FileStateChecker) { description = "Checks that the API for the Plugin-API project does not change without deliberate thought" files = sourceSets.main.allJava.files - knownHash = 'I851CCOs00yYpW10qIGIak1bKbYhKFQkV2wyCYELHKY=' + knownHash = 'W1gv5UjqU+RJZJN6xPNjVfjuz7nKIcBgmh1j2XON4EU=' } check.dependsOn('checkAPIChanges') diff --git a/plugin-api/src/main/java/org/hyperledger/besu/plugin/data/PrivateTransaction.java b/plugin-api/src/main/java/org/hyperledger/besu/plugin/data/PrivateTransaction.java index 4ab171ecae9..fe918bf8f17 100644 --- a/plugin-api/src/main/java/org/hyperledger/besu/plugin/data/PrivateTransaction.java +++ b/plugin-api/src/main/java/org/hyperledger/besu/plugin/data/PrivateTransaction.java @@ -94,6 +94,16 @@ public interface PrivateTransaction { */ Address getSender(); + /** + * An unlimited size byte array specifying the EVM-code for the account initialization procedure. + * + *

Only present if this is a contract creation transaction, which is only true if {@link + * #getTo} is empty. + * + * @return if present, the contract init code. + */ + Optional getInit(); + /** * The chainId, computed from the 'V' portion of the signature. Used for replay protection. If * replay protection is not enabled this value will not be present.