From 5215ed0f5c5fba7bcf4aa1f568c70a0b4c7c0106 Mon Sep 17 00:00:00 2001 From: thinkAfCod Date: Fri, 4 Aug 2023 20:14:21 +0800 Subject: [PATCH] logic of BatcherSubmitter --- .../io/optimism/batcher/BatcherSubmitter.java | 100 ++++++++ .../io/optimism/batcher/channel/Channel.java | 49 ++++ .../batcher/channel/ChannelConfig.java | 42 ++++ .../optimism/batcher/channel/ChannelImpl.java | 29 +++ .../batcher/channel/ChannelManager.java | 96 ++++++++ .../io/optimism/batcher/config/Config.java | 41 ++++ .../optimism/batcher/ex/ReorgException.java | 54 ++++ .../batcher/ex/Web3jCallException.java | 38 +++ .../optimism/batcher/loader/BlockLoader.java | 232 ++++++++++++++++++ .../optimism/batcher/loader/LoaderConfig.java | 40 +++ .../publisher/ChannelDataPublisher.java | 168 +++++++++++++ .../batcher/publisher/PublisherConfig.java | 42 ++++ .../java/io/optimism/type/L1BlockRef.java | 12 + .../java/io/optimism/type/TxCandidate.java | 29 +++ .../java/io/optimism/rpc/RpcServerTest.java | 2 + .../optimism/utilities/gas/GasCalculator.java | 87 +++++++ 16 files changed, 1061 insertions(+) create mode 100644 hildr-batcher/src/main/java/io/optimism/batcher/BatcherSubmitter.java create mode 100644 hildr-batcher/src/main/java/io/optimism/batcher/channel/Channel.java create mode 100644 hildr-batcher/src/main/java/io/optimism/batcher/channel/ChannelConfig.java create mode 100644 hildr-batcher/src/main/java/io/optimism/batcher/channel/ChannelImpl.java create mode 100644 hildr-batcher/src/main/java/io/optimism/batcher/channel/ChannelManager.java create mode 100644 hildr-batcher/src/main/java/io/optimism/batcher/config/Config.java create mode 100644 hildr-batcher/src/main/java/io/optimism/batcher/ex/ReorgException.java create mode 100644 hildr-batcher/src/main/java/io/optimism/batcher/ex/Web3jCallException.java create mode 100644 hildr-batcher/src/main/java/io/optimism/batcher/loader/BlockLoader.java create mode 100644 hildr-batcher/src/main/java/io/optimism/batcher/loader/LoaderConfig.java create mode 100644 hildr-batcher/src/main/java/io/optimism/batcher/publisher/ChannelDataPublisher.java create mode 100644 hildr-batcher/src/main/java/io/optimism/batcher/publisher/PublisherConfig.java create mode 100644 hildr-batcher/src/main/java/io/optimism/type/TxCandidate.java create mode 100644 hildr-utilities/src/main/java/io/optimism/utilities/gas/GasCalculator.java diff --git a/hildr-batcher/src/main/java/io/optimism/batcher/BatcherSubmitter.java b/hildr-batcher/src/main/java/io/optimism/batcher/BatcherSubmitter.java new file mode 100644 index 00000000..72f8c1c2 --- /dev/null +++ b/hildr-batcher/src/main/java/io/optimism/batcher/BatcherSubmitter.java @@ -0,0 +1,100 @@ +/* + * Copyright 2023 q315xia@163.com + * + * 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. + */ + +package io.optimism.batcher; + +import com.google.common.util.concurrent.AbstractExecutionThreadService; +import io.optimism.batcher.channel.ChannelManager; +import io.optimism.batcher.config.Config; +import io.optimism.batcher.loader.BlockLoader; +import io.optimism.batcher.loader.LoaderConfig; +import io.optimism.batcher.publisher.ChannelDataPublisher; +import io.optimism.batcher.publisher.PublisherConfig; +import io.optimism.type.L1BlockRef; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.web3j.protocol.core.methods.response.TransactionReceipt; + +/** + * BatcherSubmitter class. + * + * @author thinkAfCod + * @since 0.1.1 + */ +@SuppressWarnings("UnusedVariable") +public class BatcherSubmitter extends AbstractExecutionThreadService { + + private static final Logger LOGGER = LoggerFactory.getLogger(BatcherSubmitter.class); + + private final Config config; + + private final ChannelManager channelManager; + private final BlockLoader blockLoader; + private final ChannelDataPublisher channelPublisher; + + private boolean isShutdownTriggered = false; + + private L1BlockRef lastL1Tip; + + /** + * Constructor of BatcherSubmitter. + * + * @param config BatcherSubmitter config + */ + public BatcherSubmitter(Config config) { + this.config = config; + this.channelManager = new ChannelManager(); + this.blockLoader = new BlockLoader(LoaderConfig.from(config), this.channelManager::addL2Block); + + this.channelPublisher = + new ChannelDataPublisher( + PublisherConfig.from(config, this.blockLoader.getRollConfig().batchInboxAddress()), + this.channelManager::txData, + this::handleReceipt); + } + + private void trySubmitBatchData() { + this.blockLoader.loadBlock(); + // If no data has been sent, then sleep for a period of time. + if (!this.channelPublisher.publishPendingBlock()) { + try { + Thread.sleep(config.pollInterval()); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + } + + private void handleReceipt(TransactionReceipt receipt) { + if (receipt.isStatusOK()) { + // todo this.channelManager.txConfirmed(); + } else { + // todo this.channelManager.txFailed(); + } + } + + @Override + protected void run() throws Exception { + while (isRunning() && !this.isShutdownTriggered) { + this.trySubmitBatchData(); + } + } + + @Override + protected void triggerShutdown() { + this.isShutdownTriggered = true; + } +} diff --git a/hildr-batcher/src/main/java/io/optimism/batcher/channel/Channel.java b/hildr-batcher/src/main/java/io/optimism/batcher/channel/Channel.java new file mode 100644 index 00000000..549c413f --- /dev/null +++ b/hildr-batcher/src/main/java/io/optimism/batcher/channel/Channel.java @@ -0,0 +1,49 @@ +/* + * Copyright 2023 q315xia@163.com + * + * 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. + */ + +package io.optimism.batcher.channel; + +import org.apache.commons.lang3.ArrayUtils; + +/** + * Channel interface. cache batch submit data. + * + * @author thinkAfCod + * @since 0.1.1 + */ +public interface Channel { + + /** Derivation version. */ + byte DERIVATION_VERSION_0 = 0; + + /** + * Channel Tx Data class. + * + * @param data L2 block data that will send to L1 + * @param channelId channelId + * @param frameNumber channel frame number + */ + record TxData(byte[] data, byte[] channelId, int frameNumber) { + /** + * Get tx bytes. + * + * @return tx bytes + */ + public byte[] txBytes() { + return ArrayUtils.addAll(new byte[] {DERIVATION_VERSION_0}, data()); + } + } +} diff --git a/hildr-batcher/src/main/java/io/optimism/batcher/channel/ChannelConfig.java b/hildr-batcher/src/main/java/io/optimism/batcher/channel/ChannelConfig.java new file mode 100644 index 00000000..d866e0e7 --- /dev/null +++ b/hildr-batcher/src/main/java/io/optimism/batcher/channel/ChannelConfig.java @@ -0,0 +1,42 @@ +/* + * Copyright 2023 q315xia@163.com + * + * 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. + */ + +package io.optimism.batcher.channel; + +import io.optimism.batcher.config.Config; + +/** + * ChannelConfig class. + * + * @param channelTimeout The maximum number of L1 blocks that the inclusion transactions of a + * channel's frames can span. + * @param maxChannelDuration If 0, duration checks are disabled. + * @param maxFrameSize The maximum byte-size a frame can have. + * @author thinkAfCod + * @since 0.1.1 + */ +public record ChannelConfig(long channelTimeout, long maxChannelDuration, int maxFrameSize) { + + /** + * Create a ChannelConfig instance from Config instance. + * + * @param config Config instance + * @return ChannelConfig instance + */ + public static ChannelConfig from(Config config) { + return new ChannelConfig(30000, 0, 120_000); + } +} diff --git a/hildr-batcher/src/main/java/io/optimism/batcher/channel/ChannelImpl.java b/hildr-batcher/src/main/java/io/optimism/batcher/channel/ChannelImpl.java new file mode 100644 index 00000000..3425b20d --- /dev/null +++ b/hildr-batcher/src/main/java/io/optimism/batcher/channel/ChannelImpl.java @@ -0,0 +1,29 @@ +/* + * Copyright 2023 q315xia@163.com + * + * 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. + */ + +package io.optimism.batcher.channel; + +/** + * Channel class.Record the batcher data of block transaction and process this data with framing. + * + * @author thinkAfCod + * @since 0.1.1 + */ +public class ChannelImpl implements Channel { + + /** Constructor of ChannelImpl. */ + public ChannelImpl() {} +} diff --git a/hildr-batcher/src/main/java/io/optimism/batcher/channel/ChannelManager.java b/hildr-batcher/src/main/java/io/optimism/batcher/channel/ChannelManager.java new file mode 100644 index 00000000..4323b556 --- /dev/null +++ b/hildr-batcher/src/main/java/io/optimism/batcher/channel/ChannelManager.java @@ -0,0 +1,96 @@ +/* + * Copyright 2023 q315xia@163.com + * + * 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. + */ + +package io.optimism.batcher.channel; + +import io.optimism.batcher.ex.ReorgException; +import io.optimism.type.BlockId; +import org.apache.commons.lang3.StringUtils; +import org.jctools.queues.MessagePassingQueue; +import org.jctools.queues.SpscArrayQueue; +import org.web3j.protocol.core.methods.response.EthBlock; + +/** + * ChannelManager class. create and remove channel object. + * + * @author thinkAfCod + * @since 0.1.1 + */ +public class ChannelManager { + + private MessagePassingQueue blocks; + + private String latestBlockHash; + + /** Constructor of ChannelManager. */ + public ChannelManager() { + this.blocks = new SpscArrayQueue<>(Integer.MAX_VALUE); + } + + /** + * Adds an L2 block to the internal blocks queue. It throws ReorgException if the block does not + * extend the last block loaded into the state. If no blocks were added yet, the parent hash check + * is skipped. + * + * @param block L2 block data + */ + public void addL2Block(EthBlock.Block block) { + if (!StringUtils.isEmpty(latestBlockHash) && !latestBlockHash.equals(block.getParentHash())) { + throw new ReorgException("block does not extend existing chain"); + } + // todo metrics pending block + boolean success = this.blocks.offer(block); + if (!success) { + throw new RuntimeException(""); + } + this.latestBlockHash = block.getHash(); + } + + /** + * Returns the next tx data that should be submitted to L1. + * + *

It currently only uses one frame per transaction. If the pending channel is full, it only + * returns the remaining frames of this channel until it got successfully fully sent to L1. It + * returns io.EOF if there's no pending frame. + * + * @param l1Head l1 head block id + * @return The next tx data that should be submitted to L1. + */ + public Channel.TxData txData(BlockId l1Head) { + return null; + } + + /** + * Records a transaction as failed. It will attempt to resubmit the data in the failed + * transaction. + * + * @param txId channel tx id + */ + public void txFailed(Channel.TxData txId) {} + + /** + * Marks a transaction as confirmed on L1. Unfortunately even if all frames in a channel have been + * marked as confirmed on L1 the channel may be invalid and need to be resubmitted. This function + * may reset the pending channel if the pending channel has timed out. + * + * @param txId channel tx id + * @param inclusionBlock inclusion block id + */ + public void txConfirmed(Channel.TxData txId, BlockId inclusionBlock) {} + + /** Clear blocks and channels that have not entered the pending state. */ + public void clear() {} +} diff --git a/hildr-batcher/src/main/java/io/optimism/batcher/config/Config.java b/hildr-batcher/src/main/java/io/optimism/batcher/config/Config.java new file mode 100644 index 00000000..5a1ed812 --- /dev/null +++ b/hildr-batcher/src/main/java/io/optimism/batcher/config/Config.java @@ -0,0 +1,41 @@ +/* + * Copyright 2023 q315xia@163.com + * + * 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. + */ + +package io.optimism.batcher.config; + +/** + * Batcher config. + * + * @param l1RpcUrl L1 rpc url + * @param l2RpcUrl L2 rpc url + * @param rollupRpcUrl Op-node rpc url + * @param l1Signer L1 signer private key + * @param batchInboxAddress address of BatchInboxContract on l1 + * @param subSafetyMargin Sub-safety margin + * @param pollInterval Milliseconds of poll interval + * @param maxL1TxSize Max L1 Tx Size + * @author thinkAfCod + * @since 0.1.1 + */ +public record Config( + String l1RpcUrl, + String l2RpcUrl, + String rollupRpcUrl, + String l1Signer, + String batchInboxAddress, + Long subSafetyMargin, + Long pollInterval, + Long maxL1TxSize) {} diff --git a/hildr-batcher/src/main/java/io/optimism/batcher/ex/ReorgException.java b/hildr-batcher/src/main/java/io/optimism/batcher/ex/ReorgException.java new file mode 100644 index 00000000..a8d2fb9d --- /dev/null +++ b/hildr-batcher/src/main/java/io/optimism/batcher/ex/ReorgException.java @@ -0,0 +1,54 @@ +/* + * Copyright 2023 q315xia@163.com + * + * 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. + */ + +package io.optimism.batcher.ex; + +/** + * ReorgException class. Throws this when chain occurs reorg. + * + * @author thinkAfCod + * @since 0.1.1 + */ +public class ReorgException extends RuntimeException { + + /** + * Instantiates a new reorg exception. + * + * @param message the message + */ + public ReorgException(String message) { + super(message); + } + + /** + * Instantiates a new reorg exception. + * + * @param message the message + * @param cause the cause + */ + public ReorgException(String message, Throwable cause) { + super(message, cause); + } + + /** + * Instantiates a new reorg exception. + * + * @param cause the cause + */ + public ReorgException(Throwable cause) { + super(cause); + } +} diff --git a/hildr-batcher/src/main/java/io/optimism/batcher/ex/Web3jCallException.java b/hildr-batcher/src/main/java/io/optimism/batcher/ex/Web3jCallException.java new file mode 100644 index 00000000..5531dcb8 --- /dev/null +++ b/hildr-batcher/src/main/java/io/optimism/batcher/ex/Web3jCallException.java @@ -0,0 +1,38 @@ +package io.optimism.batcher.ex; + +/** + * Web3jCallException class. Throws it when the call of web3j request task failed. + * + * @author thinkAfCod + * @since 0.1.1 + */ +public class Web3jCallException extends RuntimeException { + + /** + * Instantiates a new Web3jCallException. + * + * @param message the message + */ + public Web3jCallException(String message) { + super(message); + } + + /** + * Instantiates a new Web3jCallException. + * + * @param message the message + * @param cause the cause + */ + public Web3jCallException(String message, Throwable cause) { + super(message, cause); + } + + /** + * Instantiates a new Web3jCallException. + * + * @param cause the cause + */ + public Web3jCallException(Throwable cause) { + super(cause); + } +} diff --git a/hildr-batcher/src/main/java/io/optimism/batcher/loader/BlockLoader.java b/hildr-batcher/src/main/java/io/optimism/batcher/loader/BlockLoader.java new file mode 100644 index 00000000..c6e9f48e --- /dev/null +++ b/hildr-batcher/src/main/java/io/optimism/batcher/loader/BlockLoader.java @@ -0,0 +1,232 @@ +/* + * Copyright 2023 q315xia@163.com + * + * 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. + */ + +package io.optimism.batcher.loader; + +import io.optimism.batcher.ex.ReorgException; +import io.optimism.batcher.ex.SyncStatusException; +import io.optimism.type.BlockId; +import io.optimism.type.Genesis; +import io.optimism.type.L1BlockInfo; +import io.optimism.type.L1BlockRef; +import io.optimism.type.L2BlockRef; +import io.optimism.type.OpEthSyncStatusRes; +import io.optimism.type.RollupConfigRes; +import io.optimism.utilities.rpc.Web3jProvider; +import java.math.BigInteger; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.function.Consumer; +import jdk.incubator.concurrent.StructuredTaskScope; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.web3j.protocol.Web3j; +import org.web3j.protocol.Web3jService; +import org.web3j.protocol.core.DefaultBlockParameter; +import org.web3j.protocol.core.Request; +import org.web3j.protocol.core.methods.response.EthBlock; +import org.web3j.tuples.generated.Tuple2; +import org.web3j.utils.Numeric; + +/** + * BlockLoader class. + * + * @author thinkAfCod + * @since 0.1.1 + */ +@SuppressWarnings("UnusedVariable") +public class BlockLoader { + + private static final Logger LOGGER = LoggerFactory.getLogger(BlockLoader.class); + + private static final String DEPOSIT_TX_TYPE = "0x7E"; + + private static final String OP_ROLLUP_CONFIG = "optimism_rollupConfig"; + + private static final String OP_SYNC_STATUS = "optimism_syncStatus"; + + private final Web3j l2Client; + + private final Web3jService rollupService; + + private final Consumer blockConsumer; + + private BlockId latestLoadedBlock; + + private RollupConfigRes.RollupConfig rollupConfig; + + /** + * Constructor of BlockLoader. + * + * @param config LoaderConfig instance + * @param blockConsumer consumer block loaded from L2 + */ + public BlockLoader(LoaderConfig config, Consumer blockConsumer) { + this.l2Client = Web3jProvider.createClient(config.l2RpcUrl()); + Tuple2 tuple = Web3jProvider.create(config.rollupUrl()); + this.rollupService = tuple.component2(); + this.blockConsumer = blockConsumer; + this.rollupConfig = loadRollConfig(); + } + + /** + * Return rollup config object from rollup node. + * + * @return rollup config object + */ + public RollupConfigRes.RollupConfig getRollConfig() { + return this.rollupConfig; + } + + /** Trigger load block from L2. */ + public void loadBlock() { + this.loadBlocksIntoState(); + } + + private RollupConfigRes.RollupConfig loadRollConfig() { + try (var scope = new StructuredTaskScope.ShutdownOnFailure()) { + var future = + scope.fork( + () -> + new Request<>( + OP_ROLLUP_CONFIG, List.of(), this.rollupService, RollupConfigRes.class) + .send() + .getConfig()); + scope.join(); + scope.throwIfFailed(); + return future.resultNow(); + } catch (ExecutionException e) { + throw new RuntimeException(""); + } catch (InterruptedException e) { + throw new RuntimeException(""); + } + } + + private void loadBlocksIntoState() { + Tuple2 blockNumbers = this.calculateL2BlockRangeToStore(); + final BigInteger start = blockNumbers.component1().number(); + final BigInteger end = blockNumbers.component2(); + var stopBlock = end.add(BigInteger.ONE); + EthBlock.Block lastestBlock = null; + for (BigInteger i = start.add(BigInteger.ONE); + i.compareTo(stopBlock) < 0; + i = i.add(BigInteger.ONE)) { + EthBlock.Block block = this.loadBlockToChannel(i); + this.latestLoadedBlock = BlockId.from(block); + lastestBlock = block; + } + if (lastestBlock == null) { + throw new RuntimeException(""); + } + var ignore = l2BlockToBlockRef(lastestBlock, rollupConfig.genesis()); + // todo metrics.RecordL2BlocksLoaded l2Ref + } + + private Tuple2 calculateL2BlockRangeToStore() { + final Request req = + new Request<>(OP_SYNC_STATUS, List.of(), this.rollupService, OpEthSyncStatusRes.class); + try (var scope = new StructuredTaskScope.ShutdownOnFailure()) { + var future = scope.fork(req::send); + scope.join(); + scope.throwIfFailed(); + var syncStatus = future.resultNow().getOpEthSyncStatus(); + if (syncStatus.headL1().equals(L1BlockRef.emptyBlock)) { + throw new RuntimeException("empty sync status"); + } + if (latestLoadedBlock == null || latestLoadedBlock.number().compareTo(BigInteger.ZERO) == 0) { + LOGGER.info("Starting batch-submitter work at L2 safe-head: {}", syncStatus.safeL2()); + latestLoadedBlock = syncStatus.safeL2().toId(); + } else if (latestLoadedBlock.number().compareTo(syncStatus.safeL2().number()) <= 0) { + LOGGER.warn( + "last submitted block lagged behind L2 safe head: batch submission will continue"); + latestLoadedBlock = syncStatus.safeL2().toId(); + } + + if (syncStatus.safeL2().number().compareTo(syncStatus.unsafeL2().number()) >= 0 + || latestLoadedBlock.number().compareTo(syncStatus.unsafeL2().number()) >= 0) { + throw new RuntimeException("L2 safe head ahead of L2 unsafe head"); + } + return new Tuple2<>(latestLoadedBlock, syncStatus.unsafeL2().number()); + } catch (ExecutionException e) { + throw new SyncStatusException("StructuredTaskScope execute syncStatus failed:", e); + } catch (InterruptedException e) { + throw new SyncStatusException( + "Thread has been interrupted while calling calculateL2BlockRangeToStore:", e); + } + } + + private L2BlockRef l2BlockToBlockRef(final EthBlock.Block block, Genesis genesis) { + BlockId l1Origin = null; + BigInteger sequenceNumber = null; + if (block.getNumber().compareTo(genesis.l2().number()) == 0) { + if (!block.getHash().equals(genesis.l2().hash())) { + throw new RuntimeException( + String.format( + "expected L2 genesis hash to match L2 block at genesis block number %d: %s <> %s", + genesis.l2().number(), block.getHash(), genesis.l2().hash())); + } + l1Origin = genesis.l1(); + sequenceNumber = BigInteger.ZERO; + } else { + var txs = block.getTransactions(); + if (txs == null || txs.size() == 0) { + throw new RuntimeException( + String.format( + "l2 block is missing L1 info deposit tx, block hash: %s", block.getHash())); + } + EthBlock.TransactionObject tx = (EthBlock.TransactionObject) txs.get(0).get(); + if (!DEPOSIT_TX_TYPE.equals(tx.getType())) { + throw new RuntimeException( + String.format("first payload tx has unexpected tx type: %s", tx.getType())); + } + final byte[] input = Numeric.hexStringToByteArray(tx.getInput()); + L1BlockInfo info = L1BlockInfo.from(input); + l1Origin = info.toId(); + sequenceNumber = info.sequenceNumber(); + } + return new L2BlockRef( + block.getHash(), + block.getNumber(), + block.getParentHash(), + block.getTimestamp(), + l1Origin, + sequenceNumber); + } + + private EthBlock.Block loadBlockToChannel(BigInteger number) { + final Request ethBlockRequest = + l2Client.ethGetBlockByNumber(DefaultBlockParameter.valueOf(number), true); + EthBlock.Block block; + try (var scope = new StructuredTaskScope.ShutdownOnFailure()) { + var blockFur = scope.fork(ethBlockRequest::send); + scope.join(); + scope.throwIfFailed(); + block = blockFur.get().getBlock(); + if (block != null) { + blockConsumer.accept(block); + } + } catch (ExecutionException e) { + throw new SyncStatusException("StructuredTaskScope execute getBlockByNumber failed:", e); + } catch (InterruptedException e) { + throw new SyncStatusException( + "Thread has been interrupted while calling loadBlockToChannel:", e); + } catch (ReorgException e) { + this.latestLoadedBlock = null; + throw e; + } + return block; + } +} diff --git a/hildr-batcher/src/main/java/io/optimism/batcher/loader/LoaderConfig.java b/hildr-batcher/src/main/java/io/optimism/batcher/loader/LoaderConfig.java new file mode 100644 index 00000000..7b5a8d45 --- /dev/null +++ b/hildr-batcher/src/main/java/io/optimism/batcher/loader/LoaderConfig.java @@ -0,0 +1,40 @@ +/* + * Copyright 2023 q315xia@163.com + * + * 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. + */ + +package io.optimism.batcher.loader; + +import io.optimism.batcher.config.Config; + +/** + * L2 loader config. + * + * @param l2RpcUrl L2 rpc url + * @param rollupUrl op-rollup node url + * @author thinkAfCod + * @since 0.1.1 + */ +public record LoaderConfig(String l2RpcUrl, String rollupUrl) { + + /** + * Create a LoaderConfig instance from Config instance. + * + * @param config Config instance + * @return LoaderConfig instance + */ + public static LoaderConfig from(Config config) { + return new LoaderConfig(config.l2RpcUrl(), config.rollupRpcUrl()); + } +} diff --git a/hildr-batcher/src/main/java/io/optimism/batcher/publisher/ChannelDataPublisher.java b/hildr-batcher/src/main/java/io/optimism/batcher/publisher/ChannelDataPublisher.java new file mode 100644 index 00000000..5905f77e --- /dev/null +++ b/hildr-batcher/src/main/java/io/optimism/batcher/publisher/ChannelDataPublisher.java @@ -0,0 +1,168 @@ +/* + * Copyright 2023 q315xia@163.com + * + * 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. + */ + +package io.optimism.batcher.publisher; + +import io.optimism.batcher.channel.Channel; +import io.optimism.batcher.ex.Web3jCallException; +import io.optimism.type.BlockId; +import io.optimism.type.L1BlockRef; +import io.optimism.type.TxCandidate; +import io.optimism.utilities.gas.GasCalculator; +import io.optimism.utilities.rpc.Web3jProvider; +import io.optimism.utilities.telemetry.TracerTaskWrapper; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.function.Consumer; +import java.util.function.Function; +import jdk.incubator.concurrent.StructuredTaskScope; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.web3j.protocol.Web3j; +import org.web3j.protocol.core.DefaultBlockParameterName; +import org.web3j.protocol.core.methods.response.EthBlock; +import org.web3j.protocol.core.methods.response.EthGetTransactionReceipt; +import org.web3j.protocol.core.methods.response.EthSendTransaction; +import org.web3j.protocol.core.methods.response.TransactionReceipt; +import org.web3j.tx.RawTransactionManager; + +/** + * ChannelDataPublisher class. It will get tx data from channelManager and push it to L1. + * + * @author thinkAfCod + * @since 0.1.1 + */ +@SuppressWarnings("UnusedVariable") +public class ChannelDataPublisher { + + private static final Logger LOGGER = LoggerFactory.getLogger(ChannelDataPublisher.class); + + private final PublisherConfig config; + + private final Web3j l1Client; + + private final RawTransactionManager txManager; + + private final Function dataSupplier; + + private final Consumer txReceiptReturn; + + private L1BlockRef lastL1Tip; + + /** + * Constructor of ChannelDataPublisher. + * + * @param config publisher config + * @param dataSupplier publisher data supplier + * @param txReceiptReturn tx receipt return callback + */ + public ChannelDataPublisher( + PublisherConfig config, + Function dataSupplier, + Consumer txReceiptReturn) { + this.config = config; + this.l1Client = Web3jProvider.createClient(config.l1RpcUrl()); + this.txManager = new RawTransactionManager(l1Client, null); + this.dataSupplier = dataSupplier; + this.txReceiptReturn = txReceiptReturn; + } + + /** + * Publish pending block data. + * + * @return return true if there any data has been published, otherwise false. + */ + public boolean publishPendingBlock() { + boolean hasData = false; + try { + boolean sendData = true; + while (sendData) { + sendData = this.publishTxToL1(); + if (!hasData) { + hasData = sendData; + } + } + return hasData; + } catch (RuntimeException e) { + throw e; + } + } + + private boolean publishTxToL1() { + final L1BlockRef l1HeadBlockRef = getL1HeadBlockRef(); + this.recordL1Head(l1HeadBlockRef); + Channel.TxData txData = dataSupplier.apply(l1HeadBlockRef.toId()); + if (txData == null) { + LOGGER.trace("no transaction data available"); + throw new RuntimeException(""); + } + this.sendTx(txData); + return true; + } + + private void sendTx(Channel.TxData txData) { + byte[] txBytes = txData.txBytes(); + long intrinsicGas = + GasCalculator.intrinsicGasWithoutAccessList(txBytes, false, true, true, false); + var txCandidate = new TxCandidate(txBytes, this.config.batchInboxAddress(), intrinsicGas); + EthSendTransaction ethSendTransaction = null; + String txHash = ethSendTransaction.getTransactionHash(); + var txReceipt = this.getTxReceipt(txHash); + txReceiptReturn.accept(txReceipt.getTransactionReceipt().get()); + // todo use txManager send tx + } + + private EthGetTransactionReceipt getTxReceipt(final String txHash) { + try (var scope = new StructuredTaskScope.ShutdownOnFailure()) { + var fur = scope.fork(() -> l1Client.ethGetTransactionReceipt(txHash).send()); + scope.join(); + scope.throwIfFailed(); + return fur.resultNow(); + } catch (InterruptedException | ExecutionException e) { + throw new Web3jCallException("task of getTxReceipt has been interrupted", e); + } + } + + private L1BlockRef getL1HeadBlockRef() { + try (var scope = new StructuredTaskScope.ShutdownOnFailure()) { + Future fur = + scope.fork( + TracerTaskWrapper.wrap( + () -> + l1Client + .ethGetBlockByNumber(DefaultBlockParameterName.LATEST, false) + .send() + .getBlock())); + scope.join(); + scope.throwIfFailed(); + var block = fur.get(); + if (block == null) { + throw new RuntimeException(""); + } + return L1BlockRef.from(block); + } catch (ExecutionException | InterruptedException e) { + throw new Web3jCallException("task of getL1HeadBlockRef has been interrupted", e); + } + } + + private void recordL1Head(L1BlockRef headRef) { + if (this.lastL1Tip.equals(headRef)) { + return; + } + this.lastL1Tip = headRef; + // todo metrics LatestL1Block + } +} diff --git a/hildr-batcher/src/main/java/io/optimism/batcher/publisher/PublisherConfig.java b/hildr-batcher/src/main/java/io/optimism/batcher/publisher/PublisherConfig.java new file mode 100644 index 00000000..94dbbd0c --- /dev/null +++ b/hildr-batcher/src/main/java/io/optimism/batcher/publisher/PublisherConfig.java @@ -0,0 +1,42 @@ +/* + * Copyright 2023 q315xia@163.com + * + * 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. + */ + +package io.optimism.batcher.publisher; + +import io.optimism.batcher.config.Config; + +/** + * Publisher Config class. + * + * @param l1RpcUrl L1 rpc url + * @param l1Signer L1 signer private key + * @param batchInboxAddress Address of BatchInboxContract on L1 + * @author thinkAfCod + * @since 0.1.1 + */ +public record PublisherConfig(String l1RpcUrl, String l1Signer, String batchInboxAddress) { + + /** + * Create a PublisherConfig instance from Config instance. + * + * @param config Config instance + * @param batchInboxAddress address of BatchInboxContract + * @return PublisherConfig instance + */ + public static PublisherConfig from(Config config, String batchInboxAddress) { + return new PublisherConfig(config.l1RpcUrl(), config.l1Signer(), batchInboxAddress); + } +} diff --git a/hildr-batcher/src/main/java/io/optimism/type/L1BlockRef.java b/hildr-batcher/src/main/java/io/optimism/type/L1BlockRef.java index 7f781c27..120ed15c 100644 --- a/hildr-batcher/src/main/java/io/optimism/type/L1BlockRef.java +++ b/hildr-batcher/src/main/java/io/optimism/type/L1BlockRef.java @@ -18,6 +18,7 @@ import java.math.BigInteger; import java.util.Objects; +import org.web3j.protocol.core.methods.response.EthBlock; /** * L1 block brief information. @@ -47,6 +48,17 @@ public BlockId toId() { return new BlockId(hash, number); } + /** + * Create a L1BlockRef instance from EthBlock.Block instance. + * + * @param block EthBlock.Block instance. + * @return a L1BlockRef instance + */ + public static L1BlockRef from(EthBlock.Block block) { + return new L1BlockRef( + block.getHash(), block.getNumber(), block.getParentHash(), block.getTimestamp()); + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/hildr-batcher/src/main/java/io/optimism/type/TxCandidate.java b/hildr-batcher/src/main/java/io/optimism/type/TxCandidate.java new file mode 100644 index 00000000..63a36d8f --- /dev/null +++ b/hildr-batcher/src/main/java/io/optimism/type/TxCandidate.java @@ -0,0 +1,29 @@ +/* + * Copyright 2023 q315xia@163.com + * + * 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. + */ + +package io.optimism.type; + +/** + * TxCandidate is a transaction candidate that can be submitted to ask to construct a transaction + * with gas price bounds. + * + * @param txData the transaction data to be used in the constructed tx. + * @param address To is the recipient of the constructed tx. Nil means contract creation. + * @param gasLimit the gas limit to be used in the constructed tx. + * @author thinkAfCod + * @since 0.1.1 + */ +public record TxCandidate(byte[] txData, String address, long gasLimit) {} diff --git a/hildr-node/src/test/java/io/optimism/rpc/RpcServerTest.java b/hildr-node/src/test/java/io/optimism/rpc/RpcServerTest.java index d9566b97..c6e9f37b 100644 --- a/hildr-node/src/test/java/io/optimism/rpc/RpcServerTest.java +++ b/hildr-node/src/test/java/io/optimism/rpc/RpcServerTest.java @@ -27,6 +27,7 @@ import io.optimism.rpc.internal.JsonRpcRequest; import io.optimism.rpc.internal.JsonRpcRequestId; import io.optimism.rpc.internal.result.OutputRootResult; +import io.optimism.telemetry.Logging; import io.optimism.utilities.telemetry.TracerTaskWrapper; import java.time.Duration; import java.util.Map; @@ -56,6 +57,7 @@ public class RpcServerTest { @BeforeAll static void setUp() { + TracerTaskWrapper.setTracerSupplier(Logging.INSTANCE::getTracer); config = TestConstants.createConfig(); } diff --git a/hildr-utilities/src/main/java/io/optimism/utilities/gas/GasCalculator.java b/hildr-utilities/src/main/java/io/optimism/utilities/gas/GasCalculator.java new file mode 100644 index 00000000..8eddb07c --- /dev/null +++ b/hildr-utilities/src/main/java/io/optimism/utilities/gas/GasCalculator.java @@ -0,0 +1,87 @@ +/* + * Copyright 2023 q315xia@163.com + * + * 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. + */ + +package io.optimism.utilities.gas; + +/** + * Gas util. + * + * @author thinkAfCod + * @since 0.1.1 + */ +@SuppressWarnings("checkstyle:AbbreviationAsWordInName") +public class GasCalculator { + + /** + * Private Constructor of GasCalculator. + */ + private GasCalculator() {} + + public static final long TX_GAS_CONTRACT_CREATION = 53000L; + + public static final long TX_GAS = 21000L; + + public static final long TX_DATA_NON_ZERO_GAS_FRONTIER = 68L; + + public static final long TX_DATA_NON_ZERO_GAS_EIP2028 = 16L; + + public static final long TX_DATA_ZERO_GAS = 4L; + + public static final long INIT_CODE_WORD_GAS = 2L; + + /** + * Calculator gas fee but exclude effective of AccessList. + * + * @param data Tx data + * @param isContractCreation Is contract creation + * @param isHomestead Is home stead + * @param isEIP2028 Is EIP2028 + * @param isEIP3860 Is EIP3860 + * @return Intrinsic gas + */ + public static long intrinsicGasWithoutAccessList( + byte[] data, + boolean isContractCreation, boolean isHomestead, boolean isEIP2028, boolean isEIP3860) { + var gas = isContractCreation && isHomestead ? TX_GAS_CONTRACT_CREATION : TX_GAS; + if (data.length <= 0) { + return gas; + } + long nz = 0; + for (var byt : data) { + if (byt != 0) { + nz += 1L; + } + } + var nonZeroGas = isEIP2028 ? TX_DATA_NON_ZERO_GAS_EIP2028 : TX_DATA_NON_ZERO_GAS_FRONTIER; + var gasRange = Long.MAX_VALUE - gas; + gas += nz * nonZeroGas; + + var z = data.length / nz; + gas += z * TX_DATA_ZERO_GAS; + if (isContractCreation && isEIP3860) { + var lenWords = toWordSize(data.length); + if (gasRange / INIT_CODE_WORD_GAS < lenWords) { + throw new RuntimeException("Gas uint overflow"); + } + gas += lenWords * INIT_CODE_WORD_GAS; + } + return gas; + } + + private static long toWordSize(int size) { + return (size + 31L) / 32L; + } +}