From 339c5e25cd7451bffc2c692be00f9c057c5c0749 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Fri, 24 Sep 2021 19:53:33 +0800 Subject: [PATCH 01/29] Add MessagePayload classes --- .../pulsar/client/api/MessagePayload.java | 37 ++++++++ .../client/impl/MessagePayloadImpl.java | 71 ++++++++++++++ .../client/impl/MessagePayloadUtils.java | 38 ++++++++ .../pulsar/client/api/MessagePayloadTest.java | 94 +++++++++++++++++++ 4 files changed, 240 insertions(+) create mode 100644 pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessagePayload.java create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadImpl.java create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadUtils.java create mode 100644 pulsar-client/src/test/java/org/apache/pulsar/client/api/MessagePayloadTest.java diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessagePayload.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessagePayload.java new file mode 100644 index 0000000000000..1e971a7ce09c1 --- /dev/null +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessagePayload.java @@ -0,0 +1,37 @@ +/** + * 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.api; + +/** + * The abstraction of a message's payload. + */ +public interface MessagePayload { + + /** + * Recycle this object if it's allocated from a memory pool. + */ + void recycle(); + + /** + * Copy the bytes of the payload into the byte array. + * + * @return the byte array that is filled with the readable bytes of the payload, it should not be null + */ + byte[] copiedBuffer(); +} \ No newline at end of file diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadImpl.java new file mode 100644 index 0000000000000..f8f106cfda556 --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadImpl.java @@ -0,0 +1,71 @@ +/** + * 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 io.netty.buffer.ByteBuf; +import io.netty.util.Recycler; +import lombok.Getter; +import lombok.NonNull; +import org.apache.pulsar.client.api.MessagePayload; + +/** + * A wrapper of {@link ByteBuf} that implements {@link MessagePayload}. + */ +public class MessagePayloadImpl implements MessagePayload { + + private static final Recycler RECYCLER = new Recycler() { + @Override + protected MessagePayloadImpl newObject(Handle handle) { + return new MessagePayloadImpl(handle); + } + }; + private static final byte[] EMPTY_BYTES = new byte[0]; + + private final Recycler.Handle recyclerHandle; + @Getter + private ByteBuf byteBuf; + + public static MessagePayloadImpl create(@NonNull final ByteBuf byteBuf) { + final MessagePayloadImpl payload = RECYCLER.get(); + payload.byteBuf = byteBuf; + return payload; + } + + private MessagePayloadImpl(final Recycler.Handle handle) { + this.recyclerHandle = handle; + } + + @Override + public void recycle() { + byteBuf = null; + recyclerHandle.recycle(this); + } + + @Override + public byte[] copiedBuffer() { + final int readable = byteBuf.readableBytes(); + if (readable > 0) { + final byte[] bytes = new byte[readable]; + byteBuf.getBytes(byteBuf.readerIndex(), bytes); + return bytes; + } else { + return EMPTY_BYTES; + } + } +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadUtils.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadUtils.java new file mode 100644 index 0000000000000..a20271a19553c --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadUtils.java @@ -0,0 +1,38 @@ +/** + * 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 io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import org.apache.pulsar.client.api.MessagePayload; + +public class MessagePayloadUtils { + + public static ByteBuf convertToByteBuf(final MessagePayload payload) { + try { + if (payload instanceof MessagePayloadImpl) { + return ((MessagePayloadImpl) payload).getByteBuf(); + } else { + return Unpooled.wrappedBuffer(payload.copiedBuffer()); + } + } finally { + payload.recycle(); + } + } +} diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/api/MessagePayloadTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/api/MessagePayloadTest.java new file mode 100644 index 0000000000000..f9442df871440 --- /dev/null +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/api/MessagePayloadTest.java @@ -0,0 +1,94 @@ +/** + * 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.api; + +import io.netty.buffer.ByteBuf; +import java.nio.ByteBuffer; +import org.apache.pulsar.client.impl.MessagePayloadImpl; +import org.apache.pulsar.client.impl.MessagePayloadUtils; +import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; +import org.testng.Assert; +import org.testng.annotations.Test; + +/** + * Unit test of {@link MessagePayload}. + */ +public class MessagePayloadTest { + + @Test + public void testConvertMessagePayloadImpl() { + final ByteBuf buf = PulsarByteBufAllocator.DEFAULT.buffer(1); + + final MessagePayloadImpl payload = MessagePayloadImpl.create(buf); + Assert.assertEquals(buf.refCnt(), 1); + + final ByteBuf convertedBuf = MessagePayloadUtils.convertToByteBuf(payload); + Assert.assertSame(convertedBuf, buf); + + Assert.assertNull(payload.getByteBuf()); + Assert.assertEquals(buf.refCnt(), 1); + buf.release(); + } + + @Test + public void testConvertCustomPayload() { + final ByteBuffer buffer = ByteBuffer.allocate(3); + buffer.put(new byte[]{ 0x11, 0x22, 0x33 }); + buffer.flip(); + buffer.get(); // skip 1st byte + + final ByteBuf buf = MessagePayloadUtils.convertToByteBuf(new ByteBufferPayload(buffer)); + Assert.assertEquals(buf.refCnt(), 1); + + Assert.assertEquals(buf.readableBytes(), 2); + Assert.assertEquals(buf.readByte(), 0x22); + Assert.assertEquals(buf.readByte(), 0x33); + + buf.release(); + } + + @Test + public void testConvertEmptyCustomPayload() { + final ByteBuf buf = MessagePayloadUtils.convertToByteBuf(new ByteBufferPayload(ByteBuffer.allocate(0))); + Assert.assertEquals(buf.refCnt(), 1); + Assert.assertEquals(buf.readableBytes(), 0); + buf.release(); + } + + private static class ByteBufferPayload implements MessagePayload { + + private final ByteBuffer buffer; + + public ByteBufferPayload(final ByteBuffer buffer) { + this.buffer = buffer; + } + + @Override + public void recycle() { + // No ops + } + + @Override + public byte[] copiedBuffer() { + final byte[] bytes = new byte[buffer.remaining()]; + buffer.get(bytes); + return bytes; + } + } +} From c35a0fe533ffa7f3c0cdf64fbd77057f819e595c Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Fri, 24 Sep 2021 20:40:41 +0800 Subject: [PATCH 02/29] Add MessageContext classes --- .../pulsar/client/api/EntryContext.java | 79 ++++++++ .../pulsar/client/impl/ConsumerImpl.java | 184 +++++++++++------- .../pulsar/client/impl/EntryContextImpl.java | 148 ++++++++++++++ .../pulsar/client/impl/MessageImpl.java | 3 +- 4 files changed, 342 insertions(+), 72 deletions(-) create mode 100644 pulsar-client-api/src/main/java/org/apache/pulsar/client/api/EntryContext.java create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/EntryContextImpl.java diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/EntryContext.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/EntryContext.java new file mode 100644 index 0000000000000..1cc04fc3f0636 --- /dev/null +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/EntryContext.java @@ -0,0 +1,79 @@ +/** + * 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.api; + +/** + * The context of an entry, which usually represents a message of a batch if batching is enabled. + */ +public interface EntryContext { + + /** + * Get a value associated with the given key. + * + * @param key + * @return the value associated with the key or null if the key or value doesn't exist + */ + String getProperty(String key); + + /** + * Get the number of messages. + * + * Since the message could be batched, a message could have multiple internal single messages. + * + * @return the number of internal single messages or 1 if the message is not batched. + */ + int getNumMessages(); + + /** + * Check whether the entry is a batch. + * + * @return true if the entry is a batch. + */ + boolean isBatch(); + + /** + * Create a new single message, which is an internal message of a batch. + * + * @param index the batch index + * @param numMessages the number of messages in the batch + * @param payload the message payload + * @param containMetadata whether the payload contains the single message metadata + * @param schema the schema of the batch + * @param + * @return the created single message + * @implNote The `index` and `numMessages` parameters are used to create the message id with batch index. + * If `containMetadata` is true, parse the single message metadata from the payload first. The fields of single + * message metadata will overwrite the same fields of the entry's metadata. + */ + Message newSingleMessage(int index, + int numMessages, + MessagePayload payload, + boolean containMetadata, + Schema schema); + + /** + * Create a new non-batched message. + * + * @param payload the message payload + * @param schema the schema of the message + * @param + * @return the created message + */ + Message newMessage(MessagePayload payload, Schema schema); +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 98767d189ee44..a634e9f89063e 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -48,7 +48,6 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; @@ -996,6 +995,96 @@ void activeConsumerChanged(boolean isActive) { }); } + protected boolean isBatch(MessageMetadata messageMetadata) { + // if message is not decryptable then it can't be parsed as a batch-message. so, add EncyrptionCtx to message + // and return undecrypted payload + return !isMessageUndecryptable(messageMetadata) && + (messageMetadata.hasNumMessagesInBatch() || messageMetadata.getNumMessagesInBatch() != 1); + } + + protected MessageImpl newSingleMessage(final int index, + final int numMessages, + final BrokerEntryMetadata brokerEntryMetadata, + final MessageMetadata msgMetadata, + final SingleMessageMetadata singleMessageMetadata, + final ByteBuf payload, + final MessageIdImpl messageId, + final Schema schema, + final boolean containMetadata, + final BitSetRecyclable ackBitSet, + final BatchMessageAcker acker, + final int redeliveryCount) { + if (log.isDebugEnabled()) { + log.debug("[{}] [{}] processing message num - {} in batch", subscription, consumerName, index); + } + + ByteBuf singleMessagePayload = null; + try { + if (containMetadata) { + singleMessagePayload = + Commands.deSerializeSingleMessageInBatch(payload, singleMessageMetadata, index, numMessages); + } + + if (isSameEntry(messageId) && isPriorBatchIndex(index)) { + // If we are receiving a batch message, we need to discard messages that were prior + // to the startMessageId + if (log.isDebugEnabled()) { + log.debug("[{}] [{}] Ignoring message from before the startMessageId: {}", subscription, + consumerName, startMessageId); + } + return null; + } + + if (singleMessageMetadata != null && singleMessageMetadata.isCompactedOut()) { + // message has been compacted out, so don't send to the user + return null; + } + + if (ackBitSet != null && !ackBitSet.get(index)) { + return null; + } + + BatchMessageIdImpl batchMessageIdImpl = new BatchMessageIdImpl(messageId.getLedgerId(), + messageId.getEntryId(), getPartitionIndex(), index, numMessages, acker); + + final ByteBuf payloadBuffer = (singleMessagePayload != null) ? singleMessagePayload : payload; + return MessageImpl.create(topicName.toString(), batchMessageIdImpl, + msgMetadata, singleMessageMetadata, payloadBuffer, + createEncryptionContext(msgMetadata), cnx(), schema, redeliveryCount, poolMessages + ).setBrokerEntryMetadata(brokerEntryMetadata); + } catch (IOException | IllegalStateException e) { + throw new IllegalStateException(e); + } finally { + if (singleMessagePayload != null) { + singleMessagePayload.release(); + } + } + } + + protected MessageImpl newMessage(final MessageIdImpl messageId, + final BrokerEntryMetadata brokerEntryMetadata, + final MessageMetadata messageMetadata, + final ByteBuf payload, + final Schema schema, + final int redeliveryCount) { + return MessageImpl.create(topicName.toString(), messageId, messageMetadata, payload, + createEncryptionContext(messageMetadata), cnx(), schema, redeliveryCount, poolMessages + ).setBrokerEntryMetadata(brokerEntryMetadata); + } + + private void executeNotifyCallback(final MessageImpl message) { + // Enqueue the message so that it can be retrieved when application calls receive() + // if the conf.getReceiverQueueSize() is 0 then discard message if no one is waiting for it. + // if asyncReceive is waiting then notify callback without adding to incomingMessages queue + internalPinnedExecutor.execute(() -> { + if (hasNextPendingReceive()) { + notifyPendingReceivedCallback(message, null); + } else if (enqueueMessageAndCheckBatchReceive(message) && hasPendingBatchReceive()) { + notifyPendingBatchReceivedCallBack(); + } + }); + } + void messageReceived(MessageIdData messageId, int redeliveryCount, List ackSet, ByteBuf headersAndPayload, ClientCnx cnx) { if (log.isDebugEnabled()) { log.debug("[{}][{}] Received message: {}/{}", topic, subscription, messageId.getLedgerId(), @@ -1053,17 +1142,20 @@ void messageReceived(MessageIdData messageId, int redeliveryCount, List ac // if message is not decryptable then it can't be parsed as a batch-message. so, add EncyrptionCtx to message // and return undecrypted payload + final EntryContextImpl entryContext = EntryContextImpl.get( + brokerEntryMetadata, msgMetadata, msgId, this, redeliveryCount, ackSet); if (isMessageUndecryptable || (numMessages == 1 && !msgMetadata.hasNumMessagesInBatch())) { // right now, chunked messages are only supported by non-shared subscription if (isChunkedMessage) { uncompressedPayload = processMessageChunk(uncompressedPayload, msgMetadata, msgId, messageId, cnx); if (uncompressedPayload == null) { + entryContext.recycle(); return; } } - if (isSameEntry(messageId) && isPriorEntryIndex(messageId.getEntryId())) { + if (isSameEntry(msgId) && isPriorEntryIndex(messageId.getEntryId())) { // We need to discard entries that were prior to startMessageId if (log.isDebugEnabled()) { log.debug("[{}] [{}] Ignoring message from before the startMessageId: {}", subscription, @@ -1071,30 +1163,20 @@ void messageReceived(MessageIdData messageId, int redeliveryCount, List ac } uncompressedPayload.release(); + entryContext.recycle(); return; } - final MessageImpl message = MessageImpl.create(topicName.toString(), msgId, msgMetadata, - uncompressedPayload, createEncryptionContext(msgMetadata), cnx, schema, redeliveryCount, - poolMessages); + final MessageImpl message = + newMessage(msgId, brokerEntryMetadata, msgMetadata, uncompressedPayload, schema, redeliveryCount); uncompressedPayload.release(); - message.setBrokerEntryMetadata(brokerEntryMetadata); - - // Enqueue the message so that it can be retrieved when application calls receive() - // if the conf.getReceiverQueueSize() is 0 then discard message if no one is waiting for it. - // if asyncReceive is waiting then notify callback without adding to incomingMessages queue - internalPinnedExecutor.execute(() -> { - if (deadLetterPolicy != null && possibleSendToDeadLetterTopicMessages != null && - redeliveryCount >= deadLetterPolicy.getMaxRedeliverCount()) { - possibleSendToDeadLetterTopicMessages.put((MessageIdImpl) message.getMessageId(), - Collections.singletonList(message)); - } - if (hasNextPendingReceive()) { - notifyPendingReceivedCallback(message, null); - } else if (enqueueMessageAndCheckBatchReceive(message) && hasPendingBatchReceive()) { - notifyPendingBatchReceivedCallBack(); - } - }); + + if (deadLetterPolicy != null && possibleSendToDeadLetterTopicMessages != null && + redeliveryCount >= deadLetterPolicy.getMaxRedeliverCount()) { + possibleSendToDeadLetterTopicMessages.put((MessageIdImpl) message.getMessageId(), + Collections.singletonList(message)); + } + executeNotifyCallback(message); } else { // handle batch message enqueuing; uncompressed payload has all messages in batch receiveIndividualMessagesFromBatch(brokerEntryMetadata, msgMetadata, redeliveryCount, ackSet, uncompressedPayload, messageId, cnx); @@ -1268,63 +1350,23 @@ void receiveIndividualMessagesFromBatch(BrokerEntryMetadata brokerEntryMetadata, int skippedMessages = 0; try { for (int i = 0; i < batchSize; ++i) { - if (log.isDebugEnabled()) { - log.debug("[{}] [{}] processing message num - {} in batch", subscription, consumerName, i); - } - - ByteBuf singleMessagePayload = Commands.deSerializeSingleMessageInBatch(uncompressedPayload, - singleMessageMetadata, i, batchSize); - - if (isSameEntry(messageId) && isPriorBatchIndex(i)) { - // If we are receiving a batch message, we need to discard messages that were prior - // to the startMessageId - if (log.isDebugEnabled()) { - log.debug("[{}] [{}] Ignoring message from before the startMessageId: {}", subscription, - consumerName, startMessageId); - } - singleMessagePayload.release(); - - ++skippedMessages; + final MessageImpl message = newSingleMessage(i, batchSize, brokerEntryMetadata, msgMetadata, + singleMessageMetadata, uncompressedPayload, batchMessage, schema, true, ackBitSet, acker, + redeliveryCount); + if (message == null) { + skippedMessages++; continue; } - - if (singleMessageMetadata.isCompactedOut()) { - // message has been compacted out, so don't send to the user - singleMessagePayload.release(); - - ++skippedMessages; - continue; - } - - if (ackBitSet != null && !ackBitSet.get(i)) { - singleMessagePayload.release(); - ++skippedMessages; - continue; - } - - BatchMessageIdImpl batchMessageIdImpl = new BatchMessageIdImpl(messageId.getLedgerId(), - messageId.getEntryId(), getPartitionIndex(), i, batchSize, acker); - final MessageImpl message = MessageImpl.create(topicName.toString(), batchMessageIdImpl, - msgMetadata, singleMessageMetadata, singleMessagePayload, - createEncryptionContext(msgMetadata), cnx, schema, redeliveryCount, poolMessages); - message.setBrokerEntryMetadata(brokerEntryMetadata); if (possibleToDeadLetter != null) { possibleToDeadLetter.add(message); } - internalPinnedExecutor.execute(() -> { - if (hasNextPendingReceive()) { - notifyPendingReceivedCallback(message, null); - } else if (enqueueMessageAndCheckBatchReceive(message) && hasPendingBatchReceive()) { - notifyPendingBatchReceivedCallBack(); - } - singleMessagePayload.release(); - }); + executeNotifyCallback(message); } if (ackBitSet != null) { ackBitSet.recycle(); } - } catch (IOException e) { - log.warn("[{}] [{}] unable to obtain message in batch", subscription, consumerName); + } catch (IllegalStateException e) { + log.warn("[{}] [{}] unable to obtain message in batch", subscription, consumerName, e); discardCorruptedMessage(messageId, cnx, ValidationError.BatchDeSerializeError); } @@ -1350,7 +1392,7 @@ private boolean isPriorBatchIndex(long idx) { return resetIncludeHead ? idx < startMessageId.getBatchIndex() : idx <= startMessageId.getBatchIndex(); } - private boolean isSameEntry(MessageIdData messageId) { + private boolean isSameEntry(MessageIdImpl messageId) { return startMessageId != null && messageId.getLedgerId() == startMessageId.getLedgerId() && messageId.getEntryId() == startMessageId.getEntryId(); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/EntryContextImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/EntryContextImpl.java new file mode 100644 index 0000000000000..1b6f1c2c59540 --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/EntryContextImpl.java @@ -0,0 +1,148 @@ +/** + * 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 io.netty.buffer.ByteBuf; +import io.netty.util.Recycler; +import java.util.List; +import lombok.NonNull; +import org.apache.pulsar.client.api.EntryContext; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessagePayload; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.api.proto.BrokerEntryMetadata; +import org.apache.pulsar.common.api.proto.KeyValue; +import org.apache.pulsar.common.api.proto.MessageMetadata; +import org.apache.pulsar.common.api.proto.SingleMessageMetadata; +import org.apache.pulsar.common.util.SafeCollectionUtils; +import org.apache.pulsar.common.util.collections.BitSetRecyclable; + +public class EntryContextImpl implements EntryContext { + + private static final Recycler RECYCLER = new Recycler() { + @Override + protected EntryContextImpl newObject(Handle handle) { + return new EntryContextImpl(handle); + } + }; + + private final Recycler.Handle recyclerHandle; + private BrokerEntryMetadata brokerEntryMetadata; + private MessageMetadata messageMetadata; + private SingleMessageMetadata singleMessageMetadata; + private MessageIdImpl messageId; + private ConsumerImpl consumer; + private int redeliveryCount; + private BatchMessageAcker acker; + private BitSetRecyclable ackBitSet; + + private EntryContextImpl(final Recycler.Handle handle) { + this.recyclerHandle = handle; + } + + public static EntryContextImpl get(final BrokerEntryMetadata brokerEntryMetadata, + @NonNull final MessageMetadata messageMetadata, + @NonNull final MessageIdImpl messageId, + @NonNull final ConsumerImpl consumer, + final int redeliveryCount, + final List ackSet) { + final EntryContextImpl context = RECYCLER.get(); + context.brokerEntryMetadata = brokerEntryMetadata; + context.messageMetadata = messageMetadata; + context.singleMessageMetadata = new SingleMessageMetadata(); + context.messageId = messageId; + context.consumer = consumer; + context.redeliveryCount = redeliveryCount; + context.acker = BatchMessageAcker.newAcker(context.getNumMessages()); + context.ackBitSet = (ackSet != null && ackSet.size() > 0) + ? BitSetRecyclable.valueOf(SafeCollectionUtils.longListToArray(ackSet)) + : null; + return context; + } + + public void recycle() { + brokerEntryMetadata = null; + messageMetadata = null; + singleMessageMetadata = null; + messageId = null; + consumer = null; + redeliveryCount = 0; + acker = null; + if (ackBitSet != null) { + ackBitSet.recycle(); + ackBitSet = null; + } + } + + @Override + public String getProperty(String key) { + for (KeyValue keyValue : messageMetadata.getPropertiesList()) { + if (keyValue.hasKey() && keyValue.getKey().equals(key)) { + return keyValue.getValue(); + } + } + return null; + } + + @Override + public int getNumMessages() { + return messageMetadata.getNumMessagesInBatch(); + } + + @Override + public boolean isBatch() { + return consumer.isBatch(messageMetadata); + } + + @Override + public Message newSingleMessage(int index, + int numMessages, + MessagePayload payload, + boolean containMetadata, + Schema schema) { + final ByteBuf payloadBuffer = MessagePayloadUtils.convertToByteBuf(payload); + try { + return consumer.newSingleMessage(index, + numMessages, + brokerEntryMetadata, + messageMetadata, + singleMessageMetadata, + payloadBuffer, + messageId, + schema, + containMetadata, + ackBitSet, + acker, + redeliveryCount); + } finally { + payloadBuffer.release(); + } + } + + @Override + public Message newMessage(MessagePayload payload, Schema schema) { + final ByteBuf payloadBuffer = MessagePayloadUtils.convertToByteBuf(payload); + try { + return consumer.newMessage( + messageId, brokerEntryMetadata, messageMetadata, payloadBuffer, schema, redeliveryCount); + } finally { + payloadBuffer.release(); + } + } +} 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 67c176cfe63bf..a00cb35071f68 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 @@ -625,8 +625,9 @@ public BrokerEntryMetadata getBrokerEntryMetadata() { return brokerEntryMetadata; } - public void setBrokerEntryMetadata(BrokerEntryMetadata brokerEntryMetadata) { + public MessageImpl setBrokerEntryMetadata(BrokerEntryMetadata brokerEntryMetadata) { this.brokerEntryMetadata = brokerEntryMetadata; + return this; } public ClientCnx getCnx() { From 837978c8efdb6c9cb85236521fb3ed5fa48c0fd5 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Fri, 24 Sep 2021 21:11:18 +0800 Subject: [PATCH 03/29] Add PayloadConverter interface --- .../pulsar/client/api/ConsumerBuilder.java | 11 ++++ .../pulsar/client/api/PayloadConverter.java | 35 +++++++++++ .../client/impl/ConsumerBuilderImpl.java | 7 +++ .../pulsar/client/impl/ConsumerImpl.java | 58 +++++++++++++++++-- .../impl/conf/ConsumerConfigurationData.java | 4 ++ 5 files changed, 111 insertions(+), 4 deletions(-) create mode 100644 pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadConverter.java diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java index 1038ba9ea8af6..e3584ddbd170a 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java @@ -741,4 +741,15 @@ public interface ConsumerBuilder extends Cloneable { * corruption, deserialization error, etc.). */ ConsumerBuilder poolMessages(boolean poolMessages); + + /** + * Set the payload converter that might convert the message payload to the format that Pulsar consumer can recognize + * before creating {@link Message}s from an entry. + * + * Default: null + * + * @param payloadConverter + * @return + */ + ConsumerBuilder payloadConverter(PayloadConverter payloadConverter); } diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadConverter.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadConverter.java new file mode 100644 index 0000000000000..b4a4317af2a8f --- /dev/null +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadConverter.java @@ -0,0 +1,35 @@ +/** + * 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.api; + +public interface PayloadConverter { + + /** + * Convert the payload to iterable messages. + * + * @param context the message context that contains the message format information and methods to create a message + * @param payload the payload whose underlying buffer is a Netty ByteBuf + * @param schema the message's schema + * @param + * @return iterable messages + * @implNote During the iteration, the message could be null, which means it will be skipped in Pulsar consumer. + * The iteration could also be interrupted by CorruptedMessageException. + */ + Iterable> convert(EntryContext context, MessagePayload payload, Schema schema); +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java index dba18d93283d6..5a97d7232926d 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java @@ -44,6 +44,7 @@ import org.apache.pulsar.client.api.KeySharedPolicy; import org.apache.pulsar.client.api.MessageCrypto; import org.apache.pulsar.client.api.MessageListener; +import org.apache.pulsar.client.api.PayloadConverter; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.PulsarClientException.InvalidConfigurationException; import org.apache.pulsar.client.api.RegexSubscriptionMode; @@ -479,4 +480,10 @@ public ConsumerBuilder poolMessages(boolean poolMessages) { conf.setPoolMessages(poolMessages); return this; } + + @Override + public ConsumerBuilder payloadConverter(PayloadConverter payloadConverter) { + conf.setPayloadConverter(payloadConverter); + return this; + } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index a634e9f89063e..57d06e05853f5 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -66,6 +66,7 @@ import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageCrypto; import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.MessagePayload; import org.apache.pulsar.client.api.Messages; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClientException; @@ -1085,6 +1086,41 @@ private void executeNotifyCallback(final MessageImpl message) { }); } + private void consumeMessagesFromConverter(final BrokerEntryMetadata brokerEntryMetadata, + final MessageMetadata messageMetadata, + final ByteBuf byteBuf, + final MessageIdImpl messageId, + final Schema schema, + final int redeliveryCount, + final List ackSet) { + final EntryContextImpl entryContext = EntryContextImpl.get( + brokerEntryMetadata, messageMetadata, messageId, this, redeliveryCount, ackSet); + final MessagePayload payload = MessagePayloadImpl.create(byteBuf); + int skippedMessages = 0; + try { + for (Message message : conf.getPayloadConverter().convert(entryContext, payload, schema)) { + if (message == null) { + skippedMessages++; + continue; + } + executeNotifyCallback((MessageImpl) message); + } + } catch (IllegalStateException e) { + log.warn("[{}] [{}] unable to obtain message in batch", subscription, consumerName, e); + discardCorruptedMessage(messageId, cnx(), ValidationError.BatchDeSerializeError); + } finally { + byteBuf.release(); + entryContext.recycle(); + } + + if (skippedMessages > 0) { + increaseAvailablePermits(cnx(), skippedMessages); + } + + internalPinnedExecutor.execute(() + -> tryTriggerListener()); + } + void messageReceived(MessageIdData messageId, int redeliveryCount, List ackSet, ByteBuf headersAndPayload, ClientCnx cnx) { if (log.isDebugEnabled()) { log.debug("[{}][{}] Received message: {}/{}", topic, subscription, messageId.getLedgerId(), @@ -1140,17 +1176,21 @@ void messageReceived(MessageIdData messageId, int redeliveryCount, List ac return; } + if (conf.getPayloadConverter() != null) { + // uncompressedPayload is released in this method so we don't need to call release() again + consumeMessagesFromConverter( + brokerEntryMetadata, msgMetadata, uncompressedPayload, msgId, schema, redeliveryCount, ackSet); + return; + } + // if message is not decryptable then it can't be parsed as a batch-message. so, add EncyrptionCtx to message // and return undecrypted payload - final EntryContextImpl entryContext = EntryContextImpl.get( - brokerEntryMetadata, msgMetadata, msgId, this, redeliveryCount, ackSet); if (isMessageUndecryptable || (numMessages == 1 && !msgMetadata.hasNumMessagesInBatch())) { // right now, chunked messages are only supported by non-shared subscription if (isChunkedMessage) { uncompressedPayload = processMessageChunk(uncompressedPayload, msgMetadata, msgId, messageId, cnx); if (uncompressedPayload == null) { - entryContext.recycle(); return; } } @@ -1163,7 +1203,6 @@ void messageReceived(MessageIdData messageId, int redeliveryCount, List ac } uncompressedPayload.release(); - entryContext.recycle(); return; } @@ -1586,6 +1625,17 @@ private boolean verifyChecksum(ByteBuf headersAndPayload, MessageIdData messageI return true; } + private void discardCorruptedMessage(MessageIdImpl messageId, ClientCnx currentCnx, + ValidationError validationError) { + log.error("[{}][{}] Discarding corrupted message at {}:{}", topic, subscription, messageId.getLedgerId(), + messageId.getEntryId()); + ByteBuf cmd = Commands.newAck(consumerId, messageId.getLedgerId(), messageId.getEntryId(), null, AckType.Individual, + validationError, Collections.emptyMap(), -1); + currentCnx.ctx().writeAndFlush(cmd, currentCnx.ctx().voidPromise()); + increaseAvailablePermits(currentCnx); + stats.incrementNumReceiveFailed(); + } + private void discardCorruptedMessage(MessageIdData messageId, ClientCnx currentCnx, ValidationError validationError) { log.error("[{}][{}] Discarding corrupted message at {}:{}", topic, subscription, messageId.getLedgerId(), diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java index a39ac21164bf4..f3cdcc589dffe 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java @@ -42,6 +42,7 @@ import org.apache.pulsar.client.api.KeySharedPolicy; import org.apache.pulsar.client.api.MessageCrypto; import org.apache.pulsar.client.api.MessageListener; +import org.apache.pulsar.client.api.PayloadConverter; import org.apache.pulsar.client.api.RegexSubscriptionMode; import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.api.SubscriptionMode; @@ -149,6 +150,9 @@ public int getMaxPendingChuckedMessage() { private boolean poolMessages = false; + @JsonIgnore + private transient PayloadConverter payloadConverter = null; + public void setAutoUpdatePartitionsIntervalSeconds(int interval, TimeUnit timeUnit) { checkArgument(interval > 0, "interval needs to be > 0"); this.autoUpdatePartitionsIntervalSeconds = timeUnit.toSeconds(interval); From cfabc260996f3ddce722ec7f7717fdfec3b97413 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Fri, 24 Sep 2021 22:53:34 +0800 Subject: [PATCH 04/29] Fix refCnt error --- .../pulsar/client/impl/MessagePayloadUtils.java | 12 ++++-------- .../apache/pulsar/client/api/MessagePayloadTest.java | 4 ++-- 2 files changed, 6 insertions(+), 10 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadUtils.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadUtils.java index a20271a19553c..64faffe3da826 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadUtils.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadUtils.java @@ -25,14 +25,10 @@ public class MessagePayloadUtils { public static ByteBuf convertToByteBuf(final MessagePayload payload) { - try { - if (payload instanceof MessagePayloadImpl) { - return ((MessagePayloadImpl) payload).getByteBuf(); - } else { - return Unpooled.wrappedBuffer(payload.copiedBuffer()); - } - } finally { - payload.recycle(); + if (payload instanceof MessagePayloadImpl) { + return ((MessagePayloadImpl) payload).getByteBuf().retain(); + } else { + return Unpooled.wrappedBuffer(payload.copiedBuffer()); } } } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/api/MessagePayloadTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/api/MessagePayloadTest.java index f9442df871440..0ddb85fbb7f4a 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/api/MessagePayloadTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/api/MessagePayloadTest.java @@ -41,8 +41,8 @@ public void testConvertMessagePayloadImpl() { final ByteBuf convertedBuf = MessagePayloadUtils.convertToByteBuf(payload); Assert.assertSame(convertedBuf, buf); - Assert.assertNull(payload.getByteBuf()); - Assert.assertEquals(buf.refCnt(), 1); + Assert.assertEquals(buf.refCnt(), 2); + buf.release(); buf.release(); } From db9f84b0a761a684ae16f6dfd2bf9d108e6deef5 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Fri, 24 Sep 2021 23:08:00 +0800 Subject: [PATCH 05/29] Add tests for default converter --- .../converter/PayloadConverterTest.java | 139 ++++++++++++++++++ .../client/impl/DefaultPayloadConverter.java | 76 ++++++++++ 2 files changed, 215 insertions(+) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/DefaultPayloadConverter.java diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java new file mode 100644 index 0000000000000..a198b1de77725 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java @@ -0,0 +1,139 @@ +/** + * 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.converter; + +import com.google.common.collect.Sets; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.TimeUnit; +import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessageRouter; +import org.apache.pulsar.client.api.MessageRoutingMode; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; +import org.apache.pulsar.client.api.TopicMetadata; +import org.apache.pulsar.client.impl.DefaultPayloadConverter; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +/** + * Tests for {@link org.apache.pulsar.client.api.PayloadConverter}. + */ +@Slf4j +public class PayloadConverterTest extends ProducerConsumerBase { + + @BeforeClass + @Override + protected void setup() throws Exception { + super.internalSetup(); + admin.clusters().createCluster("test", + ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); + admin.tenants().createTenant("public", + new TenantInfoImpl(Sets.newHashSet("appid"), Sets.newHashSet("test"))); + admin.namespaces().createNamespace("public/default", Sets.newHashSet("test")); + } + + @AfterClass + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @DataProvider + public static Object[][] config() { + return new Object[][] { + // enableBatching / numPartitions + { true, 1 }, + { false, 1 }, + { false, 3 }, + }; + } + + @Test(dataProvider = "config") + public void testDefaultConverter(boolean enableBatching, int numPartitions) throws Exception { + final String topic = "testDefaultConverter-" + enableBatching + "-" + numPartitions; + final int numMessages = 10; + final int batchingMaxMessages = 4; + final String messagePrefix = "msg-"; + + if (numPartitions > 1) { + admin.topics().createPartitionedTopic(topic, numPartitions); + } + + @Cleanup + final Producer producer = pulsarClient.newProducer(Schema.STRING) + .topic(topic) + .enableBatching(enableBatching) + .batchingMaxMessages(batchingMaxMessages) + .batchingMaxPublishDelay(100, TimeUnit.MILLISECONDS) + .messageRouter(new MessageRouter() { + int i = 0; + + @Override + public int choosePartition(Message msg, TopicMetadata metadata) { + return i++ % metadata.numPartitions(); + } + }) + .create(); + for (int i = 0; i < numMessages; i++) { + final String value = messagePrefix + i; + producer.sendAsync(value).whenComplete((id, e) -> { + if (e == null) { + log.info("Send {} to {} {}", value, topic, id); + } else { + log.error("Failed to send {}: {}", value, e.getMessage()); + } + }); + } + + @Cleanup + final Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .topic(topic) + .subscriptionName("sub") + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .payloadConverter(new DefaultPayloadConverter()) + .subscribe(); + final List values = new ArrayList<>(); + for (int i = 0; i < numMessages; i++) { + final Message message = consumer.receive(1, TimeUnit.SECONDS); + Assert.assertNotNull(message); + values.add(message.getValue()); + } + + if (numPartitions > 1) { + // messages are out of order across multiple partitions + Collections.sort(values); + } + for (int i = 0; i < numMessages; i++) { + Assert.assertEquals(values.get(i), messagePrefix + i); + } + } +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/DefaultPayloadConverter.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/DefaultPayloadConverter.java new file mode 100644 index 0000000000000..44d62b24013be --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/DefaultPayloadConverter.java @@ -0,0 +1,76 @@ +/** + * 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.util.Iterator; +import org.apache.pulsar.client.api.EntryContext; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessagePayload; +import org.apache.pulsar.client.api.PayloadConverter; +import org.apache.pulsar.client.api.Schema; + +/** + * The default converter provided for users that want to define their own {@link PayloadConverter}. + */ +public class DefaultPayloadConverter implements PayloadConverter { + + @Override + public Iterable> convert(EntryContext context, MessagePayload payload, Schema schema) { + final int numMessages = context.getNumMessages(); + + if (context.isBatch()) { + return () -> new Iterator>() { + int index = 0; + + @Override + public boolean hasNext() { + final boolean result = (index < numMessages); + if (!result) { + payload.recycle(); + } + return result; + } + + @Override + public Message next() { + index++; + return context.newSingleMessage(index, numMessages, payload, true, schema); + } + }; + } else { + return () -> new Iterator>() { + boolean first = true; + + @Override + public boolean hasNext() { + if (!first) { + payload.recycle(); + } + return first; + } + + @Override + public Message next() { + first = false; + return context.newMessage(payload, schema); + } + }; + } + } +} From 9630f418dc47ebe42e69d8a7093806189df99ea6 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Fri, 24 Sep 2021 23:42:35 +0800 Subject: [PATCH 06/29] Add tests for custom batch format --- .../converter/CustomBatchConverter.java | 77 +++++++++++ .../client/converter/CustomBatchFormat.java | 122 ++++++++++++++++++ .../converter/PayloadConverterTest.java | 62 ++++++++- 3 files changed, 260 insertions(+), 1 deletion(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchFormat.java diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java new file mode 100644 index 0000000000000..0f727d5daa06d --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java @@ -0,0 +1,77 @@ +/** + * 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.converter; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.util.ReferenceCounted; +import org.apache.pulsar.client.api.EntryContext; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessagePayload; +import org.apache.pulsar.client.api.PayloadConverter; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.DefaultPayloadConverter; +import org.apache.pulsar.client.impl.MessagePayloadImpl; +import org.apache.pulsar.client.impl.MessagePayloadUtils; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +public class CustomBatchConverter implements PayloadConverter { + + private static final PayloadConverter DEFAULT = new DefaultPayloadConverter(); + + @Override + public Iterable> convert(EntryContext context, MessagePayload payload, Schema schema) { + final String value = context.getProperty(CustomBatchFormat.KEY); + if (value == null || !value.equals(CustomBatchFormat.VALUE)) { + return DEFAULT.convert(context, payload, schema); + } + + final ByteBuf buf = MessagePayloadUtils.convertToByteBuf(payload); + final CustomBatchFormat.StringIterable strings = CustomBatchFormat.deserialize(buf); + final Iterator stringIterator = strings.iterator(); + final int numMessages = context.getNumMessages(); + final List bufList = new ArrayList<>(); + + return () -> new Iterator>() { + int index = 0; + + @Override + public boolean hasNext() { + final boolean result = stringIterator.hasNext(); + if (!result) { + bufList.forEach(ReferenceCounted::release); + buf.release(); + } + return result; + } + + @Override + public Message next() { + final String value = stringIterator.next(); + final ByteBuf valueBuf = Unpooled.wrappedBuffer(Schema.STRING.encode(value)); + bufList.add(valueBuf); + return context.newSingleMessage( + index++, numMessages, MessagePayloadImpl.create(valueBuf), false, schema); + } + }; + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchFormat.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchFormat.java new file mode 100644 index 0000000000000..03967e71223f8 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchFormat.java @@ -0,0 +1,122 @@ +/** + * 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.converter; + +import io.netty.buffer.ByteBuf; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; +import org.testng.Assert; +import org.testng.annotations.Test; + +/** + * A batch message whose format is customized. + * + * 1. First 2 bytes represent the number of messages. + * 2. Each message is a string, whose format is + * 1. First 2 bytes represent the length `N`. + * 2. Followed N bytes are the bytes of the string. + */ +public class CustomBatchFormat { + + public static final String KEY = "entry.format"; + public static final String VALUE = "custom"; + + public static ByteBuf serialize(Iterable strings) { + final ByteBuf buf = PulsarByteBufAllocator.DEFAULT.buffer(1024); + buf.writeShort(0); + short numMessages = 0; + for (String s : strings) { + writeString(buf, s); + numMessages++; + } + buf.setShort(0, numMessages); + return buf; + } + + public static abstract class StringIterable implements Iterable { + + public abstract int size(); + }; + + public static StringIterable deserialize(final ByteBuf buf) { + final int numMessages = buf.readShort(); + return new StringIterable() { + @Override + public int size() { + return numMessages; + } + + @Override + public Iterator iterator() { + return new Iterator() { + int index = 0; + + @Override + public boolean hasNext() { + return index < numMessages; + } + + @Override + public String next() { + index++; + return readString(buf); + } + }; + } + }; + } + + private static void writeString(final ByteBuf buf, final String s) { + final byte[] bytes = s.getBytes(StandardCharsets.UTF_8); + buf.writeShort(bytes.length); + buf.writeBytes(bytes); + } + + private static String readString(final ByteBuf buf) { + final short length = buf.readShort(); + final byte[] bytes = new byte[length]; + buf.readBytes(bytes); + return new String(bytes, StandardCharsets.UTF_8); + } + + @Test + public void testMultipleStrings() { + final List> inputs = new ArrayList<>(); + inputs.add(Collections.emptyList()); + inputs.add(Collections.singletonList("java")); + inputs.add(Arrays.asList("hello", "world", "java")); + + for (List input : inputs) { + final ByteBuf buf = serialize(input); + final List parsedTokens = new ArrayList<>(); + deserialize(buf).forEach(parsedTokens::add); + + Assert.assertEquals(parsedTokens, input); + Assert.assertEquals(parsedTokens.size(), input.size()); + + Assert.assertEquals(buf.refCnt(), 1); + buf.release(); + } + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java index a198b1de77725..3ff569ced8690 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java @@ -20,24 +20,27 @@ import com.google.common.collect.Sets; +import io.netty.buffer.ByteBuf; import java.util.ArrayList; import java.util.Collections; import java.util.List; 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.MessageRouter; -import org.apache.pulsar.client.api.MessageRoutingMode; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerConsumerBase; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.api.TopicMetadata; import org.apache.pulsar.client.impl.DefaultPayloadConverter; +import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.protocol.Commands; import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; @@ -136,4 +139,61 @@ public int choosePartition(Message msg, TopicMetadata metadata) { Assert.assertEquals(values.get(i), messagePrefix + i); } } + + @Test + public void testCustomConverter() throws Exception { + final String topic = "persistent://public/default/testCustomConverter"; + final int numMessages = 10; + final int batchingMaxMessages = 4; + final String messagePrefix = "msg-"; + + @Cleanup + final Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .topic(topic) + .subscriptionName("sub") + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .payloadConverter(new CustomBatchConverter()) + .subscribe(); + + final PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(topic).get().orElse(null); + Assert.assertNotNull(persistentTopic); + + final List values = new ArrayList<>(); + for (int i = 0; i < numMessages; i++) { + final String value = messagePrefix + i; + values.add(value); + if (values.size() >= batchingMaxMessages || i == numMessages - 1) { + final ByteBuf valueBuf = CustomBatchFormat.serialize(values); + values.clear(); + final ByteBuf headerAndPayload = Commands.serializeMetadataAndPayload(Commands.ChecksumType.None, + createCustomMetadata(), valueBuf); + valueBuf.release(); + persistentTopic.publishMessage(headerAndPayload, (e, ledgerId, entryId) -> { + if (e == null) { + log.info("Send {} to {} ({}, {})", value, topic, ledgerId, entryId); + } else { + log.error("Failed to send {}: {}", value, e.getMessage()); + } + }); + } + } + + for (int i = 0; i < numMessages; i++) { + final Message message = consumer.receive(1, TimeUnit.SECONDS); + Assert.assertNotNull(message); + Assert.assertEquals(message.getValue(), messagePrefix + i); + } + } + + private static MessageMetadata createCustomMetadata() { + final MessageMetadata messageMetadata = new MessageMetadata(); + // Here are required fields + messageMetadata.setProducerName("producer"); + messageMetadata.setSequenceId(0L); + messageMetadata.setPublishTime(0L); + // Add the property to identify the message format + messageMetadata.addProperty().setKey(CustomBatchFormat.KEY).setValue(CustomBatchFormat.VALUE); + return messageMetadata; + } } From 243d76a6ca1220f2e7409e141c51a4d6e7ed9b8e Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Fri, 24 Sep 2021 23:47:24 +0800 Subject: [PATCH 07/29] Move default converter to tests --- .../converter/CustomBatchConverter.java | 8 +-- .../converter}/DefaultPayloadConverter.java | 57 +++++++------------ .../converter/PayloadConverterTest.java | 2 - 3 files changed, 23 insertions(+), 44 deletions(-) rename {pulsar-client/src/main/java/org/apache/pulsar/client/impl => pulsar-broker/src/test/java/org/apache/pulsar/client/converter}/DefaultPayloadConverter.java (55%) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java index 0f727d5daa06d..11338ff8016dc 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java @@ -21,19 +21,17 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import io.netty.util.ReferenceCounted; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; import org.apache.pulsar.client.api.EntryContext; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessagePayload; import org.apache.pulsar.client.api.PayloadConverter; import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.client.impl.DefaultPayloadConverter; import org.apache.pulsar.client.impl.MessagePayloadImpl; import org.apache.pulsar.client.impl.MessagePayloadUtils; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; - public class CustomBatchConverter implements PayloadConverter { private static final PayloadConverter DEFAULT = new DefaultPayloadConverter(); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/DefaultPayloadConverter.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/DefaultPayloadConverter.java similarity index 55% rename from pulsar-client/src/main/java/org/apache/pulsar/client/impl/DefaultPayloadConverter.java rename to pulsar-broker/src/test/java/org/apache/pulsar/client/converter/DefaultPayloadConverter.java index 44d62b24013be..85ee45aa7621b 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/DefaultPayloadConverter.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/DefaultPayloadConverter.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.client.impl; +package org.apache.pulsar.client.converter; import java.util.Iterator; import org.apache.pulsar.client.api.EntryContext; @@ -33,44 +33,27 @@ public class DefaultPayloadConverter implements PayloadConverter { @Override public Iterable> convert(EntryContext context, MessagePayload payload, Schema schema) { final int numMessages = context.getNumMessages(); + final boolean isBatch = context.isBatch(); - if (context.isBatch()) { - return () -> new Iterator>() { - int index = 0; + return () -> new Iterator>() { + int index = 0; - @Override - public boolean hasNext() { - final boolean result = (index < numMessages); - if (!result) { - payload.recycle(); - } - return result; + @Override + public boolean hasNext() { + final boolean result = (index < numMessages); + if (!result) { + payload.recycle(); } - - @Override - public Message next() { - index++; - return context.newSingleMessage(index, numMessages, payload, true, schema); - } - }; - } else { - return () -> new Iterator>() { - boolean first = true; - - @Override - public boolean hasNext() { - if (!first) { - payload.recycle(); - } - return first; - } - - @Override - public Message next() { - first = false; - return context.newMessage(payload, schema); - } - }; - } + return result; + } + + @Override + public Message next() { + index++; + return isBatch + ? context.newSingleMessage(index, numMessages, payload, true, schema) + : context.newMessage(payload, schema); + } + }; } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java index 3ff569ced8690..08da957992f33 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java @@ -19,7 +19,6 @@ package org.apache.pulsar.client.converter; import com.google.common.collect.Sets; - import io.netty.buffer.ByteBuf; import java.util.ArrayList; import java.util.Collections; @@ -36,7 +35,6 @@ import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.api.TopicMetadata; -import org.apache.pulsar.client.impl.DefaultPayloadConverter; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfoImpl; From f5a0276c486a11f0b79c05650ed1b8e2b6348b35 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Fri, 24 Sep 2021 23:59:57 +0800 Subject: [PATCH 08/29] Add tests for refCnt when the iteration was stopped --- .../converter/DefaultPayloadConverter.java | 6 ++++ .../converter/PayloadConverterTest.java | 30 ++++++++++++++----- 2 files changed, 28 insertions(+), 8 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/DefaultPayloadConverter.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/DefaultPayloadConverter.java index 85ee45aa7621b..9fef0cb21beba 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/DefaultPayloadConverter.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/DefaultPayloadConverter.java @@ -18,18 +18,23 @@ */ package org.apache.pulsar.client.converter; +import lombok.Getter; import java.util.Iterator; import org.apache.pulsar.client.api.EntryContext; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessagePayload; import org.apache.pulsar.client.api.PayloadConverter; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.MessagePayloadImpl; /** * The default converter provided for users that want to define their own {@link PayloadConverter}. */ public class DefaultPayloadConverter implements PayloadConverter { + @Getter + private int totalRefCnt = 0; + @Override public Iterable> convert(EntryContext context, MessagePayload payload, Schema schema) { final int numMessages = context.getNumMessages(); @@ -42,6 +47,7 @@ public Iterable> convert(EntryContext context, MessagePayload pay public boolean hasNext() { final boolean result = (index < numMessages); if (!result) { + totalRefCnt += ((MessagePayloadImpl) payload).getByteBuf().refCnt(); payload.recycle(); } return result; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java index 08da957992f33..a57a8dc5f48c5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java @@ -71,18 +71,19 @@ protected void cleanup() throws Exception { @DataProvider public static Object[][] config() { return new Object[][] { - // enableBatching / numPartitions - { true, 1 }, - { false, 1 }, - { false, 3 }, + // numPartitions / enableBatching / batchingMaxMessages + { 1, true, 1 }, + { 1, true, 4 }, + { 1, false, 1 }, + { 3, false, 1 } }; } @Test(dataProvider = "config") - public void testDefaultConverter(boolean enableBatching, int numPartitions) throws Exception { - final String topic = "testDefaultConverter-" + enableBatching + "-" + numPartitions; + public void testDefaultConverter(int numPartitions, boolean enableBatching, int batchingMaxMessages) + throws Exception { + final String topic = "testDefaultConverter-" + numPartitions + "-" + enableBatching + "-" + batchingMaxMessages; final int numMessages = 10; - final int batchingMaxMessages = 4; final String messagePrefix = "msg-"; if (numPartitions > 1) { @@ -115,12 +116,14 @@ public int choosePartition(Message msg, TopicMetadata metadata) { }); } + final DefaultPayloadConverter converter = new DefaultPayloadConverter(); + @Cleanup final Consumer consumer = pulsarClient.newConsumer(Schema.STRING) .topic(topic) .subscriptionName("sub") .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) - .payloadConverter(new DefaultPayloadConverter()) + .payloadConverter(converter) .subscribe(); final List values = new ArrayList<>(); for (int i = 0; i < numMessages; i++) { @@ -136,6 +139,17 @@ public int choosePartition(Message msg, TopicMetadata metadata) { for (int i = 0; i < numMessages; i++) { Assert.assertEquals(values.get(i), messagePrefix + i); } + + // Each buffer's refCnt is 2 when the iteration is stopped, because it will be released in finally blocks of + // 1. ConsumerImpl#consumeMessagesFromConverter + // 2. PulsarDecoder#channelRead + if (enableBatching) { + int numBatches = numMessages / batchingMaxMessages; + numBatches += (numMessages % batchingMaxMessages == 0) ? 0 : 1; + Assert.assertEquals(converter.getTotalRefCnt(), 2 * numBatches); + } else { + Assert.assertEquals(converter.getTotalRefCnt(), 2 * numMessages); + } } @Test From 6fdaabde427450d5111af161f7b44624ebf3294c Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Sat, 25 Sep 2021 10:45:50 +0800 Subject: [PATCH 09/29] Remove recycle() method from MessagePayload --- .../pulsar/client/converter/CustomBatchConverter.java | 8 ++++++-- .../pulsar/client/converter/DefaultPayloadConverter.java | 1 - .../java/org/apache/pulsar/client/api/MessagePayload.java | 7 +------ .../java/org/apache/pulsar/client/impl/ConsumerImpl.java | 3 ++- .../org/apache/pulsar/client/impl/MessagePayloadImpl.java | 1 - .../org/apache/pulsar/client/api/MessagePayloadTest.java | 5 ----- 6 files changed, 9 insertions(+), 16 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java index 11338ff8016dc..271d2365734ac 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java @@ -67,8 +67,12 @@ public Message next() { final String value = stringIterator.next(); final ByteBuf valueBuf = Unpooled.wrappedBuffer(Schema.STRING.encode(value)); bufList.add(valueBuf); - return context.newSingleMessage( - index++, numMessages, MessagePayloadImpl.create(valueBuf), false, schema); + final MessagePayloadImpl singlePayload = MessagePayloadImpl.create(valueBuf); + try { + return context.newSingleMessage(index++, numMessages, singlePayload, false, schema); + } finally { + singlePayload.recycle(); + } } }; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/DefaultPayloadConverter.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/DefaultPayloadConverter.java index 9fef0cb21beba..8998e6fa2e463 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/DefaultPayloadConverter.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/DefaultPayloadConverter.java @@ -48,7 +48,6 @@ public boolean hasNext() { final boolean result = (index < numMessages); if (!result) { totalRefCnt += ((MessagePayloadImpl) payload).getByteBuf().refCnt(); - payload.recycle(); } return result; } diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessagePayload.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessagePayload.java index 1e971a7ce09c1..c1bc8268014e9 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessagePayload.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessagePayload.java @@ -23,15 +23,10 @@ */ public interface MessagePayload { - /** - * Recycle this object if it's allocated from a memory pool. - */ - void recycle(); - /** * Copy the bytes of the payload into the byte array. * * @return the byte array that is filled with the readable bytes of the payload, it should not be null */ byte[] copiedBuffer(); -} \ No newline at end of file +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 57d06e05853f5..2d384aa58124a 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -1095,7 +1095,7 @@ private void consumeMessagesFromConverter(final BrokerEntryMetadata brokerEntryM final List ackSet) { final EntryContextImpl entryContext = EntryContextImpl.get( brokerEntryMetadata, messageMetadata, messageId, this, redeliveryCount, ackSet); - final MessagePayload payload = MessagePayloadImpl.create(byteBuf); + final MessagePayloadImpl payload = MessagePayloadImpl.create(byteBuf); int skippedMessages = 0; try { for (Message message : conf.getPayloadConverter().convert(entryContext, payload, schema)) { @@ -1111,6 +1111,7 @@ private void consumeMessagesFromConverter(final BrokerEntryMetadata brokerEntryM } finally { byteBuf.release(); entryContext.recycle(); + payload.recycle(); } if (skippedMessages > 0) { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadImpl.java index f8f106cfda556..9a366223c80bd 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadImpl.java @@ -51,7 +51,6 @@ private MessagePayloadImpl(final Recycler.Handle handle) { this.recyclerHandle = handle; } - @Override public void recycle() { byteBuf = null; recyclerHandle.recycle(this); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/api/MessagePayloadTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/api/MessagePayloadTest.java index 0ddb85fbb7f4a..51b2c1ae7c3ad 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/api/MessagePayloadTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/api/MessagePayloadTest.java @@ -79,11 +79,6 @@ public ByteBufferPayload(final ByteBuffer buffer) { this.buffer = buffer; } - @Override - public void recycle() { - // No ops - } - @Override public byte[] copiedBuffer() { final byte[] bytes = new byte[buffer.remaining()]; From 6ec4a28a1419b88c7a92aee3c61be3227389cc68 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Sat, 25 Sep 2021 11:06:38 +0800 Subject: [PATCH 10/29] Add cleanup() method to PayloadConverter interface --- .../client/converter/CustomBatchConverter.java | 16 +++++++++------- .../converter/DefaultPayloadConverter.java | 14 +++++++++----- .../client/converter/PayloadConverterTest.java | 9 ++++----- .../pulsar/client/api/PayloadConverter.java | 8 ++++++++ .../apache/pulsar/client/impl/ConsumerImpl.java | 1 + 5 files changed, 31 insertions(+), 17 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java index 271d2365734ac..2cc0e04fbfebf 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java @@ -35,6 +35,7 @@ public class CustomBatchConverter implements PayloadConverter { private static final PayloadConverter DEFAULT = new DefaultPayloadConverter(); + private final List bufList = new ArrayList<>(); @Override public Iterable> convert(EntryContext context, MessagePayload payload, Schema schema) { @@ -44,22 +45,17 @@ public Iterable> convert(EntryContext context, MessagePayload pay } final ByteBuf buf = MessagePayloadUtils.convertToByteBuf(payload); + bufList.add(buf); final CustomBatchFormat.StringIterable strings = CustomBatchFormat.deserialize(buf); final Iterator stringIterator = strings.iterator(); final int numMessages = context.getNumMessages(); - final List bufList = new ArrayList<>(); return () -> new Iterator>() { int index = 0; @Override public boolean hasNext() { - final boolean result = stringIterator.hasNext(); - if (!result) { - bufList.forEach(ReferenceCounted::release); - buf.release(); - } - return result; + return stringIterator.hasNext(); } @Override @@ -76,4 +72,10 @@ public Message next() { } }; } + + @Override + public void cleanup() { + bufList.forEach(ReferenceCounted::release); + bufList.clear(); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/DefaultPayloadConverter.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/DefaultPayloadConverter.java index 8998e6fa2e463..0b1fdb9303dab 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/DefaultPayloadConverter.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/DefaultPayloadConverter.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.converter; +import io.netty.buffer.ByteBuf; import lombok.Getter; import java.util.Iterator; import org.apache.pulsar.client.api.EntryContext; @@ -34,22 +35,20 @@ public class DefaultPayloadConverter implements PayloadConverter { @Getter private int totalRefCnt = 0; + private ByteBuf payloadBuffer; @Override public Iterable> convert(EntryContext context, MessagePayload payload, Schema schema) { final int numMessages = context.getNumMessages(); final boolean isBatch = context.isBatch(); + this.payloadBuffer = ((MessagePayloadImpl) payload).getByteBuf(); return () -> new Iterator>() { int index = 0; @Override public boolean hasNext() { - final boolean result = (index < numMessages); - if (!result) { - totalRefCnt += ((MessagePayloadImpl) payload).getByteBuf().refCnt(); - } - return result; + return index < numMessages; } @Override @@ -61,4 +60,9 @@ public Message next() { } }; } + + @Override + public void cleanup() { + totalRefCnt += payloadBuffer.refCnt(); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java index a57a8dc5f48c5..16cb8541e51c1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java @@ -140,15 +140,14 @@ public int choosePartition(Message msg, TopicMetadata metadata) { Assert.assertEquals(values.get(i), messagePrefix + i); } - // Each buffer's refCnt is 2 when the iteration is stopped, because it will be released in finally blocks of - // 1. ConsumerImpl#consumeMessagesFromConverter - // 2. PulsarDecoder#channelRead + // Each buffer's refCnt is 1 when the iteration is stopped, because it will be released in + // PulsarDecoder#channelRead() finally. if (enableBatching) { int numBatches = numMessages / batchingMaxMessages; numBatches += (numMessages % batchingMaxMessages == 0) ? 0 : 1; - Assert.assertEquals(converter.getTotalRefCnt(), 2 * numBatches); + Assert.assertEquals(converter.getTotalRefCnt(), numBatches); } else { - Assert.assertEquals(converter.getTotalRefCnt(), 2 * numMessages); + Assert.assertEquals(converter.getTotalRefCnt(), numMessages); } } diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadConverter.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadConverter.java index b4a4317af2a8f..a8f6b3921aba8 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadConverter.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadConverter.java @@ -32,4 +32,12 @@ public interface PayloadConverter { * The iteration could also be interrupted by CorruptedMessageException. */ Iterable> convert(EntryContext context, MessagePayload payload, Schema schema); + + /** + * Do some cleanup work, like deallocating the resources. + * + * The returned value of {@link PayloadConverter#convert} will be iterated in the internal implementation, after + * the iteration is stopped, this method will be called. + */ + default void cleanup() {} } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 2d384aa58124a..f40950dc04a75 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -1112,6 +1112,7 @@ private void consumeMessagesFromConverter(final BrokerEntryMetadata brokerEntryM byteBuf.release(); entryContext.recycle(); payload.recycle(); + conf.getPayloadConverter().cleanup(); } if (skippedMessages > 0) { From e5fc9584dd04c08d57645f4de3b1827403ec2122 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 28 Sep 2021 09:56:51 +0800 Subject: [PATCH 11/29] Rename cleanup method to afterConvert --- .../client/converter/CustomBatchConverter.java | 2 +- .../client/converter/DefaultPayloadConverter.java | 2 +- .../apache/pulsar/client/api/PayloadConverter.java | 13 ++++++++++--- .../org/apache/pulsar/client/impl/ConsumerImpl.java | 3 +-- 4 files changed, 13 insertions(+), 7 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java index 2cc0e04fbfebf..1b0ab6988941a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java @@ -74,7 +74,7 @@ public Message next() { } @Override - public void cleanup() { + public void afterConvert() { bufList.forEach(ReferenceCounted::release); bufList.clear(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/DefaultPayloadConverter.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/DefaultPayloadConverter.java index 0b1fdb9303dab..877196cde1d93 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/DefaultPayloadConverter.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/DefaultPayloadConverter.java @@ -62,7 +62,7 @@ public Message next() { } @Override - public void cleanup() { + public void afterConvert() { totalRefCnt += payloadBuffer.refCnt(); } } diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadConverter.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadConverter.java index a8f6b3921aba8..e549a0d9c06f8 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadConverter.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadConverter.java @@ -34,10 +34,17 @@ public interface PayloadConverter { Iterable> convert(EntryContext context, MessagePayload payload, Schema schema); /** - * Do some cleanup work, like deallocating the resources. - * * The returned value of {@link PayloadConverter#convert} will be iterated in the internal implementation, after * the iteration is stopped, this method will be called. + * + * The internal implementation is like: + * + * ```java + * for (Message msg : converter.convert(context, payload, schema) { + * // Do something with `msg` + * } + * converter.afterConvert(); + * ``` */ - default void cleanup() {} + default void afterConvert() {} } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index f40950dc04a75..df770f170959d 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -66,7 +66,6 @@ import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageCrypto; import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.MessagePayload; import org.apache.pulsar.client.api.Messages; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClientException; @@ -1112,7 +1111,7 @@ private void consumeMessagesFromConverter(final BrokerEntryMetadata brokerEntryM byteBuf.release(); entryContext.recycle(); payload.recycle(); - conf.getPayloadConverter().cleanup(); + conf.getPayloadConverter().afterConvert(); } if (skippedMessages > 0) { From b050d635c9c606acff9e43acb0abea096a75ddbe Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 28 Sep 2021 10:06:40 +0800 Subject: [PATCH 12/29] Rename methods of EntryContext --- .../converter/CustomBatchConverter.java | 2 +- .../converter/DefaultPayloadConverter.java | 4 ++-- .../pulsar/client/api/EntryContext.java | 20 +++++++++---------- .../pulsar/client/impl/EntryContextImpl.java | 12 +++++------ 4 files changed, 19 insertions(+), 19 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java index 1b0ab6988941a..080992c22cc66 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java @@ -65,7 +65,7 @@ public Message next() { bufList.add(valueBuf); final MessagePayloadImpl singlePayload = MessagePayloadImpl.create(valueBuf); try { - return context.newSingleMessage(index++, numMessages, singlePayload, false, schema); + return context.getMessageAt(index++, numMessages, singlePayload, false, schema); } finally { singlePayload.recycle(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/DefaultPayloadConverter.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/DefaultPayloadConverter.java index 877196cde1d93..888c14bb42f26 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/DefaultPayloadConverter.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/DefaultPayloadConverter.java @@ -55,8 +55,8 @@ public boolean hasNext() { public Message next() { index++; return isBatch - ? context.newSingleMessage(index, numMessages, payload, true, schema) - : context.newMessage(payload, schema); + ? context.getMessageAt(index, numMessages, payload, true, schema) + : context.asSingleMessage(payload, schema); } }; } diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/EntryContext.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/EntryContext.java index 1cc04fc3f0636..a6399ec4b0412 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/EntryContext.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/EntryContext.java @@ -48,7 +48,7 @@ public interface EntryContext { boolean isBatch(); /** - * Create a new single message, which is an internal message of a batch. + * Get the internal single message with a specific index from a payload if the entry is a batch. * * @param index the batch index * @param numMessages the number of messages in the batch @@ -56,24 +56,24 @@ public interface EntryContext { * @param containMetadata whether the payload contains the single message metadata * @param schema the schema of the batch * @param - * @return the created single message + * @return the single message * @implNote The `index` and `numMessages` parameters are used to create the message id with batch index. * If `containMetadata` is true, parse the single message metadata from the payload first. The fields of single * message metadata will overwrite the same fields of the entry's metadata. */ - Message newSingleMessage(int index, - int numMessages, - MessagePayload payload, - boolean containMetadata, - Schema schema); + Message getMessageAt(int index, + int numMessages, + MessagePayload payload, + boolean containMetadata, + Schema schema); /** - * Create a new non-batched message. + * Convert the given payload to a single message if the entry is non-batched. * * @param payload the message payload * @param schema the schema of the message * @param - * @return the created message + * @return the converted single message */ - Message newMessage(MessagePayload payload, Schema schema); + Message asSingleMessage(MessagePayload payload, Schema schema); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/EntryContextImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/EntryContextImpl.java index 1b6f1c2c59540..907893f009446 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/EntryContextImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/EntryContextImpl.java @@ -111,11 +111,11 @@ public boolean isBatch() { } @Override - public Message newSingleMessage(int index, - int numMessages, - MessagePayload payload, - boolean containMetadata, - Schema schema) { + public Message getMessageAt(int index, + int numMessages, + MessagePayload payload, + boolean containMetadata, + Schema schema) { final ByteBuf payloadBuffer = MessagePayloadUtils.convertToByteBuf(payload); try { return consumer.newSingleMessage(index, @@ -136,7 +136,7 @@ public Message newSingleMessage(int index, } @Override - public Message newMessage(MessagePayload payload, Schema schema) { + public Message asSingleMessage(MessagePayload payload, Schema schema) { final ByteBuf payloadBuffer = MessagePayloadUtils.convertToByteBuf(payload); try { return consumer.newMessage( From 441f8a02b2619012004ae946e99ffc05bfafb3bb Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 28 Sep 2021 10:48:15 +0800 Subject: [PATCH 13/29] Use the actual number of messages --- .../apache/pulsar/client/converter/CustomBatchConverter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java index 080992c22cc66..2a00ad88f768d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java @@ -48,7 +48,7 @@ public Iterable> convert(EntryContext context, MessagePayload pay bufList.add(buf); final CustomBatchFormat.StringIterable strings = CustomBatchFormat.deserialize(buf); final Iterator stringIterator = strings.iterator(); - final int numMessages = context.getNumMessages(); + final int numMessages = strings.size(); return () -> new Iterator>() { int index = 0; From a45483d6659044b986061db5d672767d9e00efb9 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Wed, 29 Sep 2021 12:20:19 +0800 Subject: [PATCH 14/29] Add MessagePayloadFactory to avoid raw use of MessagePayloadImpl --- .../converter/CustomBatchConverter.java | 9 ++-- .../pulsar/client/api/MessagePayload.java | 10 ++++ .../client/api/MessagePayloadFactory.java | 46 +++++++++++++++++++ .../PulsarClientImplementationBinding.java | 3 ++ .../pulsar/client/impl/ConsumerImpl.java | 4 +- .../impl/MessagePayloadFactoryImpl.java | 37 +++++++++++++++ .../client/impl/MessagePayloadImpl.java | 5 +- ...PulsarClientImplementationBindingImpl.java | 4 ++ 8 files changed, 109 insertions(+), 9 deletions(-) create mode 100644 pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessagePayloadFactory.java create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadFactoryImpl.java diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java index 2a00ad88f768d..0cb3c6b4f5ea1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java @@ -19,7 +19,6 @@ package org.apache.pulsar.client.converter; import io.netty.buffer.ByteBuf; -import io.netty.buffer.Unpooled; import io.netty.util.ReferenceCounted; import java.util.ArrayList; import java.util.Iterator; @@ -27,9 +26,9 @@ import org.apache.pulsar.client.api.EntryContext; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessagePayload; +import org.apache.pulsar.client.api.MessagePayloadFactory; import org.apache.pulsar.client.api.PayloadConverter; import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.client.impl.MessagePayloadImpl; import org.apache.pulsar.client.impl.MessagePayloadUtils; public class CustomBatchConverter implements PayloadConverter { @@ -61,13 +60,11 @@ public boolean hasNext() { @Override public Message next() { final String value = stringIterator.next(); - final ByteBuf valueBuf = Unpooled.wrappedBuffer(Schema.STRING.encode(value)); - bufList.add(valueBuf); - final MessagePayloadImpl singlePayload = MessagePayloadImpl.create(valueBuf); + final MessagePayload singlePayload = MessagePayloadFactory.DEFAULT.wrap(Schema.STRING.encode(value)); try { return context.getMessageAt(index++, numMessages, singlePayload, false, schema); } finally { - singlePayload.recycle(); + singlePayload.release(); } } }; diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessagePayload.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessagePayload.java index c1bc8268014e9..5d2ff6361f69f 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessagePayload.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessagePayload.java @@ -29,4 +29,14 @@ public interface MessagePayload { * @return the byte array that is filled with the readable bytes of the payload, it should not be null */ byte[] copiedBuffer(); + + /** + * Release the resources if necessary. + * + * NOTE: For a MessagePayload object that is created from {@link MessagePayloadFactory#DEFAULT}, this method must be + * called to avoid memory leak. + */ + default void release() { + // No ops + } } diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessagePayloadFactory.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessagePayloadFactory.java new file mode 100644 index 0000000000000..0181d6dbd2942 --- /dev/null +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessagePayloadFactory.java @@ -0,0 +1,46 @@ +/** + * 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.api; + +import java.nio.ByteBuffer; +import org.apache.pulsar.client.internal.DefaultImplementation; + +/** + * The factory class of {@link MessagePayload}. + */ +public interface MessagePayloadFactory { + + MessagePayloadFactory DEFAULT = DefaultImplementation.getDefaultImplementation().newDefaultMessagePayloadFactory(); + + /** + * Create a payload whose underlying buffer refers to a byte array. + * + * @param bytes the byte array + * @return the created MessagePayload object + */ + MessagePayload wrap(byte[] bytes); + + /** + * Create a payload whose underlying buffer refers to a NIO buffer. + * + * @param buffer the NIO buffer + * @return the created MessagePayload object + */ + MessagePayload wrap(ByteBuffer buffer); +} diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/internal/PulsarClientImplementationBinding.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/internal/PulsarClientImplementationBinding.java index 98b4d5c40c02a..f7bcf05230e82 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/internal/PulsarClientImplementationBinding.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/internal/PulsarClientImplementationBinding.java @@ -21,6 +21,7 @@ import org.apache.pulsar.client.api.BatcherBuilder; import org.apache.pulsar.client.api.ClientBuilder; import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.MessagePayloadFactory; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.schema.GenericRecord; @@ -229,6 +230,8 @@ SchemaInfo encodeKeyValueSchemaInfo(String schemaName, BatcherBuilder newKeyBasedBatcherBuilder(); + MessagePayloadFactory newDefaultMessagePayloadFactory(); + /** * Retrieves ByteBuffer data into byte[]. * diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index df770f170959d..8b7fd955b747b 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -1094,7 +1094,7 @@ private void consumeMessagesFromConverter(final BrokerEntryMetadata brokerEntryM final List ackSet) { final EntryContextImpl entryContext = EntryContextImpl.get( brokerEntryMetadata, messageMetadata, messageId, this, redeliveryCount, ackSet); - final MessagePayloadImpl payload = MessagePayloadImpl.create(byteBuf); + final MessagePayloadImpl payload = MessagePayloadImpl.create(byteBuf.retain()); int skippedMessages = 0; try { for (Message message : conf.getPayloadConverter().convert(entryContext, payload, schema)) { @@ -1110,7 +1110,7 @@ private void consumeMessagesFromConverter(final BrokerEntryMetadata brokerEntryM } finally { byteBuf.release(); entryContext.recycle(); - payload.recycle(); + payload.release(); conf.getPayloadConverter().afterConvert(); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadFactoryImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadFactoryImpl.java new file mode 100644 index 0000000000000..70a5b5616e6c6 --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadFactoryImpl.java @@ -0,0 +1,37 @@ +/** + * 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 io.netty.buffer.Unpooled; +import java.nio.ByteBuffer; +import org.apache.pulsar.client.api.MessagePayload; +import org.apache.pulsar.client.api.MessagePayloadFactory; + +public class MessagePayloadFactoryImpl implements MessagePayloadFactory { + + @Override + public MessagePayload wrap(byte[] bytes) { + return MessagePayloadImpl.create(Unpooled.wrappedBuffer(bytes)); + } + + @Override + public MessagePayload wrap(ByteBuffer buffer) { + return MessagePayloadImpl.create(Unpooled.wrappedBuffer(buffer)); + } +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadImpl.java index 9a366223c80bd..e5499f5e12ee6 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadImpl.java @@ -20,6 +20,7 @@ import io.netty.buffer.ByteBuf; import io.netty.util.Recycler; +import io.netty.util.ReferenceCountUtil; import lombok.Getter; import lombok.NonNull; import org.apache.pulsar.client.api.MessagePayload; @@ -51,7 +52,9 @@ private MessagePayloadImpl(final Recycler.Handle handle) { this.recyclerHandle = handle; } - public void recycle() { + @Override + public void release() { + ReferenceCountUtil.release(byteBuf); byteBuf = null; recyclerHandle.recycle(this); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImplementationBindingImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImplementationBindingImpl.java index 225e2467b416c..c146f238d55fc 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImplementationBindingImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImplementationBindingImpl.java @@ -35,6 +35,7 @@ import org.apache.pulsar.client.api.BatcherBuilder; import org.apache.pulsar.client.api.ClientBuilder; import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.MessagePayloadFactory; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.schema.GenericRecord; @@ -379,4 +380,7 @@ public BatcherBuilder newKeyBasedBatcherBuilder() { return new KeyBasedBatcherBuilder(); } + public MessagePayloadFactory newDefaultMessagePayloadFactory() { + return new MessagePayloadFactoryImpl(); + } } From 668b1ca13c03c5628624b484b2397939a07e0531 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Wed, 29 Sep 2021 12:52:29 +0800 Subject: [PATCH 15/29] Add tests for refCnt --- .../converter/CustomBatchConverter.java | 14 ++++---- .../converter/PayloadConverterTest.java | 33 ++++++++++++++----- .../pulsar/client/api/MessagePayloadTest.java | 15 +++++++++ 3 files changed, 46 insertions(+), 16 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java index 0cb3c6b4f5ea1..dbd366e2d6f93 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java @@ -19,10 +19,8 @@ package org.apache.pulsar.client.converter; import io.netty.buffer.ByteBuf; -import io.netty.util.ReferenceCounted; -import java.util.ArrayList; +import lombok.Getter; import java.util.Iterator; -import java.util.List; import org.apache.pulsar.client.api.EntryContext; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessagePayload; @@ -34,7 +32,9 @@ public class CustomBatchConverter implements PayloadConverter { private static final PayloadConverter DEFAULT = new DefaultPayloadConverter(); - private final List bufList = new ArrayList<>(); + @Getter + private int totalRefCnt = 0; + private ByteBuf tempBuf = null; @Override public Iterable> convert(EntryContext context, MessagePayload payload, Schema schema) { @@ -44,7 +44,7 @@ public Iterable> convert(EntryContext context, MessagePayload pay } final ByteBuf buf = MessagePayloadUtils.convertToByteBuf(payload); - bufList.add(buf); + tempBuf = buf; final CustomBatchFormat.StringIterable strings = CustomBatchFormat.deserialize(buf); final Iterator stringIterator = strings.iterator(); final int numMessages = strings.size(); @@ -72,7 +72,7 @@ public Message next() { @Override public void afterConvert() { - bufList.forEach(ReferenceCounted::release); - bufList.clear(); + tempBuf.release(); + totalRefCnt += tempBuf.refCnt(); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java index 16cb8541e51c1..af2f0b75dc9f0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java @@ -79,6 +79,21 @@ public static Object[][] config() { }; } + @DataProvider + public static Object[][] customBatchConfig() { + return new Object[][] { + // numMessages / batchingMaxMessages + { 10, 1 }, + { 10, 4 } + }; + } + + private static int getNumBatches(final int numMessages, final int batchingMaxMessages) { + int numBatches = numMessages / batchingMaxMessages; + numBatches += (numMessages % batchingMaxMessages == 0) ? 0 : 1; + return numBatches; + } + @Test(dataProvider = "config") public void testDefaultConverter(int numPartitions, boolean enableBatching, int batchingMaxMessages) throws Exception { @@ -143,27 +158,25 @@ public int choosePartition(Message msg, TopicMetadata metadata) { // Each buffer's refCnt is 1 when the iteration is stopped, because it will be released in // PulsarDecoder#channelRead() finally. if (enableBatching) { - int numBatches = numMessages / batchingMaxMessages; - numBatches += (numMessages % batchingMaxMessages == 0) ? 0 : 1; - Assert.assertEquals(converter.getTotalRefCnt(), numBatches); + Assert.assertEquals(converter.getTotalRefCnt(), getNumBatches(numMessages, batchingMaxMessages)); } else { Assert.assertEquals(converter.getTotalRefCnt(), numMessages); } } - @Test - public void testCustomConverter() throws Exception { - final String topic = "persistent://public/default/testCustomConverter"; - final int numMessages = 10; - final int batchingMaxMessages = 4; + @Test(dataProvider = "customBatchConfig") + public void testCustomConverter(final int numMessages, final int batchingMaxMessages) throws Exception { + final String topic = "persistent://public/default/testCustomConverter-" + + numMessages + "-" + batchingMaxMessages; final String messagePrefix = "msg-"; + final CustomBatchConverter converter = new CustomBatchConverter(); @Cleanup final Consumer consumer = pulsarClient.newConsumer(Schema.STRING) .topic(topic) .subscriptionName("sub") .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) - .payloadConverter(new CustomBatchConverter()) + .payloadConverter(converter) .subscribe(); final PersistentTopic persistentTopic = @@ -195,6 +208,8 @@ public void testCustomConverter() throws Exception { Assert.assertNotNull(message); Assert.assertEquals(message.getValue(), messagePrefix + i); } + + Assert.assertEquals(converter.getTotalRefCnt(), getNumBatches(numMessages, batchingMaxMessages)); } private static MessageMetadata createCustomMetadata() { diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/api/MessagePayloadTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/api/MessagePayloadTest.java index 51b2c1ae7c3ad..f980e5141886b 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/api/MessagePayloadTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/api/MessagePayloadTest.java @@ -86,4 +86,19 @@ public byte[] copiedBuffer() { return bytes; } } + + @Test + public void testFactoryWrap() { + MessagePayloadImpl payload = (MessagePayloadImpl) MessagePayloadFactory.DEFAULT.wrap(new byte[1]); + ByteBuf byteBuf = payload.getByteBuf(); + Assert.assertEquals(byteBuf.refCnt(), 1); + payload.release(); + Assert.assertEquals(byteBuf.refCnt(), 0); + + payload = (MessagePayloadImpl) MessagePayloadFactory.DEFAULT.wrap(ByteBuffer.allocate(1)); + byteBuf = payload.getByteBuf(); + Assert.assertEquals(byteBuf.refCnt(), 1); + payload.release(); + Assert.assertEquals(byteBuf.refCnt(), 0); + } } From f0b1de18ebfc123654f46addb8534c01d9d66eba Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Wed, 29 Sep 2021 12:55:42 +0800 Subject: [PATCH 16/29] Test BatchMessageAcker works when converter is configured --- .../apache/pulsar/client/converter/PayloadConverterTest.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java index af2f0b75dc9f0..cbd990032d029 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java @@ -145,6 +145,8 @@ public int choosePartition(Message msg, TopicMetadata metadata) { final Message message = consumer.receive(1, TimeUnit.SECONDS); Assert.assertNotNull(message); values.add(message.getValue()); + consumer.acknowledge(message.getMessageId()); + consumer.acknowledgeCumulative(message.getMessageId()); } if (numPartitions > 1) { @@ -207,6 +209,8 @@ public void testCustomConverter(final int numMessages, final int batchingMaxMess final Message message = consumer.receive(1, TimeUnit.SECONDS); Assert.assertNotNull(message); Assert.assertEquals(message.getValue(), messagePrefix + i); + consumer.acknowledge(message.getMessageId()); + consumer.acknowledgeCumulative(message.getMessageId()); } Assert.assertEquals(converter.getTotalRefCnt(), getNumBatches(numMessages, batchingMaxMessages)); From 2b4fd061cabc0b8cf9268dbad72a473b500dc3fc Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Wed, 29 Sep 2021 17:20:54 +0800 Subject: [PATCH 17/29] Add whenInterrupted() method to converter --- .../pulsar/client/api/PayloadConverter.java | 40 ++++++++++++++----- .../pulsar/client/impl/ConsumerImpl.java | 2 +- 2 files changed, 31 insertions(+), 11 deletions(-) diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadConverter.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadConverter.java index e549a0d9c06f8..4be7ac48c4bc1 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadConverter.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadConverter.java @@ -18,6 +18,25 @@ */ package org.apache.pulsar.client.api; +/** + * The converter that is responsible to convert a message payload to messages for consumers to consume. + * + * It's internally in consumer's implementation internally like: + * + * ```java + * try { + * for (Message msg : converter.convert(context, payload, schema) { + * // Do something with `msg`... + * } + * } catch (Throwable e) { + * converter.whenInterrupted(e); + * // Handle the exception... + * } finally { + * // Do some cleanup work... + * converter.afterConvert(); + * } + * ``` + */ public interface PayloadConverter { /** @@ -33,18 +52,19 @@ public interface PayloadConverter { */ Iterable> convert(EntryContext context, MessagePayload payload, Schema schema); + /** + * The returned value of {@link PayloadConverter#convert} will be iterated in the internal implementation, if any + * exception was thrown, the iteration would stop. In this case, this method will be called. + */ + default void whenInterrupted(Throwable e) { + // No ops + } + /** * The returned value of {@link PayloadConverter#convert} will be iterated in the internal implementation, after * the iteration is stopped, this method will be called. - * - * The internal implementation is like: - * - * ```java - * for (Message msg : converter.convert(context, payload, schema) { - * // Do something with `msg` - * } - * converter.afterConvert(); - * ``` */ - default void afterConvert() {} + default void afterConvert() { + // No ops + } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 8b7fd955b747b..897eb14fec247 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -1104,7 +1104,7 @@ private void consumeMessagesFromConverter(final BrokerEntryMetadata brokerEntryM } executeNotifyCallback((MessageImpl) message); } - } catch (IllegalStateException e) { + } catch (Throwable e) { log.warn("[{}] [{}] unable to obtain message in batch", subscription, consumerName, e); discardCorruptedMessage(messageId, cnx(), ValidationError.BatchDeSerializeError); } finally { From 2ebd276e5550f87c7466abb0005b3b252275763a Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Wed, 29 Sep 2021 20:06:29 +0800 Subject: [PATCH 18/29] Add whenInterrupted call --- .../java/org/apache/pulsar/client/impl/ConsumerImpl.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 897eb14fec247..73c6d15b87df5 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -67,6 +67,7 @@ import org.apache.pulsar.client.api.MessageCrypto; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Messages; +import org.apache.pulsar.client.api.PayloadConverter; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.PulsarClientException.TopicDoesNotExistException; @@ -1095,9 +1096,10 @@ private void consumeMessagesFromConverter(final BrokerEntryMetadata brokerEntryM final EntryContextImpl entryContext = EntryContextImpl.get( brokerEntryMetadata, messageMetadata, messageId, this, redeliveryCount, ackSet); final MessagePayloadImpl payload = MessagePayloadImpl.create(byteBuf.retain()); + final PayloadConverter converter = conf.getPayloadConverter(); int skippedMessages = 0; try { - for (Message message : conf.getPayloadConverter().convert(entryContext, payload, schema)) { + for (Message message : converter.convert(entryContext, payload, schema)) { if (message == null) { skippedMessages++; continue; @@ -1105,6 +1107,7 @@ private void consumeMessagesFromConverter(final BrokerEntryMetadata brokerEntryM executeNotifyCallback((MessageImpl) message); } } catch (Throwable e) { + converter.whenInterrupted(e); log.warn("[{}] [{}] unable to obtain message in batch", subscription, consumerName, e); discardCorruptedMessage(messageId, cnx(), ValidationError.BatchDeSerializeError); } finally { From 985e35e83808de4735ca13481edebf99ff83cc94 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Wed, 29 Sep 2021 21:49:08 +0800 Subject: [PATCH 19/29] Use PayloadProcessor instead of PayloadConverter --- .../converter/CustomBatchConverter.java | 78 ------ .../converter/DefaultPayloadConverter.java | 68 ------ .../converter/PayloadConverterTest.java | 229 ------------------ .../CustomBatchFormat.java | 2 +- .../pulsar/client/api/ConsumerBuilder.java | 9 +- .../pulsar/client/api/PayloadConverter.java | 70 ------ .../pulsar/client/api/PayloadProcessor.java | 76 ++++++ .../client/impl/ConsumerBuilderImpl.java | 6 +- .../pulsar/client/impl/ConsumerImpl.java | 45 ++-- .../impl/conf/ConsumerConfigurationData.java | 4 +- 10 files changed, 106 insertions(+), 481 deletions(-) delete mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java delete mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/converter/DefaultPayloadConverter.java delete mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java rename pulsar-broker/src/test/java/org/apache/pulsar/client/{converter => processor}/CustomBatchFormat.java (98%) delete mode 100644 pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadConverter.java create mode 100644 pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadProcessor.java diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java deleted file mode 100644 index dbd366e2d6f93..0000000000000 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchConverter.java +++ /dev/null @@ -1,78 +0,0 @@ -/** - * 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.converter; - -import io.netty.buffer.ByteBuf; -import lombok.Getter; -import java.util.Iterator; -import org.apache.pulsar.client.api.EntryContext; -import org.apache.pulsar.client.api.Message; -import org.apache.pulsar.client.api.MessagePayload; -import org.apache.pulsar.client.api.MessagePayloadFactory; -import org.apache.pulsar.client.api.PayloadConverter; -import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.client.impl.MessagePayloadUtils; - -public class CustomBatchConverter implements PayloadConverter { - - private static final PayloadConverter DEFAULT = new DefaultPayloadConverter(); - @Getter - private int totalRefCnt = 0; - private ByteBuf tempBuf = null; - - @Override - public Iterable> convert(EntryContext context, MessagePayload payload, Schema schema) { - final String value = context.getProperty(CustomBatchFormat.KEY); - if (value == null || !value.equals(CustomBatchFormat.VALUE)) { - return DEFAULT.convert(context, payload, schema); - } - - final ByteBuf buf = MessagePayloadUtils.convertToByteBuf(payload); - tempBuf = buf; - final CustomBatchFormat.StringIterable strings = CustomBatchFormat.deserialize(buf); - final Iterator stringIterator = strings.iterator(); - final int numMessages = strings.size(); - - return () -> new Iterator>() { - int index = 0; - - @Override - public boolean hasNext() { - return stringIterator.hasNext(); - } - - @Override - public Message next() { - final String value = stringIterator.next(); - final MessagePayload singlePayload = MessagePayloadFactory.DEFAULT.wrap(Schema.STRING.encode(value)); - try { - return context.getMessageAt(index++, numMessages, singlePayload, false, schema); - } finally { - singlePayload.release(); - } - } - }; - } - - @Override - public void afterConvert() { - tempBuf.release(); - totalRefCnt += tempBuf.refCnt(); - } -} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/DefaultPayloadConverter.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/DefaultPayloadConverter.java deleted file mode 100644 index 888c14bb42f26..0000000000000 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/DefaultPayloadConverter.java +++ /dev/null @@ -1,68 +0,0 @@ -/** - * 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.converter; - -import io.netty.buffer.ByteBuf; -import lombok.Getter; -import java.util.Iterator; -import org.apache.pulsar.client.api.EntryContext; -import org.apache.pulsar.client.api.Message; -import org.apache.pulsar.client.api.MessagePayload; -import org.apache.pulsar.client.api.PayloadConverter; -import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.client.impl.MessagePayloadImpl; - -/** - * The default converter provided for users that want to define their own {@link PayloadConverter}. - */ -public class DefaultPayloadConverter implements PayloadConverter { - - @Getter - private int totalRefCnt = 0; - private ByteBuf payloadBuffer; - - @Override - public Iterable> convert(EntryContext context, MessagePayload payload, Schema schema) { - final int numMessages = context.getNumMessages(); - final boolean isBatch = context.isBatch(); - this.payloadBuffer = ((MessagePayloadImpl) payload).getByteBuf(); - - return () -> new Iterator>() { - int index = 0; - - @Override - public boolean hasNext() { - return index < numMessages; - } - - @Override - public Message next() { - index++; - return isBatch - ? context.getMessageAt(index, numMessages, payload, true, schema) - : context.asSingleMessage(payload, schema); - } - }; - } - - @Override - public void afterConvert() { - totalRefCnt += payloadBuffer.refCnt(); - } -} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java deleted file mode 100644 index cbd990032d029..0000000000000 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/PayloadConverterTest.java +++ /dev/null @@ -1,229 +0,0 @@ -/** - * 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.converter; - -import com.google.common.collect.Sets; -import io.netty.buffer.ByteBuf; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -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.MessageRouter; -import org.apache.pulsar.client.api.Producer; -import org.apache.pulsar.client.api.ProducerConsumerBase; -import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.client.api.SubscriptionInitialPosition; -import org.apache.pulsar.client.api.TopicMetadata; -import org.apache.pulsar.common.api.proto.MessageMetadata; -import org.apache.pulsar.common.policies.data.ClusterData; -import org.apache.pulsar.common.policies.data.TenantInfoImpl; -import org.apache.pulsar.common.protocol.Commands; -import org.testng.Assert; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.DataProvider; -import org.testng.annotations.Test; - -/** - * Tests for {@link org.apache.pulsar.client.api.PayloadConverter}. - */ -@Slf4j -public class PayloadConverterTest extends ProducerConsumerBase { - - @BeforeClass - @Override - protected void setup() throws Exception { - super.internalSetup(); - admin.clusters().createCluster("test", - ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); - admin.tenants().createTenant("public", - new TenantInfoImpl(Sets.newHashSet("appid"), Sets.newHashSet("test"))); - admin.namespaces().createNamespace("public/default", Sets.newHashSet("test")); - } - - @AfterClass - @Override - protected void cleanup() throws Exception { - super.internalCleanup(); - } - - @DataProvider - public static Object[][] config() { - return new Object[][] { - // numPartitions / enableBatching / batchingMaxMessages - { 1, true, 1 }, - { 1, true, 4 }, - { 1, false, 1 }, - { 3, false, 1 } - }; - } - - @DataProvider - public static Object[][] customBatchConfig() { - return new Object[][] { - // numMessages / batchingMaxMessages - { 10, 1 }, - { 10, 4 } - }; - } - - private static int getNumBatches(final int numMessages, final int batchingMaxMessages) { - int numBatches = numMessages / batchingMaxMessages; - numBatches += (numMessages % batchingMaxMessages == 0) ? 0 : 1; - return numBatches; - } - - @Test(dataProvider = "config") - public void testDefaultConverter(int numPartitions, boolean enableBatching, int batchingMaxMessages) - throws Exception { - final String topic = "testDefaultConverter-" + numPartitions + "-" + enableBatching + "-" + batchingMaxMessages; - final int numMessages = 10; - final String messagePrefix = "msg-"; - - if (numPartitions > 1) { - admin.topics().createPartitionedTopic(topic, numPartitions); - } - - @Cleanup - final Producer producer = pulsarClient.newProducer(Schema.STRING) - .topic(topic) - .enableBatching(enableBatching) - .batchingMaxMessages(batchingMaxMessages) - .batchingMaxPublishDelay(100, TimeUnit.MILLISECONDS) - .messageRouter(new MessageRouter() { - int i = 0; - - @Override - public int choosePartition(Message msg, TopicMetadata metadata) { - return i++ % metadata.numPartitions(); - } - }) - .create(); - for (int i = 0; i < numMessages; i++) { - final String value = messagePrefix + i; - producer.sendAsync(value).whenComplete((id, e) -> { - if (e == null) { - log.info("Send {} to {} {}", value, topic, id); - } else { - log.error("Failed to send {}: {}", value, e.getMessage()); - } - }); - } - - final DefaultPayloadConverter converter = new DefaultPayloadConverter(); - - @Cleanup - final Consumer consumer = pulsarClient.newConsumer(Schema.STRING) - .topic(topic) - .subscriptionName("sub") - .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) - .payloadConverter(converter) - .subscribe(); - final List values = new ArrayList<>(); - for (int i = 0; i < numMessages; i++) { - final Message message = consumer.receive(1, TimeUnit.SECONDS); - Assert.assertNotNull(message); - values.add(message.getValue()); - consumer.acknowledge(message.getMessageId()); - consumer.acknowledgeCumulative(message.getMessageId()); - } - - if (numPartitions > 1) { - // messages are out of order across multiple partitions - Collections.sort(values); - } - for (int i = 0; i < numMessages; i++) { - Assert.assertEquals(values.get(i), messagePrefix + i); - } - - // Each buffer's refCnt is 1 when the iteration is stopped, because it will be released in - // PulsarDecoder#channelRead() finally. - if (enableBatching) { - Assert.assertEquals(converter.getTotalRefCnt(), getNumBatches(numMessages, batchingMaxMessages)); - } else { - Assert.assertEquals(converter.getTotalRefCnt(), numMessages); - } - } - - @Test(dataProvider = "customBatchConfig") - public void testCustomConverter(final int numMessages, final int batchingMaxMessages) throws Exception { - final String topic = "persistent://public/default/testCustomConverter-" - + numMessages + "-" + batchingMaxMessages; - final String messagePrefix = "msg-"; - - final CustomBatchConverter converter = new CustomBatchConverter(); - @Cleanup - final Consumer consumer = pulsarClient.newConsumer(Schema.STRING) - .topic(topic) - .subscriptionName("sub") - .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) - .payloadConverter(converter) - .subscribe(); - - final PersistentTopic persistentTopic = - (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(topic).get().orElse(null); - Assert.assertNotNull(persistentTopic); - - final List values = new ArrayList<>(); - for (int i = 0; i < numMessages; i++) { - final String value = messagePrefix + i; - values.add(value); - if (values.size() >= batchingMaxMessages || i == numMessages - 1) { - final ByteBuf valueBuf = CustomBatchFormat.serialize(values); - values.clear(); - final ByteBuf headerAndPayload = Commands.serializeMetadataAndPayload(Commands.ChecksumType.None, - createCustomMetadata(), valueBuf); - valueBuf.release(); - persistentTopic.publishMessage(headerAndPayload, (e, ledgerId, entryId) -> { - if (e == null) { - log.info("Send {} to {} ({}, {})", value, topic, ledgerId, entryId); - } else { - log.error("Failed to send {}: {}", value, e.getMessage()); - } - }); - } - } - - for (int i = 0; i < numMessages; i++) { - final Message message = consumer.receive(1, TimeUnit.SECONDS); - Assert.assertNotNull(message); - Assert.assertEquals(message.getValue(), messagePrefix + i); - consumer.acknowledge(message.getMessageId()); - consumer.acknowledgeCumulative(message.getMessageId()); - } - - Assert.assertEquals(converter.getTotalRefCnt(), getNumBatches(numMessages, batchingMaxMessages)); - } - - private static MessageMetadata createCustomMetadata() { - final MessageMetadata messageMetadata = new MessageMetadata(); - // Here are required fields - messageMetadata.setProducerName("producer"); - messageMetadata.setSequenceId(0L); - messageMetadata.setPublishTime(0L); - // Add the property to identify the message format - messageMetadata.addProperty().setKey(CustomBatchFormat.KEY).setValue(CustomBatchFormat.VALUE); - return messageMetadata; - } -} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchFormat.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/CustomBatchFormat.java similarity index 98% rename from pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchFormat.java rename to pulsar-broker/src/test/java/org/apache/pulsar/client/processor/CustomBatchFormat.java index 03967e71223f8..a32b28eb83802 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/converter/CustomBatchFormat.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/CustomBatchFormat.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pulsar.client.converter; +package org.apache.pulsar.client.processor; import io.netty.buffer.ByteBuf; import java.nio.charset.StandardCharsets; diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java index e3584ddbd170a..2ed35f92781fe 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java @@ -743,13 +743,10 @@ public interface ConsumerBuilder extends Cloneable { ConsumerBuilder poolMessages(boolean poolMessages); /** - * Set the payload converter that might convert the message payload to the format that Pulsar consumer can recognize - * before creating {@link Message}s from an entry. + * If it's configured with a non-null value, the consumer will use the processor to process the payload, including + * decoding it to messages and triggering the listener. * * Default: null - * - * @param payloadConverter - * @return */ - ConsumerBuilder payloadConverter(PayloadConverter payloadConverter); + ConsumerBuilder payloadProcessor(PayloadProcessor payloadProcessor); } diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadConverter.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadConverter.java deleted file mode 100644 index 4be7ac48c4bc1..0000000000000 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadConverter.java +++ /dev/null @@ -1,70 +0,0 @@ -/** - * 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.api; - -/** - * The converter that is responsible to convert a message payload to messages for consumers to consume. - * - * It's internally in consumer's implementation internally like: - * - * ```java - * try { - * for (Message msg : converter.convert(context, payload, schema) { - * // Do something with `msg`... - * } - * } catch (Throwable e) { - * converter.whenInterrupted(e); - * // Handle the exception... - * } finally { - * // Do some cleanup work... - * converter.afterConvert(); - * } - * ``` - */ -public interface PayloadConverter { - - /** - * Convert the payload to iterable messages. - * - * @param context the message context that contains the message format information and methods to create a message - * @param payload the payload whose underlying buffer is a Netty ByteBuf - * @param schema the message's schema - * @param - * @return iterable messages - * @implNote During the iteration, the message could be null, which means it will be skipped in Pulsar consumer. - * The iteration could also be interrupted by CorruptedMessageException. - */ - Iterable> convert(EntryContext context, MessagePayload payload, Schema schema); - - /** - * The returned value of {@link PayloadConverter#convert} will be iterated in the internal implementation, if any - * exception was thrown, the iteration would stop. In this case, this method will be called. - */ - default void whenInterrupted(Throwable e) { - // No ops - } - - /** - * The returned value of {@link PayloadConverter#convert} will be iterated in the internal implementation, after - * the iteration is stopped, this method will be called. - */ - default void afterConvert() { - // No ops - } -} diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadProcessor.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadProcessor.java new file mode 100644 index 0000000000000..361cab07e94ac --- /dev/null +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadProcessor.java @@ -0,0 +1,76 @@ +/** + * 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.api; + +import java.util.function.Consumer; + +/** + * The processor to process a message payload. + * + * It's responsible to convert the raw buffer to some messages, then trigger some callbacks so that consumer can consume + * these messages and handle the exception if it existed. + * + * The most important part is to decode the raw buffer. After that, we can call {@link EntryContext#getMessageAt} or + * {@link EntryContext#asSingleMessage} to construct {@link Message} for consumer to consume. Since we need to pass the + * {@link MessagePayload} object to these methods, we can use {@link MessagePayloadFactory#DEFAULT} to create it or just + * reuse the payload argument. + */ +public interface PayloadProcessor { + + /** + * Process the message payload. + * + * @param payload the payload whose underlying buffer is a Netty ByteBuf + * @param context the message context that contains the message format information and methods to create a message + * @param schema the message's schema + * @param messageConsumer the callback to consume each message + * @param throwableConsumer the callback to handle the exception + * @param + */ + void process(MessagePayload payload, + EntryContext context, + Schema schema, + java.util.function.Consumer> messageConsumer, + Consumer throwableConsumer); + + // The default processor for Pulsar format payload. It should be noted getNumMessages() and isBatch() methods of + // EntryContext only work for Pulsar format. For other formats, the message metadata might be stored in the payload. + PayloadProcessor DEFAULT = new PayloadProcessor() { + + @Override + public void process(MessagePayload payload, + EntryContext context, + Schema schema, + Consumer> messageConsumer, + Consumer throwableConsumer) { + try { + if (context.isBatch()) { + final int numMessages = context.getNumMessages(); + for (int i = 0; i < numMessages; i++) { + messageConsumer.accept(context.getMessageAt(i, numMessages, payload, true, schema)); + } + } else { + messageConsumer.accept(context.asSingleMessage(payload, schema)); + } + } catch (Throwable throwable) { + throwableConsumer.accept(throwable); + } + } + }; +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java index 5a97d7232926d..258cef689442a 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java @@ -44,7 +44,7 @@ import org.apache.pulsar.client.api.KeySharedPolicy; import org.apache.pulsar.client.api.MessageCrypto; import org.apache.pulsar.client.api.MessageListener; -import org.apache.pulsar.client.api.PayloadConverter; +import org.apache.pulsar.client.api.PayloadProcessor; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.PulsarClientException.InvalidConfigurationException; import org.apache.pulsar.client.api.RegexSubscriptionMode; @@ -482,8 +482,8 @@ public ConsumerBuilder poolMessages(boolean poolMessages) { } @Override - public ConsumerBuilder payloadConverter(PayloadConverter payloadConverter) { - conf.setPayloadConverter(payloadConverter); + public ConsumerBuilder payloadProcessor(PayloadProcessor payloadProcessor) { + conf.setPayloadProcessor(payloadProcessor); return this; } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 73c6d15b87df5..2bac0c8b33b25 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -50,6 +50,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLongFieldUpdater; @@ -67,7 +68,7 @@ import org.apache.pulsar.client.api.MessageCrypto; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Messages; -import org.apache.pulsar.client.api.PayloadConverter; +import org.apache.pulsar.client.api.PayloadProcessor; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.PulsarClientException.TopicDoesNotExistException; @@ -1086,39 +1087,35 @@ private void executeNotifyCallback(final MessageImpl message) { }); } - private void consumeMessagesFromConverter(final BrokerEntryMetadata brokerEntryMetadata, - final MessageMetadata messageMetadata, - final ByteBuf byteBuf, - final MessageIdImpl messageId, - final Schema schema, - final int redeliveryCount, - final List ackSet) { + private void processPayloadByProcessor(final BrokerEntryMetadata brokerEntryMetadata, + final MessageMetadata messageMetadata, + final ByteBuf byteBuf, + final MessageIdImpl messageId, + final Schema schema, + final int redeliveryCount, + final List ackSet) { + final MessagePayloadImpl payload = MessagePayloadImpl.create(byteBuf.retain()); final EntryContextImpl entryContext = EntryContextImpl.get( brokerEntryMetadata, messageMetadata, messageId, this, redeliveryCount, ackSet); - final MessagePayloadImpl payload = MessagePayloadImpl.create(byteBuf.retain()); - final PayloadConverter converter = conf.getPayloadConverter(); - int skippedMessages = 0; + final AtomicInteger skippedMessages = new AtomicInteger(0); try { - for (Message message : converter.convert(entryContext, payload, schema)) { + conf.getPayloadProcessor().process(payload, entryContext, schema, message -> { if (message == null) { - skippedMessages++; - continue; + skippedMessages.incrementAndGet(); } executeNotifyCallback((MessageImpl) message); - } - } catch (Throwable e) { - converter.whenInterrupted(e); - log.warn("[{}] [{}] unable to obtain message in batch", subscription, consumerName, e); - discardCorruptedMessage(messageId, cnx(), ValidationError.BatchDeSerializeError); + }, throwable -> { + log.warn("[{}] [{}] unable to obtain message in batch", subscription, consumerName, throwable); + discardCorruptedMessage(messageId, cnx(), ValidationError.BatchDeSerializeError); + }); } finally { byteBuf.release(); entryContext.recycle(); payload.release(); - conf.getPayloadConverter().afterConvert(); } - if (skippedMessages > 0) { - increaseAvailablePermits(cnx(), skippedMessages); + if (skippedMessages.get() > 0) { + increaseAvailablePermits(cnx(), skippedMessages.get()); } internalPinnedExecutor.execute(() @@ -1180,9 +1177,9 @@ void messageReceived(MessageIdData messageId, int redeliveryCount, List ac return; } - if (conf.getPayloadConverter() != null) { + if (conf.getPayloadProcessor() != null) { // uncompressedPayload is released in this method so we don't need to call release() again - consumeMessagesFromConverter( + processPayloadByProcessor( brokerEntryMetadata, msgMetadata, uncompressedPayload, msgId, schema, redeliveryCount, ackSet); return; } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java index f3cdcc589dffe..5dcee82c82ea1 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java @@ -42,7 +42,7 @@ import org.apache.pulsar.client.api.KeySharedPolicy; import org.apache.pulsar.client.api.MessageCrypto; import org.apache.pulsar.client.api.MessageListener; -import org.apache.pulsar.client.api.PayloadConverter; +import org.apache.pulsar.client.api.PayloadProcessor; import org.apache.pulsar.client.api.RegexSubscriptionMode; import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.api.SubscriptionMode; @@ -151,7 +151,7 @@ public int getMaxPendingChuckedMessage() { private boolean poolMessages = false; @JsonIgnore - private transient PayloadConverter payloadConverter = null; + private transient PayloadProcessor payloadProcessor = null; public void setAutoUpdatePartitionsIntervalSeconds(int interval, TimeUnit timeUnit) { checkArgument(interval > 0, "interval needs to be > 0"); From 9635eb4fd6132262e8967d3b9f5ceb018cdf9a0c Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Wed, 29 Sep 2021 22:20:31 +0800 Subject: [PATCH 20/29] Test default payload processor --- .../processor/DefaultProcessorWithRefCnt.java | 47 ++++++ .../processor/PayloadProcessorTest.java | 155 ++++++++++++++++++ .../pulsar/client/impl/ConsumerImpl.java | 5 +- 3 files changed, 204 insertions(+), 3 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/processor/DefaultProcessorWithRefCnt.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/processor/PayloadProcessorTest.java diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/DefaultProcessorWithRefCnt.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/DefaultProcessorWithRefCnt.java new file mode 100644 index 0000000000000..115e000792d22 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/DefaultProcessorWithRefCnt.java @@ -0,0 +1,47 @@ +/** + * 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.processor; + +import java.util.function.Consumer; +import lombok.Getter; +import org.apache.pulsar.client.api.EntryContext; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessagePayload; +import org.apache.pulsar.client.api.PayloadProcessor; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.MessagePayloadImpl; + +/** + * The processor for Pulsar format messages and maintains a total reference count. + * + * It's used to verify {@link EntryContext#getMessageAt} and {@link EntryContext#asSingleMessage} have release the + * ByteBuf successfully. + */ +public class DefaultProcessorWithRefCnt implements PayloadProcessor { + + @Getter + int totalRefCnt = 0; + + @Override + public void process(MessagePayload payload, EntryContext context, Schema schema, + Consumer> messageConsumer, Consumer throwableConsumer) { + PayloadProcessor.DEFAULT.process(payload, context, schema, messageConsumer, throwableConsumer); + totalRefCnt += ((MessagePayloadImpl) payload).getByteBuf().refCnt(); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/PayloadProcessorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/PayloadProcessorTest.java new file mode 100644 index 0000000000000..9a08ae8acd5a2 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/PayloadProcessorTest.java @@ -0,0 +1,155 @@ +/** + * 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.processor; + +import com.google.common.collect.Sets; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.TimeUnit; +import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessageRouter; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; +import org.apache.pulsar.client.api.TopicMetadata; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +/** + * Test for {@link org.apache.pulsar.client.api.PayloadProcessor}. + */ +@Slf4j +public class PayloadProcessorTest extends ProducerConsumerBase { + + @BeforeClass + @Override + protected void setup() throws Exception { + super.internalSetup(); + admin.clusters().createCluster("test", + ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); + admin.tenants().createTenant("public", + new TenantInfoImpl(Sets.newHashSet("appid"), Sets.newHashSet("test"))); + admin.namespaces().createNamespace("public/default", Sets.newHashSet("test")); + } + + @AfterClass + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @DataProvider + public static Object[][] config() { + return new Object[][] { + // numPartitions / enableBatching / batchingMaxMessages + { 1, true, 1 }, + { 1, true, 4 }, + { 1, false, 1 }, + { 3, false, 1 } + }; + } + + private static int getNumBatches(final int numMessages, final int batchingMaxMessages) { + int numBatches = numMessages / batchingMaxMessages; + numBatches += (numMessages % batchingMaxMessages == 0) ? 0 : 1; + return numBatches; + } + + @Test(dataProvider = "config") + public void testDefaultProcessor(int numPartitions, boolean enableBatching, int batchingMaxMessages) + throws Exception { + final String topic = "testDefaultProcessor-" + numPartitions + "-" + enableBatching + "-" + batchingMaxMessages; + final int numMessages = 10; + final String messagePrefix = "msg-"; + + if (numPartitions > 1) { + admin.topics().createPartitionedTopic(topic, numPartitions); + } + + @Cleanup + final Producer producer = pulsarClient.newProducer(Schema.STRING) + .topic(topic) + .enableBatching(enableBatching) + .batchingMaxMessages(batchingMaxMessages) + .batchingMaxPublishDelay(100, TimeUnit.MILLISECONDS) + .messageRouter(new MessageRouter() { + int i = 0; + + @Override + public int choosePartition(Message msg, TopicMetadata metadata) { + return i++ % metadata.numPartitions(); + } + }) + .create(); + for (int i = 0; i < numMessages; i++) { + final String value = messagePrefix + i; + producer.sendAsync(value).whenComplete((id, e) -> { + if (e == null) { + log.info("Send {} to {} {}", value, topic, id); + } else { + log.error("Failed to send {}: {}", value, e.getMessage()); + } + }); + } + + final DefaultProcessorWithRefCnt processor = new DefaultProcessorWithRefCnt(); + @Cleanup + final Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .topic(topic) + .subscriptionName("sub") + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .payloadProcessor(processor) + .subscribe(); + final List values = new ArrayList<>(); + for (int i = 0; i < numMessages; i++) { + final Message message = consumer.receive(1, TimeUnit.SECONDS); + Assert.assertNotNull(message); + values.add(message.getValue()); + consumer.acknowledge(message.getMessageId()); + consumer.acknowledgeCumulative(message.getMessageId()); + } + + if (numPartitions > 1) { + // messages are out of order across multiple partitions + Collections.sort(values); + } + for (int i = 0; i < numMessages; i++) { + Assert.assertEquals(values.get(i), messagePrefix + i); + } + + // Each buffer's refCnt is 2 because after retrieving the refCnt, there will be two release for the ByteBuf: + // 1. ConsumerImpl#processPayloadByProcessor + // 2. PulsarDecoder#channelRead + if (enableBatching) { + Assert.assertEquals(processor.getTotalRefCnt(), 2 * getNumBatches(numMessages, batchingMaxMessages)); + } else { + Assert.assertEquals(processor.getTotalRefCnt(), 2 * numMessages); + } + } +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 2bac0c8b33b25..71fd1436026cb 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -1094,7 +1094,7 @@ private void processPayloadByProcessor(final BrokerEntryMetadata brokerEntryMeta final Schema schema, final int redeliveryCount, final List ackSet) { - final MessagePayloadImpl payload = MessagePayloadImpl.create(byteBuf.retain()); + final MessagePayloadImpl payload = MessagePayloadImpl.create(byteBuf); final EntryContextImpl entryContext = EntryContextImpl.get( brokerEntryMetadata, messageMetadata, messageId, this, redeliveryCount, ackSet); final AtomicInteger skippedMessages = new AtomicInteger(0); @@ -1109,9 +1109,8 @@ private void processPayloadByProcessor(final BrokerEntryMetadata brokerEntryMeta discardCorruptedMessage(messageId, cnx(), ValidationError.BatchDeSerializeError); }); } finally { - byteBuf.release(); entryContext.recycle(); - payload.release(); + payload.release(); // byteBuf.release() is called in this method } if (skippedMessages.get() > 0) { From 8b3b61aaa656c9d613ac7fd7503a699ec4f2a705 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Thu, 30 Sep 2021 09:43:58 +0800 Subject: [PATCH 21/29] Remove the Iterable interfaces of custom batch --- .../client/processor/CustomBatchFormat.java | 66 ++++++++----------- 1 file changed, 26 insertions(+), 40 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/CustomBatchFormat.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/CustomBatchFormat.java index a32b28eb83802..92580d133c4ab 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/CustomBatchFormat.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/CustomBatchFormat.java @@ -19,12 +19,13 @@ package org.apache.pulsar.client.processor; import io.netty.buffer.ByteBuf; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.Iterator; import java.util.List; +import lombok.AllArgsConstructor; +import lombok.Getter; +import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.testng.Assert; import org.testng.annotations.Test; @@ -42,6 +43,12 @@ public class CustomBatchFormat { public static final String KEY = "entry.format"; public static final String VALUE = "custom"; + @AllArgsConstructor + @Getter + public static class Metadata { + private final int numMessages; + } + public static ByteBuf serialize(Iterable strings) { final ByteBuf buf = PulsarByteBufAllocator.DEFAULT.buffer(1024); buf.writeShort(0); @@ -54,50 +61,30 @@ public static ByteBuf serialize(Iterable strings) { return buf; } - public static abstract class StringIterable implements Iterable { - - public abstract int size(); - }; - - public static StringIterable deserialize(final ByteBuf buf) { - final int numMessages = buf.readShort(); - return new StringIterable() { - @Override - public int size() { - return numMessages; - } - - @Override - public Iterator iterator() { - return new Iterator() { - int index = 0; - - @Override - public boolean hasNext() { - return index < numMessages; - } - - @Override - public String next() { - index++; - return readString(buf); - } - }; - } - }; - } - private static void writeString(final ByteBuf buf, final String s) { - final byte[] bytes = s.getBytes(StandardCharsets.UTF_8); + final byte[] bytes = Schema.STRING.encode(s); buf.writeShort(bytes.length); buf.writeBytes(bytes); } - private static String readString(final ByteBuf buf) { + public static Metadata readMetadata(final ByteBuf buf) { + return new Metadata(buf.readShort()); + } + + public static byte[] readMessage(final ByteBuf buf) { final short length = buf.readShort(); final byte[] bytes = new byte[length]; buf.readBytes(bytes); - return new String(bytes, StandardCharsets.UTF_8); + return bytes; + } + + public static List deserialize(final ByteBuf buf) { + final Metadata metadata = readMetadata(buf); + final List strings = new ArrayList<>(); + for (int i = 0; i < metadata.getNumMessages(); i++) { + strings.add(Schema.STRING.decode(readMessage(buf))); + } + return strings; } @Test @@ -109,8 +96,7 @@ public void testMultipleStrings() { for (List input : inputs) { final ByteBuf buf = serialize(input); - final List parsedTokens = new ArrayList<>(); - deserialize(buf).forEach(parsedTokens::add); + final List parsedTokens = deserialize(buf); Assert.assertEquals(parsedTokens, input); Assert.assertEquals(parsedTokens.size(), input.size()); From a551152fba3f115af1e79a65219136ae81bc3170 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Thu, 30 Sep 2021 09:54:16 +0800 Subject: [PATCH 22/29] Catch Throwable in pulsar-client --- .../pulsar/client/api/PayloadProcessor.java | 24 +++++++------------ .../pulsar/client/impl/ConsumerImpl.java | 6 ++--- 2 files changed, 12 insertions(+), 18 deletions(-) diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadProcessor.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadProcessor.java index 361cab07e94ac..f46b574f36cd8 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadProcessor.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadProcessor.java @@ -40,14 +40,13 @@ public interface PayloadProcessor { * @param context the message context that contains the message format information and methods to create a message * @param schema the message's schema * @param messageConsumer the callback to consume each message - * @param throwableConsumer the callback to handle the exception * @param + * @throws Exception */ void process(MessagePayload payload, EntryContext context, Schema schema, - java.util.function.Consumer> messageConsumer, - Consumer throwableConsumer); + Consumer> messageConsumer) throws Exception; // The default processor for Pulsar format payload. It should be noted getNumMessages() and isBatch() methods of // EntryContext only work for Pulsar format. For other formats, the message metadata might be stored in the payload. @@ -57,19 +56,14 @@ void process(MessagePayload payload, public void process(MessagePayload payload, EntryContext context, Schema schema, - Consumer> messageConsumer, - Consumer throwableConsumer) { - try { - if (context.isBatch()) { - final int numMessages = context.getNumMessages(); - for (int i = 0; i < numMessages; i++) { - messageConsumer.accept(context.getMessageAt(i, numMessages, payload, true, schema)); - } - } else { - messageConsumer.accept(context.asSingleMessage(payload, schema)); + Consumer> messageConsumer) { + if (context.isBatch()) { + final int numMessages = context.getNumMessages(); + for (int i = 0; i < numMessages; i++) { + messageConsumer.accept(context.getMessageAt(i, numMessages, payload, true, schema)); } - } catch (Throwable throwable) { - throwableConsumer.accept(throwable); + } else { + messageConsumer.accept(context.asSingleMessage(payload, schema)); } } }; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 71fd1436026cb..eb4835176510c 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -1104,10 +1104,10 @@ private void processPayloadByProcessor(final BrokerEntryMetadata brokerEntryMeta skippedMessages.incrementAndGet(); } executeNotifyCallback((MessageImpl) message); - }, throwable -> { - log.warn("[{}] [{}] unable to obtain message in batch", subscription, consumerName, throwable); - discardCorruptedMessage(messageId, cnx(), ValidationError.BatchDeSerializeError); }); + } catch (Throwable throwable) { + log.warn("[{}] [{}] unable to obtain message in batch", subscription, consumerName, throwable); + discardCorruptedMessage(messageId, cnx(), ValidationError.BatchDeSerializeError); } finally { entryContext.recycle(); payload.release(); // byteBuf.release() is called in this method From 4bd13b2975b240db89c8b4a248e92b6332509dac Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Thu, 30 Sep 2021 16:07:36 +0800 Subject: [PATCH 23/29] Rename some interfaces and classes --- ....java => MessagePayloadProcessorTest.java} | 0 .../pulsar/client/api/ConsumerBuilder.java | 2 +- ...ontext.java => MessagePayloadContext.java} | 25 +++++++++-------- ...ssor.java => MessagePayloadProcessor.java} | 16 +++++------ .../client/impl/ConsumerBuilderImpl.java | 4 +-- .../pulsar/client/impl/ConsumerImpl.java | 3 +- ...pl.java => MessagePayloadContextImpl.java} | 28 +++++++++---------- .../impl/conf/ConsumerConfigurationData.java | 4 +-- 8 files changed, 41 insertions(+), 41 deletions(-) rename pulsar-broker/src/test/java/org/apache/pulsar/client/processor/{PayloadProcessorTest.java => MessagePayloadProcessorTest.java} (100%) rename pulsar-client-api/src/main/java/org/apache/pulsar/client/api/{EntryContext.java => MessagePayloadContext.java} (75%) rename pulsar-client-api/src/main/java/org/apache/pulsar/client/api/{PayloadProcessor.java => MessagePayloadProcessor.java} (82%) rename pulsar-client/src/main/java/org/apache/pulsar/client/impl/{EntryContextImpl.java => MessagePayloadContextImpl.java} (80%) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/PayloadProcessorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/MessagePayloadProcessorTest.java similarity index 100% rename from pulsar-broker/src/test/java/org/apache/pulsar/client/processor/PayloadProcessorTest.java rename to pulsar-broker/src/test/java/org/apache/pulsar/client/processor/MessagePayloadProcessorTest.java diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java index 2ed35f92781fe..3c3ce177177f3 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ConsumerBuilder.java @@ -748,5 +748,5 @@ public interface ConsumerBuilder extends Cloneable { * * Default: null */ - ConsumerBuilder payloadProcessor(PayloadProcessor payloadProcessor); + ConsumerBuilder messagePayloadProcessor(MessagePayloadProcessor payloadProcessor); } diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/EntryContext.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessagePayloadContext.java similarity index 75% rename from pulsar-client-api/src/main/java/org/apache/pulsar/client/api/EntryContext.java rename to pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessagePayloadContext.java index a6399ec4b0412..a444de724166f 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/EntryContext.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessagePayloadContext.java @@ -19,36 +19,37 @@ package org.apache.pulsar.client.api; /** - * The context of an entry, which usually represents a message of a batch if batching is enabled. + * The context of the message payload, which usually represents a batched message (batch) or a single message. */ -public interface EntryContext { +public interface MessagePayloadContext { /** * Get a value associated with the given key. * + * When the message payload is not produced by Pulsar producer, a specific property is usually added to indicate the + * format. So this method is useful to determine whether the payload is produced by Pulsar producer. + * * @param key * @return the value associated with the key or null if the key or value doesn't exist */ String getProperty(String key); /** - * Get the number of messages. - * - * Since the message could be batched, a message could have multiple internal single messages. + * Get the number of messages when the payload is produced by Pulsar producer. * - * @return the number of internal single messages or 1 if the message is not batched. + * @return the number of messages */ int getNumMessages(); /** - * Check whether the entry is a batch. + * Check whether the payload is a batch when the payload is produced by Pulsar producer. * - * @return true if the entry is a batch. + * @return true if the payload is a batch */ boolean isBatch(); /** - * Get the internal single message with a specific index from a payload if the entry is a batch. + * Get the internal single message with a specific index from a payload if the payload is a batch. * * @param index the batch index * @param numMessages the number of messages in the batch @@ -56,7 +57,7 @@ public interface EntryContext { * @param containMetadata whether the payload contains the single message metadata * @param schema the schema of the batch * @param - * @return the single message + * @return the created message * @implNote The `index` and `numMessages` parameters are used to create the message id with batch index. * If `containMetadata` is true, parse the single message metadata from the payload first. The fields of single * message metadata will overwrite the same fields of the entry's metadata. @@ -68,12 +69,12 @@ Message getMessageAt(int index, Schema schema); /** - * Convert the given payload to a single message if the entry is non-batched. + * Convert the given payload to a single message if the entry is not a batch. * * @param payload the message payload * @param schema the schema of the message * @param - * @return the converted single message + * @return the created message */ Message asSingleMessage(MessagePayload payload, Schema schema); } diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadProcessor.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessagePayloadProcessor.java similarity index 82% rename from pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadProcessor.java rename to pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessagePayloadProcessor.java index f46b574f36cd8..1d50a50627942 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PayloadProcessor.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessagePayloadProcessor.java @@ -26,12 +26,12 @@ * It's responsible to convert the raw buffer to some messages, then trigger some callbacks so that consumer can consume * these messages and handle the exception if it existed. * - * The most important part is to decode the raw buffer. After that, we can call {@link EntryContext#getMessageAt} or - * {@link EntryContext#asSingleMessage} to construct {@link Message} for consumer to consume. Since we need to pass the - * {@link MessagePayload} object to these methods, we can use {@link MessagePayloadFactory#DEFAULT} to create it or just - * reuse the payload argument. + * The most important part is to decode the raw buffer. After that, we can call + * {@link MessagePayloadContext#getMessageAt} or {@link MessagePayloadContext#asSingleMessage} to construct + * {@link Message} for consumer to consume. Since we need to pass the {@link MessagePayload} object to these methods, we + * can use {@link MessagePayloadFactory#DEFAULT} to create it or just reuse the payload argument. */ -public interface PayloadProcessor { +public interface MessagePayloadProcessor { /** * Process the message payload. @@ -44,17 +44,17 @@ public interface PayloadProcessor { * @throws Exception */ void process(MessagePayload payload, - EntryContext context, + MessagePayloadContext context, Schema schema, Consumer> messageConsumer) throws Exception; // The default processor for Pulsar format payload. It should be noted getNumMessages() and isBatch() methods of // EntryContext only work for Pulsar format. For other formats, the message metadata might be stored in the payload. - PayloadProcessor DEFAULT = new PayloadProcessor() { + MessagePayloadProcessor DEFAULT = new MessagePayloadProcessor() { @Override public void process(MessagePayload payload, - EntryContext context, + MessagePayloadContext context, Schema schema, Consumer> messageConsumer) { if (context.isBatch()) { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java index 258cef689442a..cbfc27d2b0381 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBuilderImpl.java @@ -44,7 +44,7 @@ import org.apache.pulsar.client.api.KeySharedPolicy; import org.apache.pulsar.client.api.MessageCrypto; import org.apache.pulsar.client.api.MessageListener; -import org.apache.pulsar.client.api.PayloadProcessor; +import org.apache.pulsar.client.api.MessagePayloadProcessor; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.PulsarClientException.InvalidConfigurationException; import org.apache.pulsar.client.api.RegexSubscriptionMode; @@ -482,7 +482,7 @@ public ConsumerBuilder poolMessages(boolean poolMessages) { } @Override - public ConsumerBuilder payloadProcessor(PayloadProcessor payloadProcessor) { + public ConsumerBuilder messagePayloadProcessor(MessagePayloadProcessor payloadProcessor) { conf.setPayloadProcessor(payloadProcessor); return this; } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index eb4835176510c..a03259f856eff 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -68,7 +68,6 @@ import org.apache.pulsar.client.api.MessageCrypto; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Messages; -import org.apache.pulsar.client.api.PayloadProcessor; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.PulsarClientException.TopicDoesNotExistException; @@ -1095,7 +1094,7 @@ private void processPayloadByProcessor(final BrokerEntryMetadata brokerEntryMeta final int redeliveryCount, final List ackSet) { final MessagePayloadImpl payload = MessagePayloadImpl.create(byteBuf); - final EntryContextImpl entryContext = EntryContextImpl.get( + final MessagePayloadContextImpl entryContext = MessagePayloadContextImpl.get( brokerEntryMetadata, messageMetadata, messageId, this, redeliveryCount, ackSet); final AtomicInteger skippedMessages = new AtomicInteger(0); try { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/EntryContextImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadContextImpl.java similarity index 80% rename from pulsar-client/src/main/java/org/apache/pulsar/client/impl/EntryContextImpl.java rename to pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadContextImpl.java index 907893f009446..aa6cab80db2bc 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/EntryContextImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadContextImpl.java @@ -22,7 +22,7 @@ import io.netty.util.Recycler; import java.util.List; import lombok.NonNull; -import org.apache.pulsar.client.api.EntryContext; +import org.apache.pulsar.client.api.MessagePayloadContext; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessagePayload; import org.apache.pulsar.client.api.Schema; @@ -33,16 +33,16 @@ import org.apache.pulsar.common.util.SafeCollectionUtils; import org.apache.pulsar.common.util.collections.BitSetRecyclable; -public class EntryContextImpl implements EntryContext { +public class MessagePayloadContextImpl implements MessagePayloadContext { - private static final Recycler RECYCLER = new Recycler() { + private static final Recycler RECYCLER = new Recycler() { @Override - protected EntryContextImpl newObject(Handle handle) { - return new EntryContextImpl(handle); + protected MessagePayloadContextImpl newObject(Handle handle) { + return new MessagePayloadContextImpl(handle); } }; - private final Recycler.Handle recyclerHandle; + private final Recycler.Handle recyclerHandle; private BrokerEntryMetadata brokerEntryMetadata; private MessageMetadata messageMetadata; private SingleMessageMetadata singleMessageMetadata; @@ -52,17 +52,17 @@ protected EntryContextImpl newObject(Handle handle) { private BatchMessageAcker acker; private BitSetRecyclable ackBitSet; - private EntryContextImpl(final Recycler.Handle handle) { + private MessagePayloadContextImpl(final Recycler.Handle handle) { this.recyclerHandle = handle; } - public static EntryContextImpl get(final BrokerEntryMetadata brokerEntryMetadata, - @NonNull final MessageMetadata messageMetadata, - @NonNull final MessageIdImpl messageId, - @NonNull final ConsumerImpl consumer, - final int redeliveryCount, - final List ackSet) { - final EntryContextImpl context = RECYCLER.get(); + public static MessagePayloadContextImpl get(final BrokerEntryMetadata brokerEntryMetadata, + @NonNull final MessageMetadata messageMetadata, + @NonNull final MessageIdImpl messageId, + @NonNull final ConsumerImpl consumer, + final int redeliveryCount, + final List ackSet) { + final MessagePayloadContextImpl context = RECYCLER.get(); context.brokerEntryMetadata = brokerEntryMetadata; context.messageMetadata = messageMetadata; context.singleMessageMetadata = new SingleMessageMetadata(); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java index 5dcee82c82ea1..107694637b3a4 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/conf/ConsumerConfigurationData.java @@ -42,7 +42,7 @@ import org.apache.pulsar.client.api.KeySharedPolicy; import org.apache.pulsar.client.api.MessageCrypto; import org.apache.pulsar.client.api.MessageListener; -import org.apache.pulsar.client.api.PayloadProcessor; +import org.apache.pulsar.client.api.MessagePayloadProcessor; import org.apache.pulsar.client.api.RegexSubscriptionMode; import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.api.SubscriptionMode; @@ -151,7 +151,7 @@ public int getMaxPendingChuckedMessage() { private boolean poolMessages = false; @JsonIgnore - private transient PayloadProcessor payloadProcessor = null; + private transient MessagePayloadProcessor payloadProcessor = null; public void setAutoUpdatePartitionsIntervalSeconds(int interval, TimeUnit timeUnit) { checkArgument(interval > 0, "interval needs to be > 0"); From 723c8f593c1024a7a3e7b8f784c9e66e1c2e14e3 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Thu, 30 Sep 2021 16:09:34 +0800 Subject: [PATCH 24/29] Fix tests --- .../processor/DefaultProcessorWithRefCnt.java | 14 +++++++------- .../processor/MessagePayloadProcessorTest.java | 7 ++++--- 2 files changed, 11 insertions(+), 10 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/DefaultProcessorWithRefCnt.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/DefaultProcessorWithRefCnt.java index 115e000792d22..63e295f2bbb0f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/DefaultProcessorWithRefCnt.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/DefaultProcessorWithRefCnt.java @@ -20,28 +20,28 @@ import java.util.function.Consumer; import lombok.Getter; -import org.apache.pulsar.client.api.EntryContext; +import org.apache.pulsar.client.api.MessagePayloadContext; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessagePayload; -import org.apache.pulsar.client.api.PayloadProcessor; +import org.apache.pulsar.client.api.MessagePayloadProcessor; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.MessagePayloadImpl; /** * The processor for Pulsar format messages and maintains a total reference count. * - * It's used to verify {@link EntryContext#getMessageAt} and {@link EntryContext#asSingleMessage} have release the + * It's used to verify {@link MessagePayloadContext#getMessageAt} and {@link MessagePayloadContext#asSingleMessage} have release the * ByteBuf successfully. */ -public class DefaultProcessorWithRefCnt implements PayloadProcessor { +public class DefaultProcessorWithRefCnt implements MessagePayloadProcessor { @Getter int totalRefCnt = 0; @Override - public void process(MessagePayload payload, EntryContext context, Schema schema, - Consumer> messageConsumer, Consumer throwableConsumer) { - PayloadProcessor.DEFAULT.process(payload, context, schema, messageConsumer, throwableConsumer); + public void process(MessagePayload payload, MessagePayloadContext context, Schema schema, + Consumer> messageConsumer) throws Exception { + MessagePayloadProcessor.DEFAULT.process(payload, context, schema, messageConsumer); totalRefCnt += ((MessagePayloadImpl) payload).getByteBuf().refCnt(); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/MessagePayloadProcessorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/MessagePayloadProcessorTest.java index 9a08ae8acd5a2..e6a999d188565 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/MessagePayloadProcessorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/MessagePayloadProcessorTest.java @@ -27,6 +27,7 @@ import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessagePayloadProcessor; import org.apache.pulsar.client.api.MessageRouter; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerConsumerBase; @@ -42,10 +43,10 @@ import org.testng.annotations.Test; /** - * Test for {@link org.apache.pulsar.client.api.PayloadProcessor}. + * Test for {@link MessagePayloadProcessor}. */ @Slf4j -public class PayloadProcessorTest extends ProducerConsumerBase { +public class MessagePayloadProcessorTest extends ProducerConsumerBase { @BeforeClass @Override @@ -124,7 +125,7 @@ public int choosePartition(Message msg, TopicMetadata metadata) { .topic(topic) .subscriptionName("sub") .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) - .payloadProcessor(processor) + .messagePayloadProcessor(processor) .subscribe(); final List values = new ArrayList<>(); for (int i = 0; i < numMessages; i++) { From 31e346dc08a1dc5966e0aa23035d7b37ba83b9c0 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Thu, 30 Sep 2021 18:28:06 +0800 Subject: [PATCH 25/29] Add tests for custom batch --- .../client/processor/CustomBatchFormat.java | 16 ++--- .../CustomBatchPayloadProcessor.java | 61 ++++++++++++++++ .../client/processor/CustomBatchProducer.java | 70 +++++++++++++++++++ .../MessagePayloadProcessorTest.java | 49 +++++++++++-- 4 files changed, 180 insertions(+), 16 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/processor/CustomBatchPayloadProcessor.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/processor/CustomBatchProducer.java diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/CustomBatchFormat.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/CustomBatchFormat.java index 92580d133c4ab..61c5b35a7950a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/CustomBatchFormat.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/CustomBatchFormat.java @@ -78,15 +78,6 @@ public static byte[] readMessage(final ByteBuf buf) { return bytes; } - public static List deserialize(final ByteBuf buf) { - final Metadata metadata = readMetadata(buf); - final List strings = new ArrayList<>(); - for (int i = 0; i < metadata.getNumMessages(); i++) { - strings.add(Schema.STRING.decode(readMessage(buf))); - } - return strings; - } - @Test public void testMultipleStrings() { final List> inputs = new ArrayList<>(); @@ -96,7 +87,12 @@ public void testMultipleStrings() { for (List input : inputs) { final ByteBuf buf = serialize(input); - final List parsedTokens = deserialize(buf); + + final Metadata metadata = readMetadata(buf); + final List parsedTokens = new ArrayList<>(); + for (int i = 0; i < metadata.getNumMessages(); i++) { + parsedTokens.add(Schema.STRING.decode(readMessage(buf))); + } Assert.assertEquals(parsedTokens, input); Assert.assertEquals(parsedTokens.size(), input.size()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/CustomBatchPayloadProcessor.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/CustomBatchPayloadProcessor.java new file mode 100644 index 0000000000000..c83b13b443087 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/CustomBatchPayloadProcessor.java @@ -0,0 +1,61 @@ +/** + * 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.processor; + +import io.netty.buffer.ByteBuf; +import java.util.function.Consumer; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.client.api.MessagePayloadContext; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessagePayload; +import org.apache.pulsar.client.api.MessagePayloadFactory; +import org.apache.pulsar.client.api.MessagePayloadProcessor; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.MessagePayloadUtils; + +@Slf4j +public class CustomBatchPayloadProcessor implements MessagePayloadProcessor { + + @Override + public void process(MessagePayload payload, MessagePayloadContext context, Schema schema, + Consumer> messageConsumer) throws Exception { + final String value = context.getProperty(CustomBatchFormat.KEY); + if (value == null || !value.equals(CustomBatchFormat.VALUE)) { + DEFAULT.process(payload, context, schema, messageConsumer); + return; + } + + final ByteBuf buf = MessagePayloadUtils.convertToByteBuf(payload); + try { + final int numMessages = CustomBatchFormat.readMetadata(buf).getNumMessages(); + for (int i = 0; i < numMessages; i++) { + final MessagePayload singlePayload = + MessagePayloadFactory.DEFAULT.wrap(CustomBatchFormat.readMessage(buf)); + try { + messageConsumer.accept( + context.getMessageAt(i, numMessages, singlePayload, false, schema)); + } finally { + singlePayload.release(); + } + } + } finally { + buf.release(); + } + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/CustomBatchProducer.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/CustomBatchProducer.java new file mode 100644 index 0000000000000..bc0720661e3f9 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/CustomBatchProducer.java @@ -0,0 +1,70 @@ +/** + * 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.processor; + +import io.netty.buffer.ByteBuf; +import java.util.ArrayList; +import java.util.List; +import lombok.RequiredArgsConstructor; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.common.api.proto.MessageMetadata; +import org.apache.pulsar.common.protocol.Commands; + +@RequiredArgsConstructor +@Slf4j +public class CustomBatchProducer { + + private final List messages = new ArrayList<>(); + private final PersistentTopic persistentTopic; + private final int batchingMaxMessages; + + public void sendAsync(final String value) { + messages.add(value); + if (messages.size() >= batchingMaxMessages) { + flush(); + } + } + + public void flush() { + final ByteBuf buf = CustomBatchFormat.serialize(messages); + final ByteBuf headerAndPayload = Commands.serializeMetadataAndPayload(Commands.ChecksumType.None, + createCustomMetadata(), buf); + buf.release(); + persistentTopic.publishMessage(headerAndPayload, (e, ledgerId, entryId) -> { + if (e == null) { + log.info("Send successfully to {} ({}, {})", persistentTopic.getName(), ledgerId, entryId); + } else { + log.error("Failed to send: {}", e.getMessage()); + } + }); + messages.clear(); + } + + private static MessageMetadata createCustomMetadata() { + final MessageMetadata messageMetadata = new MessageMetadata(); + // Here are required fields + messageMetadata.setProducerName(""); + messageMetadata.setSequenceId(0L); + messageMetadata.setPublishTime(0L); + // Add the property to identify the message format + messageMetadata.addProperty().setKey(CustomBatchFormat.KEY).setValue(CustomBatchFormat.VALUE); + return messageMetadata; + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/MessagePayloadProcessorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/MessagePayloadProcessorTest.java index e6a999d188565..8c6bc26f3d1cf 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/MessagePayloadProcessorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/MessagePayloadProcessorTest.java @@ -25,6 +25,7 @@ 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.MessagePayloadProcessor; @@ -76,10 +77,13 @@ public static Object[][] config() { }; } - private static int getNumBatches(final int numMessages, final int batchingMaxMessages) { - int numBatches = numMessages / batchingMaxMessages; - numBatches += (numMessages % batchingMaxMessages == 0) ? 0 : 1; - return numBatches; + @DataProvider + public static Object[][] customBatchConfig() { + return new Object[][] { + // numMessages / batchingMaxMessages + { 10, 1 }, + { 10, 4 } + }; } @Test(dataProvider = "config") @@ -133,7 +137,6 @@ public int choosePartition(Message msg, TopicMetadata metadata) { Assert.assertNotNull(message); values.add(message.getValue()); consumer.acknowledge(message.getMessageId()); - consumer.acknowledgeCumulative(message.getMessageId()); } if (numPartitions > 1) { @@ -148,9 +151,43 @@ public int choosePartition(Message msg, TopicMetadata metadata) { // 1. ConsumerImpl#processPayloadByProcessor // 2. PulsarDecoder#channelRead if (enableBatching) { - Assert.assertEquals(processor.getTotalRefCnt(), 2 * getNumBatches(numMessages, batchingMaxMessages)); + int numBatches = numMessages / batchingMaxMessages; + numBatches += (numMessages % batchingMaxMessages == 0) ? 0 : 1; + Assert.assertEquals(processor.getTotalRefCnt(), 2 * numBatches); } else { Assert.assertEquals(processor.getTotalRefCnt(), 2 * numMessages); } } + + @Test(dataProvider = "customBatchConfig") + public void testCustomProcessor(final int numMessages, final int batchingMaxMessages) throws Exception { + final String topic = "persistent://public/default/testCustomProcessor-" + + numMessages + "-" + batchingMaxMessages; + + @Cleanup + final Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .topic(topic) + .subscriptionName("sub") + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .messagePayloadProcessor(new CustomBatchPayloadProcessor()) + .subscribe(); + + final PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopicIfExists(topic).get().orElse(null); + Assert.assertNotNull(persistentTopic); + + final String messagePrefix = "msg-"; + final CustomBatchProducer producer = new CustomBatchProducer(persistentTopic, batchingMaxMessages); + for (int i = 0; i < numMessages; i++) { + producer.sendAsync(messagePrefix + i); + } + producer.flush(); + + for (int i = 0; i < numMessages; i++) { + final Message message = consumer.receive(1, TimeUnit.SECONDS); + Assert.assertNotNull(message); + Assert.assertEquals(message.getValue(), messagePrefix + i); + consumer.acknowledge(message.getMessageId()); + } + } } From 121685dc9a931447cc95ca51772a43418d491bd8 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 5 Oct 2021 13:20:47 +0800 Subject: [PATCH 26/29] Move CustomBatchFormat test to MessagePayloadProcessorTest for CI --- .../client/processor/CustomBatchFormat.java | 30 ------------------- .../MessagePayloadProcessorTest.java | 27 +++++++++++++++++ 2 files changed, 27 insertions(+), 30 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/CustomBatchFormat.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/CustomBatchFormat.java index 61c5b35a7950a..571d292aaa41e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/CustomBatchFormat.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/CustomBatchFormat.java @@ -19,16 +19,10 @@ package org.apache.pulsar.client.processor; import io.netty.buffer.ByteBuf; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; import lombok.AllArgsConstructor; import lombok.Getter; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; -import org.testng.Assert; -import org.testng.annotations.Test; /** * A batch message whose format is customized. @@ -77,28 +71,4 @@ public static byte[] readMessage(final ByteBuf buf) { buf.readBytes(bytes); return bytes; } - - @Test - public void testMultipleStrings() { - final List> inputs = new ArrayList<>(); - inputs.add(Collections.emptyList()); - inputs.add(Collections.singletonList("java")); - inputs.add(Arrays.asList("hello", "world", "java")); - - for (List input : inputs) { - final ByteBuf buf = serialize(input); - - final Metadata metadata = readMetadata(buf); - final List parsedTokens = new ArrayList<>(); - for (int i = 0; i < metadata.getNumMessages(); i++) { - parsedTokens.add(Schema.STRING.decode(readMessage(buf))); - } - - Assert.assertEquals(parsedTokens, input); - Assert.assertEquals(parsedTokens.size(), input.size()); - - Assert.assertEquals(buf.refCnt(), 1); - buf.release(); - } - } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/MessagePayloadProcessorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/MessagePayloadProcessorTest.java index 8c6bc26f3d1cf..6a12101fe68b6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/MessagePayloadProcessorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/processor/MessagePayloadProcessorTest.java @@ -20,9 +20,11 @@ import com.google.common.collect.Sets; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.concurrent.TimeUnit; +import io.netty.buffer.ByteBuf; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -47,6 +49,7 @@ * Test for {@link MessagePayloadProcessor}. */ @Slf4j +@Test(groups = "broker-impl") public class MessagePayloadProcessorTest extends ProducerConsumerBase { @BeforeClass @@ -159,6 +162,30 @@ public int choosePartition(Message msg, TopicMetadata metadata) { } } + @Test + public void testCustomBatchFormat() { + final List> inputs = new ArrayList<>(); + inputs.add(Collections.emptyList()); + inputs.add(Collections.singletonList("java")); + inputs.add(Arrays.asList("hello", "world", "java")); + + for (List input : inputs) { + final ByteBuf buf = CustomBatchFormat.serialize(input); + + final CustomBatchFormat.Metadata metadata = CustomBatchFormat.readMetadata(buf); + final List parsedTokens = new ArrayList<>(); + for (int i = 0; i < metadata.getNumMessages(); i++) { + parsedTokens.add(Schema.STRING.decode(CustomBatchFormat.readMessage(buf))); + } + + Assert.assertEquals(parsedTokens, input); + Assert.assertEquals(parsedTokens.size(), input.size()); + + Assert.assertEquals(buf.refCnt(), 1); + buf.release(); + } + } + @Test(dataProvider = "customBatchConfig") public void testCustomProcessor(final int numMessages, final int batchingMaxMessages) throws Exception { final String topic = "persistent://public/default/testCustomProcessor-" From 1cabbd2de86a35b5591a717cff012509cf25b350 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 5 Oct 2021 13:25:00 +0800 Subject: [PATCH 27/29] Return a copy of an empty array instead of a reference --- .../java/org/apache/pulsar/client/impl/MessagePayloadImpl.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadImpl.java index e5499f5e12ee6..d99cd8311f914 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadImpl.java @@ -36,7 +36,6 @@ protected MessagePayloadImpl newObject(Handle handle) { return new MessagePayloadImpl(handle); } }; - private static final byte[] EMPTY_BYTES = new byte[0]; private final Recycler.Handle recyclerHandle; @Getter @@ -67,7 +66,7 @@ public byte[] copiedBuffer() { byteBuf.getBytes(byteBuf.readerIndex(), bytes); return bytes; } else { - return EMPTY_BYTES; + return new byte[0]; } } } From b9ca0ed6e75b5fa3bd3bd864847e2102dc7a4fbd Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 5 Oct 2021 13:27:26 +0800 Subject: [PATCH 28/29] Revert changes for MessageImpl#setBrokerEntryMetadata --- .../org/apache/pulsar/client/impl/ConsumerImpl.java | 12 ++++++++---- .../org/apache/pulsar/client/impl/MessageImpl.java | 3 +-- 2 files changed, 9 insertions(+), 6 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index a03259f856eff..be077de831754 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -1049,10 +1049,12 @@ protected MessageImpl newSingleMessage(final int index, messageId.getEntryId(), getPartitionIndex(), index, numMessages, acker); final ByteBuf payloadBuffer = (singleMessagePayload != null) ? singleMessagePayload : payload; - return MessageImpl.create(topicName.toString(), batchMessageIdImpl, + final MessageImpl message = MessageImpl.create(topicName.toString(), batchMessageIdImpl, msgMetadata, singleMessageMetadata, payloadBuffer, createEncryptionContext(msgMetadata), cnx(), schema, redeliveryCount, poolMessages - ).setBrokerEntryMetadata(brokerEntryMetadata); + ); + message.setBrokerEntryMetadata(brokerEntryMetadata); + return message; } catch (IOException | IllegalStateException e) { throw new IllegalStateException(e); } finally { @@ -1068,9 +1070,11 @@ protected MessageImpl newMessage(final MessageIdImpl messageId, final ByteBuf payload, final Schema schema, final int redeliveryCount) { - return MessageImpl.create(topicName.toString(), messageId, messageMetadata, payload, + final MessageImpl message = MessageImpl.create(topicName.toString(), messageId, messageMetadata, payload, createEncryptionContext(messageMetadata), cnx(), schema, redeliveryCount, poolMessages - ).setBrokerEntryMetadata(brokerEntryMetadata); + ); + message.setBrokerEntryMetadata(brokerEntryMetadata); + return message; } private void executeNotifyCallback(final MessageImpl message) { 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 a00cb35071f68..67c176cfe63bf 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 @@ -625,9 +625,8 @@ public BrokerEntryMetadata getBrokerEntryMetadata() { return brokerEntryMetadata; } - public MessageImpl setBrokerEntryMetadata(BrokerEntryMetadata brokerEntryMetadata) { + public void setBrokerEntryMetadata(BrokerEntryMetadata brokerEntryMetadata) { this.brokerEntryMetadata = brokerEntryMetadata; - return this; } public ClientCnx getCnx() { From c9e8a1f8cb08c2ed26274dae8c7abd0f3474794c Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 5 Oct 2021 13:29:49 +0800 Subject: [PATCH 29/29] Skip null messages --- .../java/org/apache/pulsar/client/impl/ConsumerImpl.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index be077de831754..97fb13f89fa35 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -1103,10 +1103,11 @@ private void processPayloadByProcessor(final BrokerEntryMetadata brokerEntryMeta final AtomicInteger skippedMessages = new AtomicInteger(0); try { conf.getPayloadProcessor().process(payload, entryContext, schema, message -> { - if (message == null) { + if (message != null) { + executeNotifyCallback((MessageImpl) message); + } else { skippedMessages.incrementAndGet(); } - executeNotifyCallback((MessageImpl) message); }); } catch (Throwable throwable) { log.warn("[{}] [{}] unable to obtain message in batch", subscription, consumerName, throwable);