merlimat closed pull request #1514: Compaction allows keyless messages to pass 
through
URL: https://github.com/apache/incubator-pulsar/pull/1514
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchConverter.java
 
b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchConverter.java
index fa629827af..4e628bc5f8 100644
--- 
a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchConverter.java
+++ 
b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchConverter.java
@@ -105,12 +105,15 @@ public static boolean isBatch(RawMessage msg) {
                 ByteBuf singleMessagePayload = 
Commands.deSerializeSingleMessageInBatch(payload,
                                                                                
         singleMessageMetadataBuilder,
                                                                                
         0, batchSize);
-                String key = singleMessageMetadataBuilder.getPartitionKey();
                 MessageId id = new 
BatchMessageIdImpl(msg.getMessageIdData().getLedgerId(),
                                                       
msg.getMessageIdData().getEntryId(),
                                                       
msg.getMessageIdData().getPartition(),
                                                       i);
-                if (filter.test(key, id)) {
+                if (!singleMessageMetadataBuilder.hasPartitionKey()) {
+                    messagesRetained++;
+                    
Commands.serializeSingleMessageInBatchWithPayload(singleMessageMetadataBuilder,
+                                                                      
singleMessagePayload, batchBuffer);
+                } else if 
(filter.test(singleMessageMetadataBuilder.getPartitionKey(), id)) {
                     messagesRetained++;
                     
Commands.serializeSingleMessageInBatchWithPayload(singleMessageMetadataBuilder,
                                                                       
singleMessagePayload, batchBuffer);
diff --git 
a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java
 
b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java
index f12870b7ba..fbad47ea30 100644
--- 
a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java
+++ 
b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java
@@ -123,7 +123,9 @@ private void phaseOneLoop(RawReader reader,
                             }
                         } else {
                             String key = extractKey(m);
-                            latestForKey.put(key, id);
+                            if (key != null) {
+                                latestForKey.put(key, id);
+                            }
                         }
 
                         if (id.compareTo(lastMessageId) == 0) {
@@ -213,7 +215,9 @@ private void phaseTwoLoop(RawReader reader, MessageId to, 
Map<String, MessageId>
                         }
                     } else {
                         String key = extractKey(m);
-                        if (latestForKey.get(key).equals(id)) {
+                        if (key == null) { // pass through messages without a 
key
+                            messageToAdd = Optional.of(m);
+                        } else if (latestForKey.get(key).equals(id)) {
                             messageToAdd = Optional.of(m);
                         } else {
                             m.close();
@@ -306,7 +310,11 @@ private void phaseTwoLoop(RawReader reader, MessageId to, 
Map<String, MessageId>
     private static String extractKey(RawMessage m) {
         ByteBuf headersAndPayload = m.getHeadersAndPayload();
         MessageMetadata msgMetadata = 
Commands.parseMessageMetadata(headersAndPayload);
-        return msgMetadata.getPartitionKey();
+        if (msgMetadata.hasPartitionKey()) {
+            return msgMetadata.getPartitionKey();
+        } else {
+            return null;
+        }
     }
 
     private static class PhaseOneResult {
diff --git 
a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java 
b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java
index af39e12de3..a0f0f972e4 100644
--- 
a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java
+++ 
b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java
@@ -442,4 +442,74 @@ public void testWholeBatchCompactedOut() throws Exception {
             Assert.assertEquals(new String(message.getData()), "my-message-4");
         }
     }
+
+    @Test
+    public void testKeyLessMessagesPassThrough() throws Exception {
+        String topic = "persistent://my-property/use/my-ns/my-topic1";
+
+        // subscribe before sending anything, so that we get all messages
+        pulsarClient.newConsumer().topic(topic).subscriptionName("sub1")
+            .readCompacted(true).subscribe().close();
+
+        try (Producer producerNormal = 
pulsarClient.newProducer().topic(topic).create();
+             Producer producerBatch = 
pulsarClient.newProducer().topic(topic).maxPendingMessages(3)
+                .enableBatching(true).batchingMaxMessages(3)
+                .batchingMaxPublishDelay(1, TimeUnit.HOURS).create()) {
+            producerNormal.sendAsync(MessageBuilder.create()
+                                     
.setContent("my-message-1".getBytes()).build()).get();
+
+            producerBatch.sendAsync(MessageBuilder.create()
+                                    
.setContent("my-message-2".getBytes()).build());
+            producerBatch.sendAsync(MessageBuilder.create()
+                                    .setKey("key1")
+                                    
.setContent("my-message-3".getBytes()).build());
+            producerBatch.sendAsync(MessageBuilder.create()
+                                    .setKey("key1")
+                                    
.setContent("my-message-4".getBytes()).build()).get();
+
+            producerBatch.sendAsync(MessageBuilder.create()
+                                    .setKey("key2")
+                                    
.setContent("my-message-5".getBytes()).build());
+            producerBatch.sendAsync(MessageBuilder.create()
+                                    .setKey("key2")
+                                    
.setContent("my-message-6".getBytes()).build());
+            producerBatch.sendAsync(MessageBuilder.create()
+                                    
.setContent("my-message-7".getBytes()).build()).get();
+
+            producerNormal.sendAsync(MessageBuilder.create()
+                                     
.setContent("my-message-8".getBytes()).build()).get();
+        }
+
+        // compact the topic
+        Compactor compactor = new TwoPhaseCompactor(conf, pulsarClient, bk, 
compactionScheduler);
+        compactor.compact(topic).get();
+
+        try (Consumer consumer = pulsarClient.newConsumer().topic(topic)
+                .subscriptionName("sub1").readCompacted(true).subscribe()){
+            Message message1 = consumer.receive();
+            Assert.assertFalse(message1.hasKey());
+            Assert.assertEquals(new String(message1.getData()), 
"my-message-1");
+
+            Message message2 = consumer.receive();
+            Assert.assertFalse(message2.hasKey());
+            Assert.assertEquals(new String(message2.getData()), 
"my-message-2");
+
+            Message message3 = consumer.receive();
+            Assert.assertEquals(message3.getKey(), "key1");
+            Assert.assertEquals(new String(message3.getData()), 
"my-message-4");
+
+            Message message4 = consumer.receive();
+            Assert.assertEquals(message4.getKey(), "key2");
+            Assert.assertEquals(new String(message4.getData()), 
"my-message-6");
+
+            Message message5 = consumer.receive();
+            Assert.assertFalse(message5.hasKey());
+            Assert.assertEquals(new String(message5.getData()), 
"my-message-7");
+
+            Message message6 = consumer.receive();
+            Assert.assertFalse(message6.hasKey());
+            Assert.assertEquals(new String(message6.getData()), 
"my-message-8");
+        }
+    }
+
 }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to