sijie commented on a change in pull request #4738: [Transaction][buffer] Add basic operation of transaction URL: https://github.com/apache/pulsar/pull/4738#discussion_r306119336
########## File path: pulsar-transaction/buffer/src/main/java/org/apache/pulsar/transaction/buffer/impl/PersistentTransactionBufferReader.java ########## @@ -0,0 +1,141 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.pulsar.transaction.buffer.impl; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.SortedMap; +import java.util.concurrent.CompletableFuture; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.AsyncCallbacks; +import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.transaction.buffer.TransactionBufferReader; +import org.apache.pulsar.transaction.buffer.TransactionEntry; +import org.apache.pulsar.transaction.buffer.TransactionMeta; +import org.apache.pulsar.transaction.buffer.exceptions.EndOfTransactionException; +import org.apache.pulsar.transaction.buffer.exceptions.TransactionNotSealedException; +import org.apache.pulsar.transaction.impl.common.TxnStatus; + +/** + * A persistent transaction buffer reader implementation. + */ +@Slf4j +public class PersistentTransactionBufferReader implements TransactionBufferReader { + + private final ManagedCursor readCursor; + private final TransactionMeta meta; + private long currentSequenceId = -1L; + + + PersistentTransactionBufferReader(TransactionMeta meta, ManagedLedger ledger) + throws ManagedLedgerException, TransactionNotSealedException { + if (TxnStatus.OPEN == meta.status()) { + throw new TransactionNotSealedException("Transaction `" + meta.id() + "` is not sealed yet"); + } + this.meta = meta; + this.readCursor = ledger.newNonDurableCursor(PositionImpl.earliest); + } + + @Override + public CompletableFuture<List<TransactionEntry>> readNext(int numEntries) { + return meta.readEntries(numEntries, currentSequenceId).thenCompose(this::readEntry); + } + + private CompletableFuture<List<TransactionEntry>> readEntry(SortedMap<Long, Position> entries) { + CompletableFuture<List<TransactionEntry>> readFuture = new CompletableFuture<>(); + + List<TransactionEntry> txnEntries = new ArrayList<>(entries.size()); + List<CompletableFuture<Void>> futures = new ArrayList<>(); + + for (Map.Entry<Long, Position> longPositionEntry : entries.entrySet()) { + CompletableFuture<Void> tmpFuture = new CompletableFuture<>(); + readEntry(longPositionEntry.getValue()).whenComplete((entry, throwable) -> { + if (null != throwable) { + tmpFuture.completeExceptionally(throwable); + } else { + TransactionEntry txnEntry = new TransactionEntryImpl(meta.id(), longPositionEntry.getKey(), + entry.getDataBuffer(), + meta.committedAtLedgerId(), + meta.committedAtEntryId()); + txnEntries.add(txnEntry); + tmpFuture.complete(null); + } + }); + } + + FutureUtil.waitForAll(futures).whenComplete((v, e) -> { + if (e != null) { + readFuture.completeExceptionally(e); + } else { + readFuture.complete(txnEntries); + } + }); + + return readFuture; + } + + private CompletableFuture<Entry> readEntry(Position position) { + CompletableFuture<Entry> readFuture = new CompletableFuture<>(); + + readCursor.seek(position); Review comment: This is a problem. You can't really use readCursor here. Because you can have many outstanding reads attempting to read from different positions. You have to bypass readCursor to read directly from managed ledger. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected] With regards, Apache Git Services
