You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2019/06/13 10:38:55 UTC

[GitHub] [pulsar] codelipenghui commented on a change in pull request #4265: [transaction][acknowledge] Introduce in-memory PENDING_ACK state in acknowledgement path

codelipenghui commented on a change in pull request #4265: [transaction][acknowledge] Introduce in-memory PENDING_ACK state in acknowledgement path
URL: https://github.com/apache/pulsar/pull/4265#discussion_r293313645
 
 

 ##########
 File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java
 ##########
 @@ -242,6 +309,94 @@ public void acknowledgeMessage(List<Position> positions, AckType ackType, Map<St
         }
     }
 
+    /**
+     * Acknowledge message(s) for an ongoing transaction.
+     * <p>
+     * It can be of {@link AckType#Individual} or {@link AckType#Cumulative}. Single messages acked by ongoing
+     * transaction will be put in pending_ack state and only marked as deleted after transaction is committed.
+     * <p>
+     * Only one transaction is allowed to do cumulative ack on a subscription at a given time.
+     * If a transaction do multiple cumulative ack, only the one with largest position determined by
+     * {@link PositionImpl#compareTo(PositionImpl)} will be kept as it cover all position smaller than it.
+     * <p>
+     * If an ongoing transaction cumulative acked a message and then try to ack single message which is
+     * smaller than that one it cumulative acked, it'll succeed.
+     * <p>
+     * If transaction is aborted all messages acked by it will be put back to pending state.
+     *
+     * @param txnId                  TransactionID of an ongoing transaction trying to sck message.
+     * @param positions              {@link Position}(s) it try to ack.
+     * @param ackType                {@link AckType}.
+     * @throws TransactionConflictException if try to do cumulative ack when another ongoing transaction already doing
+     *  cumulative ack or try to single ack message already acked by any ongoing transaction.
+     * @throws IllegalArgumentException if try to cumulative ack but passed in multiple positions.
+     */
+    public synchronized void acknowledgeMessage(TxnID txnId, List<Position> positions, AckType ackType) throws TransactionConflictException {
+        checkArgument(txnId != null, "TransactionID can not be null.");
+        if (AckType.Cumulative == ackType) {
+            // Check if another transaction is already using cumulative ack on this subscription.
+            if (this.pendingCumulativeAckTxnId != null && this.pendingCumulativeAckTxnId != txnId) {
+                String errorMsg = "[" + topicName + "][" + subName + "] Transaction:" + txnId +
+                                  " try to cumulative ack message while transaction:" + this.pendingCumulativeAckTxnId +
+                                  " already cumulative acked messages.";
+                log.error(errorMsg);
+                throw new TransactionConflictException(errorMsg);
+            }
+
+            if (positions.size() != 1) {
+                String errorMsg = "[" + topicName + "][" + subName + "] Transaction:" + txnId +
+                                  " invalid cumulative ack received with multiple message ids.";
+                log.error(errorMsg);
+                throw new IllegalArgumentException(errorMsg);
+            }
+
+            Position position = positions.get(0);
+            checkArgument(position instanceof PositionImpl);
+
+            if (log.isDebugEnabled()) {
+                log.debug("[{}][{}] TxnID:[{}] Cumulative ack on {}.", topicName, subName, txnId.toString(), position);
+            }
+
+             if (this.pendingCumulativeAckTxnId == null) {
+                // Only set pendingCumulativeAckTxnId if no transaction is doing cumulative ack.
+                PENDING_CUMULATIVE_ACK_TXNID_UPDATER.set(this, txnId);
+                POSITION_UPDATER.set(this, position);
+            } else if (((PositionImpl)position).compareTo((PositionImpl)this.pendingCumulativeAckMessage) > 0) {
+                // If new cumulative ack position is greater than current one, update it.
+                POSITION_UPDATER.set(this, position);
+            }
+        } else {
+            if (log.isDebugEnabled()) {
+                log.debug("[{}][{}] TxnID:[{}] Individual acks on {}", topicName, subName, txnId.toString(), positions);
+            }
+
+            if (pendingAckMessagesMap == null) {
+                pendingAckMessagesMap = new ConcurrentOpenHashMap<>();
+            }
+
+            if (pendingAckMessages == null) {
+                pendingAckMessages = new ConcurrentOpenHashSet<>();
+            }
+
+            ConcurrentOpenHashSet<Position> pendingAckMessageForCurrentTxn =
+                    pendingAckMessagesMap.computeIfAbsent(txnId, txn -> new ConcurrentOpenHashSet<>());
+
+            for (Position position : positions) {
+                // If try to ack message already acked by some transaction(can be itself), throw exception.
+                // Acking single message within range of cumulative ack(if exist) is considered valid operation.
+                if (this.pendingAckMessages.contains(position)) {
 
 Review comment:
   If ack message already acked by consumer without transaction, shall we need to abort the transaction?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services