BewareMyPower commented on code in PR #15413:
URL: https://github.com/apache/pulsar/pull/15413#discussion_r864468071


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageKeyBasedContainer.java:
##########
@@ -115,66 +82,46 @@ public boolean isMultiBatches() {
         return true;
     }
 
-    private ProducerImpl.OpSendMsg createOpSendMsg(KeyedBatch keyedBatch) 
throws IOException {
-        ByteBuf encryptedPayload = 
producer.encryptMessage(keyedBatch.messageMetadata,
-                keyedBatch.getCompressedBatchMetadataAndPayload());
-        if (encryptedPayload.readableBytes() > ClientCnx.getMaxMessageSize()) {
-            keyedBatch.discard(new 
PulsarClientException.InvalidMessageException(
-                    "Message size is bigger than " + 
ClientCnx.getMaxMessageSize() + " bytes"));
-            return null;
-        }
-
-        final int numMessagesInBatch = keyedBatch.messages.size();
-        long currentBatchSizeBytes = 0;
-        for (MessageImpl<?> message : keyedBatch.messages) {
-            currentBatchSizeBytes += message.getDataBuffer().readableBytes();
-        }
-        keyedBatch.messageMetadata.setNumMessagesInBatch(numMessagesInBatch);
-        if (currentTxnidMostBits != -1) {
-            keyedBatch.messageMetadata.setTxnidMostBits(currentTxnidMostBits);
-        }
-        if (currentTxnidLeastBits != -1) {
-            
keyedBatch.messageMetadata.setTxnidLeastBits(currentTxnidLeastBits);
-        }
-        ByteBufPair cmd = producer.sendMessage(producer.producerId, 
keyedBatch.sequenceId, numMessagesInBatch,
-                keyedBatch.messageMetadata, encryptedPayload);
-
-        ProducerImpl.OpSendMsg op = ProducerImpl.OpSendMsg.create(
-                keyedBatch.messages, cmd, keyedBatch.sequenceId, 
keyedBatch.firstCallback);
-
-        op.setNumMessagesInBatch(numMessagesInBatch);
-        op.setBatchSizeByte(currentBatchSizeBytes);
-        return op;
-    }
-
     @Override
     public List<ProducerImpl.OpSendMsg> createOpSendMsgs() throws IOException {
-        List<ProducerImpl.OpSendMsg> result = new ArrayList<>();
-        List<KeyedBatch> list = new ArrayList<>(batches.values());
-        list.sort(((o1, o2) -> ComparisonChain.start()
-                .compare(o1.sequenceId, o2.sequenceId)
-                .result()));
-        for (KeyedBatch keyedBatch : list) {
-            ProducerImpl.OpSendMsg op = createOpSendMsg(keyedBatch);
-            if (op != null) {
-                result.add(op);
+        try {
+            // In key based batching, the sequence ids might not be ordered, 
for example,
+            // | key | sequence id list |
+            // | :-- | :--------------- |
+            // | A | 0, 3, 4 |
+            // | B | 1, 2 |
+            // The message order should be 1, 2, 0, 3, 4 so that a message 
with a sequence id <= 4 should be dropped.
+            // However, for a MessageMetadata with both `sequence_id` and 
`highest_sequence_id` fields, the broker will
+            // expect a strict order so that the batch of key "A" (0, 3, 4) 
will be dropped.
+            // Therefore, we should update the `sequence_id` field to the 
highest sequence id and remove the
+            // `highest_sequence_id` field to allow the weak order.

Review Comment:
   > If the client didn't receive the ack receipt of batch B, it might resend 
batch B.
   
   After this patch the send order will be batch B, batch A as they are sorted 
by the highest sequence id (1 < 3).
   
   If batch A was not persisted, it would be OK. Otherwise, since batch B was 
sent before batch A, it means batch A has been persisted as well. So rejecting 
batch B is reasonable.
   
   



-- 
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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to