codelipenghui commented on code in PR #20948:
URL: https://github.com/apache/pulsar/pull/20948#discussion_r1310403563


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java:
##########
@@ -1459,6 +1460,29 @@ private ByteBuf processMessageChunk(ByteBuf 
compressedPayload, MessageMetadata m
         // discard message if chunk is out-of-order
         if (chunkedMsgCtx == null || chunkedMsgCtx.chunkedMsgBuffer == null
                 || msgMetadata.getChunkId() != 
(chunkedMsgCtx.lastChunkedMessageId + 1)) {
+            // Filter and ack duplicated chunks instead of discard ctx.
+            // For example:
+            //     Chunk-1 sequence ID: 0, chunk ID: 0, msgID: 1:1
+            //     Chunk-2 sequence ID: 0, chunk ID: 1, msgID: 1:2
+            //     Chunk-3 sequence ID: 0, chunk ID: 2, msgID: 1:3
+            //     Chunk-4 sequence ID: 0, chunk ID: 1, msgID: 1:4
+            //     Chunk-5 sequence ID: 0, chunk ID: 2, msgID: 1:5
+            //     Chunk-6 sequence ID: 0, chunk ID: 3, msgID: 1:6
+            // We should filter and ack chunk-4 and chunk-5.

Review Comment:
   It could be risky here if the producer side changed the chunk policy. For 
example:
   
   ```
   Chunk-1 sequence ID: 0, chunk ID: 0, size 10kb
   Chunk-1 sequence ID: 0, chunk ID: 1, size 10kb
   Chunk-1 sequence ID: 0, chunk ID: 2, size 10kb
   
   ```
   
   The producer restarted and sent the message with the same seq with the 
changed chunk policy.
   
   ```
   Chunk-1 sequence ID: 0, chunk ID: 0, size 5kb
   Chunk-1 sequence ID: 0, chunk ID: 1, size 5kb
   Chunk-1 sequence ID: 0, chunk ID: 2, size 5kb
   Chunk-1 sequence ID: 0, chunk ID: 3, size 5kb
   Chunk-1 sequence ID: 0, chunk ID: 4, size 5kb
   Chunk-1 sequence ID: 0, chunk ID: 5, size 5kb
   Chunk-1 sequence ID: 0, chunk ID: 6, size 5kb
   Chunk-1 sequence ID: 0, chunk ID: 7, size 5kb
   ```
   
   If you filter out 
   
   ```
   Chunk-1 sequence ID: 0, chunk ID: 0, size 5kb
   Chunk-1 sequence ID: 0, chunk ID: 1, size 5kb
   Chunk-1 sequence ID: 0, chunk ID: 2, size 5kb
   ```
   
   You will get a corrupted message:
   
   ```
   Chunk-1 sequence ID: 0, chunk ID: 0, size 10kb
   Chunk-1 sequence ID: 0, chunk ID: 1, size 10kb
   Chunk-1 sequence ID: 0, chunk ID: 2, size 10kb
   Chunk-1 sequence ID: 0, chunk ID: 3, size 5kb
   Chunk-1 sequence ID: 0, chunk ID: 4, size 5kb
   Chunk-1 sequence ID: 0, chunk ID: 5, size 5kb
   Chunk-1 sequence ID: 0, chunk ID: 6, size 5kb
   Chunk-1 sequence ID: 0, chunk ID: 7, size 5kb
   ```
   
   IMO, the correct solution is always to discard the unsealed chunks and make 
sure the producer side should only resend the whole chunk message instead of 
part of the chunks.
   
   For example:
   
   ```
               //     Chunk-1 sequence ID: 0, chunk ID: 0, msgID: 1:1
               //     Chunk-2 sequence ID: 0, chunk ID: 1, msgID: 1:2
               //     Chunk-3 sequence ID: 0, chunk ID: 2, msgID: 1:3
               //     Chunk-4 sequence ID: 0, chunk ID: 1, msgID: 1:4
               //     Chunk-5 sequence ID: 0, chunk ID: 2, msgID: 1:5
               //     Chunk-6 sequence ID: 0, chunk ID: 3, msgID: 1:6
   ```
   
   It shouldn't happen. You should always see
   
   ```
               //     Chunk-1 sequence ID: 0, chunk ID: 0, msgID: 1:1
               //     Chunk-2 sequence ID: 0, chunk ID: 1, msgID: 1:2
               //     Chunk-3 sequence ID: 0, chunk ID: 2, msgID: 1:3
               //     Chunk-4 sequence ID: 0, chunk ID: 0, msgID: 1:4
               //     Chunk-5 sequence ID: 0, chunk ID: 1, msgID: 1:5
               //     Chunk-6 sequence ID: 0, chunk ID: 2, msgID: 1:6
               //     Chunk-7 sequence ID: 0, chunk ID: 3, msgID: 1:7
   ```



##########
pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChunkingDeduplicationTest.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.client.impl;
+
+import static 
org.apache.pulsar.client.impl.MessageChunkingSharedTest.sendChunk;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertNull;
+import static org.testng.Assert.assertTrue;
+import java.lang.reflect.Field;
+import java.util.concurrent.TimeUnit;
+import lombok.Cleanup;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.service.persistent.PersistentTopic;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerConsumerBase;
+import org.apache.pulsar.client.api.Schema;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+@Slf4j
+@Test(groups = "broker-impl")
+public class MessageChunkingDeduplicationTest extends ProducerConsumerBase {
+
+    @BeforeClass
+    @Override
+    protected void setup() throws Exception {
+        this.conf.setBrokerDeduplicationEnabled(true);
+        super.internalSetup();
+        super.producerBaseSetup();
+    }
+
+    @AfterClass(alwaysRun = true)
+    @Override
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+    }
+
+    @Test
+    public void testSendChunkMessageWithSameSequenceID() throws Exception {
+        String topicName = 
"persistent://my-property/my-ns/testSendChunkMessageWithSameSequenceID";
+        String producerName = "test-producer";
+        @Cleanup
+        Consumer<String> consumer = pulsarClient
+                .newConsumer(Schema.STRING)
+                .subscriptionName("test-sub")
+                .topic(topicName)
+                .subscribe();
+        @Cleanup
+        Producer<String> producer = pulsarClient
+                .newProducer(Schema.STRING)
+                .producerName(producerName)
+                .topic(topicName)
+                .enableChunking(true)
+                .enableBatching(false)
+                .create();
+        int messageSize = 6000; // payload size in KB
+        String message = "a".repeat(messageSize * 1000);
+        producer.newMessage().value(message).sequenceId(10).send();
+        Message<String> msg = consumer.receive(10, TimeUnit.SECONDS);
+        assertNotNull(msg);
+        assertTrue(msg.getMessageId() instanceof ChunkMessageIdImpl);
+        assertEquals(msg.getValue(), message);
+        producer.newMessage().value(message).sequenceId(10).send();
+        msg = consumer.receive(3, TimeUnit.SECONDS);
+        assertNull(msg);
+    }
+
+    @Test
+    public void testDuplicateForChunkMessage() throws Exception {

Review Comment:
   Or is it just a test method name issue? Do you want to test the case that 
without any duplicated chunk messages?



##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java:
##########
@@ -1459,6 +1460,29 @@ private ByteBuf processMessageChunk(ByteBuf 
compressedPayload, MessageMetadata m
         // discard message if chunk is out-of-order
         if (chunkedMsgCtx == null || chunkedMsgCtx.chunkedMsgBuffer == null
                 || msgMetadata.getChunkId() != 
(chunkedMsgCtx.lastChunkedMessageId + 1)) {
+            // Filter and ack duplicated chunks instead of discard ctx.
+            // For example:
+            //     Chunk-1 sequence ID: 0, chunk ID: 0, msgID: 1:1
+            //     Chunk-2 sequence ID: 0, chunk ID: 1, msgID: 1:2
+            //     Chunk-3 sequence ID: 0, chunk ID: 2, msgID: 1:3
+            //     Chunk-4 sequence ID: 0, chunk ID: 1, msgID: 1:4
+            //     Chunk-5 sequence ID: 0, chunk ID: 2, msgID: 1:5
+            //     Chunk-6 sequence ID: 0, chunk ID: 3, msgID: 1:6
+            // We should filter and ack chunk-4 and chunk-5.
+            if (chunkedMsgCtx != null && msgMetadata.getChunkId() <= 
chunkedMsgCtx.lastChunkedMessageId) {
+                log.warn("[{}] Receive a repeated chunk messageId {}, 
last-chunk-id{}, chunkId = {}",

Review Comment:
   ```suggestion
                   log.warn("[{}] Receive a duplicated chunk messageId {}, 
last-chunk-id{}, chunkId = {}",
   ```
   
   Could you please also add the real message ID to the log? So that we can 
know which entry is the duplicated one in the topic.



##########
pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChunkingDeduplicationTest.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.client.impl;
+
+import static 
org.apache.pulsar.client.impl.MessageChunkingSharedTest.sendChunk;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertNull;
+import static org.testng.Assert.assertTrue;
+import java.lang.reflect.Field;
+import java.util.concurrent.TimeUnit;
+import lombok.Cleanup;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.broker.service.persistent.PersistentTopic;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerConsumerBase;
+import org.apache.pulsar.client.api.Schema;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+@Slf4j
+@Test(groups = "broker-impl")
+public class MessageChunkingDeduplicationTest extends ProducerConsumerBase {
+
+    @BeforeClass
+    @Override
+    protected void setup() throws Exception {
+        this.conf.setBrokerDeduplicationEnabled(true);
+        super.internalSetup();
+        super.producerBaseSetup();
+    }
+
+    @AfterClass(alwaysRun = true)
+    @Override
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+    }
+
+    @Test
+    public void testSendChunkMessageWithSameSequenceID() throws Exception {
+        String topicName = 
"persistent://my-property/my-ns/testSendChunkMessageWithSameSequenceID";
+        String producerName = "test-producer";
+        @Cleanup
+        Consumer<String> consumer = pulsarClient
+                .newConsumer(Schema.STRING)
+                .subscriptionName("test-sub")
+                .topic(topicName)
+                .subscribe();
+        @Cleanup
+        Producer<String> producer = pulsarClient
+                .newProducer(Schema.STRING)
+                .producerName(producerName)
+                .topic(topicName)
+                .enableChunking(true)
+                .enableBatching(false)
+                .create();
+        int messageSize = 6000; // payload size in KB
+        String message = "a".repeat(messageSize * 1000);
+        producer.newMessage().value(message).sequenceId(10).send();
+        Message<String> msg = consumer.receive(10, TimeUnit.SECONDS);
+        assertNotNull(msg);
+        assertTrue(msg.getMessageId() instanceof ChunkMessageIdImpl);
+        assertEquals(msg.getValue(), message);
+        producer.newMessage().value(message).sequenceId(10).send();
+        msg = consumer.receive(3, TimeUnit.SECONDS);
+        assertNull(msg);
+    }
+
+    @Test
+    public void testDuplicateForChunkMessage() throws Exception {

Review Comment:
   Which chunk message is duplicated in this test?
   
   What I see is
   
   - Send a message with seq (init_value)
   - Send a message with seq (init_value + 1)
   - Send a message with seq (init_value + 1 + 1024)



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