You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by eo...@apache.org on 2022/08/26 07:59:14 UTC

[pulsar] branch master updated: Add messgae_id in CommandSend for Shadow Topic (#17195)

This is an automated email from the ASF dual-hosted git repository.

eolivelli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/pulsar.git


The following commit(s) were added to refs/heads/master by this push:
     new 6798f88f472 Add messgae_id in CommandSend for Shadow Topic (#17195)
6798f88f472 is described below

commit 6798f88f472897b1756260afe8947af3facef0f2
Author: JiangHaiting <ji...@apache.org>
AuthorDate: Fri Aug 26 15:59:05 2022 +0800

    Add messgae_id in CommandSend for Shadow Topic (#17195)
---
 .../client/impl/ProduceWithMessageIdTest.java      | 119 +++++++++++++++++++++
 .../org/apache/pulsar/client/impl/MessageImpl.java |   2 +-
 .../apache/pulsar/client/impl/ProducerImpl.java    |  20 +++-
 .../apache/pulsar/common/protocol/Commands.java    |  18 +++-
 pulsar-common/src/main/proto/PulsarApi.proto       |   3 +
 5 files changed, 153 insertions(+), 9 deletions(-)

diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProduceWithMessageIdTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProduceWithMessageIdTest.java
new file mode 100644
index 00000000000..e03f29677b9
--- /dev/null
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProduceWithMessageIdTest.java
@@ -0,0 +1,119 @@
+/**
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.Cleanup;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.client.api.MessageId;
+import org.apache.pulsar.client.api.MockBrokerService;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.common.api.proto.MessageMetadata;
+import org.apache.pulsar.common.protocol.Commands;
+import org.awaitility.Awaitility;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+@Test(groups = "broker-impl")
+@Slf4j
+public class ProduceWithMessageIdTest {
+    MockBrokerService mockBrokerService;
+
+    @BeforeClass(alwaysRun = true)
+    public void setup() {
+        mockBrokerService = new MockBrokerService();
+        mockBrokerService.start();
+    }
+
+    @AfterClass(alwaysRun = true)
+    public void teardown() {
+        if (mockBrokerService != null) {
+            mockBrokerService.stop();
+            mockBrokerService = null;
+        }
+    }
+
+    @Test
+    public void testSend() throws Exception {
+        long ledgerId = 123;
+        long entryId = 456;
+        mockBrokerService.setHandleSend((ctx, send, headersAndPayload) -> {
+            Assert.assertTrue(send.hasMessageId());
+            log.info("receive messageId in ServerCnx, id={}", send.getMessageId());
+            Assert.assertEquals(send.getMessageId().getLedgerId(), ledgerId);
+            Assert.assertEquals(send.getMessageId().getEntryId(), entryId);
+            ctx.writeAndFlush(
+                    Commands.newSendReceipt(send.getProducerId(), send.getSequenceId(), 0, ledgerId, entryId));
+        });
+
+        @Cleanup
+        PulsarClientImpl client = (PulsarClientImpl) PulsarClient.builder()
+                .serviceUrl(mockBrokerService.getBrokerAddress())
+                .build();
+
+        String topic = "persistent://public/default/t1";
+        ProducerImpl<byte[]> producer =
+                (ProducerImpl<byte[]>) client.newProducer().topic(topic).enableBatching(false).create();
+
+        MessageMetadata metadata = new MessageMetadata();
+        ByteBuffer buffer = ByteBuffer.wrap("data".getBytes(StandardCharsets.UTF_8));
+        MessageImpl<byte[]> msg = MessageImpl.create(metadata, buffer, Schema.BYTES, topic);
+        //set message id here.
+        msg.setMessageId(new MessageIdImpl(ledgerId, entryId, -1));
+
+        AtomicBoolean result = new AtomicBoolean(false);
+        producer.sendAsync(msg, new SendCallback() {
+            @Override
+            public void sendComplete(Exception e) {
+                log.info("sendComplete", e);
+                result.set(e == null);
+            }
+
+            @Override
+            public void addCallback(MessageImpl<?> msg, SendCallback scb) {
+
+            }
+
+            @Override
+            public SendCallback getNextSendCallback() {
+                return null;
+            }
+
+            @Override
+            public MessageImpl<?> getNextMessage() {
+                return null;
+            }
+
+            @Override
+            public CompletableFuture<MessageId> getFuture() {
+                return null;
+            }
+        });
+
+        // the result is true only if broker received right message id.
+        Awaitility.await().untilTrue(result);
+    }
+}
diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java
index 6d1d586b49b..22874cc8b2b 100644
--- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java
@@ -753,7 +753,7 @@ public class MessageImpl<T> implements Message<T> {
         return msgMetadata.hasReplicatedFrom();
     }
 
-    void setMessageId(MessageId messageId) {
+    public void setMessageId(MessageId messageId) {
         this.messageId = messageId;
     }
 
diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java
index 5b5fe8f9dc5..19dd5372589 100644
--- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java
+++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ProducerImpl.java
@@ -681,13 +681,15 @@ public class ProducerImpl<T> extends ProducerBase<T> implements TimerTask, Conne
                     : 1;
             final OpSendMsg op;
             if (msg.getSchemaState() == MessageImpl.SchemaState.Ready) {
-                ByteBufPair cmd = sendMessage(producerId, sequenceId, numMessages, msgMetadata, encryptedPayload);
+                ByteBufPair cmd = sendMessage(producerId, sequenceId, numMessages, msg.getMessageId(), msgMetadata,
+                        encryptedPayload);
                 op = OpSendMsg.create(msg, cmd, sequenceId, callback);
             } else {
                 op = OpSendMsg.create(msg, null, sequenceId, callback);
                 final MessageMetadata finalMsgMetadata = msgMetadata;
                 op.rePopulate = () -> {
-                    op.cmd = sendMessage(producerId, sequenceId, numMessages, finalMsgMetadata, encryptedPayload);
+                    op.cmd = sendMessage(producerId, sequenceId, numMessages, msg.getMessageId(), finalMsgMetadata,
+                            encryptedPayload);
                 };
             }
             op.setNumMessagesInBatch(numMessages);
@@ -813,9 +815,17 @@ public class ProducerImpl<T> extends ProducerBase<T> implements TimerTask, Conne
         }
     }
 
-    protected ByteBufPair sendMessage(long producerId, long sequenceId, int numMessages, MessageMetadata msgMetadata,
-            ByteBuf compressedPayload) {
-        return Commands.newSend(producerId, sequenceId, numMessages, getChecksumType(), msgMetadata, compressedPayload);
+    protected ByteBufPair sendMessage(long producerId, long sequenceId, int numMessages,
+                                      MessageId messageId, MessageMetadata msgMetadata,
+                                      ByteBuf compressedPayload) {
+        if (messageId instanceof MessageIdImpl) {
+            return Commands.newSend(producerId, sequenceId, numMessages, getChecksumType(),
+                    ((MessageIdImpl) messageId).getLedgerId(), ((MessageIdImpl) messageId).getEntryId(),
+                    msgMetadata, compressedPayload);
+        } else {
+            return Commands.newSend(producerId, sequenceId, numMessages, getChecksumType(), -1, -1, msgMetadata,
+                    compressedPayload);
+        }
     }
 
     protected ByteBufPair sendMessage(long producerId, long lowestSequenceId, long highestSequenceId, int numMessages,
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java
index 62243063741..95fca7f431b 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java
@@ -497,12 +497,20 @@ public class Commands {
                         DEFAULT_CONSUMER_EPOCH), metadataAndPayload);
     }
 
+    public static ByteBufPair newSend(long producerId, long sequenceId, int numMessaegs, ChecksumType checksumType,
+                                      long ledgerId, long entryId, MessageMetadata messageMetadata, ByteBuf payload) {
+        return newSend(producerId, sequenceId, -1 /* highestSequenceId */, numMessaegs,
+                messageMetadata.hasTxnidLeastBits() ? messageMetadata.getTxnidLeastBits() : -1,
+                messageMetadata.hasTxnidMostBits() ? messageMetadata.getTxnidMostBits() : -1,
+                checksumType, ledgerId, entryId, messageMetadata, payload);
+    }
+
     public static ByteBufPair newSend(long producerId, long sequenceId, int numMessaegs, ChecksumType checksumType,
                                       MessageMetadata messageMetadata, ByteBuf payload) {
         return newSend(producerId, sequenceId, -1 /* highestSequenceId */, numMessaegs,
                 messageMetadata.hasTxnidLeastBits() ? messageMetadata.getTxnidLeastBits() : -1,
                 messageMetadata.hasTxnidMostBits() ? messageMetadata.getTxnidMostBits() : -1,
-                checksumType, messageMetadata, payload);
+                checksumType, -1, -1, messageMetadata, payload);
     }
 
     public static ByteBufPair newSend(long producerId, long lowestSequenceId, long highestSequenceId, int numMessaegs,
@@ -510,12 +518,12 @@ public class Commands {
         return newSend(producerId, lowestSequenceId, highestSequenceId, numMessaegs,
                 messageMetadata.hasTxnidLeastBits() ? messageMetadata.getTxnidLeastBits() : -1,
                 messageMetadata.hasTxnidMostBits() ? messageMetadata.getTxnidMostBits() : -1,
-                checksumType, messageMetadata, payload);
+                checksumType, -1, -1, messageMetadata, payload);
     }
 
     public static ByteBufPair newSend(long producerId, long sequenceId, long highestSequenceId, int numMessages,
                                       long txnIdLeastBits, long txnIdMostBits, ChecksumType checksumType,
-            MessageMetadata messageData, ByteBuf payload) {
+                                      long ledgerId, long entryId, MessageMetadata messageData, ByteBuf payload) {
         BaseCommand cmd = localCmd(Type.SEND);
         CommandSend send = cmd.setSend()
                 .setProducerId(producerId)
@@ -540,6 +548,10 @@ public class Commands {
             send.setMarker(true);
         }
 
+        if (ledgerId >= 0 && entryId >= 0) {
+            send.setMessageId().setLedgerId(ledgerId).setEntryId(entryId);
+        }
+
         return serializeCommandSendWithSize(cmd, checksumType, messageData, payload);
     }
 
diff --git a/pulsar-common/src/main/proto/PulsarApi.proto b/pulsar-common/src/main/proto/PulsarApi.proto
index 03cc6cc54da..7be65224f0c 100644
--- a/pulsar-common/src/main/proto/PulsarApi.proto
+++ b/pulsar-common/src/main/proto/PulsarApi.proto
@@ -518,6 +518,9 @@ message CommandSend {
 
     // Specify if the message being published is a Pulsar marker or not
     optional bool marker = 8 [default = false];
+
+    // Message id of this message, currently is used in replicator for shadow topic.
+    optional MessageIdData message_id = 9;
 }
 
 message CommandSendReceipt {