diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentFailoverE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentFailoverE2ETest.java index ffc1444676b23b..2e1719ad9f39f5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentFailoverE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentFailoverE2ETest.java @@ -40,11 +40,11 @@ import org.apache.pulsar.client.api.ConsumerEventListener; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.client.api.MessageRoutingMode; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.SubscriptionType; -import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.util.FutureUtil; @@ -336,8 +336,7 @@ public void testSimpleConsumerEventsWithPartition() throws Exception { } totalMessages++; consumer1.acknowledge(msg); - MessageIdImpl msgId = MessageIdImpl.convertToMessageIdImpl(msg.getMessageId()); - receivedPtns.add(msgId.getPartitionIndex()); + receivedPtns.add(((MessageIdAdv) msg.getMessageId()).getPartitionIndex()); } assertTrue(Sets.difference(listener1.activePtns, receivedPtns).isEmpty()); @@ -353,8 +352,7 @@ public void testSimpleConsumerEventsWithPartition() throws Exception { } totalMessages++; consumer2.acknowledge(msg); - MessageIdImpl msgId = MessageIdImpl.convertToMessageIdImpl(msg.getMessageId()); - receivedPtns.add(msgId.getPartitionIndex()); + receivedPtns.add(((MessageIdAdv) msg.getMessageId()).getPartitionIndex()); } assertTrue(Sets.difference(listener1.inactivePtns, receivedPtns).isEmpty()); assertTrue(Sets.difference(listener2.activePtns, receivedPtns).isEmpty()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java index 2c2f62529d20a5..808de133394401 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java @@ -678,7 +678,7 @@ public void testSeekByFunction() throws Exception { if (message == null) { break; } - received.add(MessageIdImpl.convertToMessageIdImpl(message.getMessageId())); + received.add(message.getMessageId()); } int msgNumFromPartition1 = list.size() / 2; int msgNumFromPartition2 = 1; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/CustomMessageIdTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/CustomMessageIdTest.java new file mode 100644 index 00000000000000..52bfc9dda37e49 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/CustomMessageIdTest.java @@ -0,0 +1,142 @@ +/* + * 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 static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; +import java.util.ArrayList; +import java.util.concurrent.TimeUnit; +import lombok.Cleanup; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +@Test(groups = "broker-api") +public class CustomMessageIdTest extends ProducerConsumerBase { + + @BeforeClass + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @DataProvider + public static Object[][] enableBatching() { + return new Object[][]{ + { true }, + { false } + }; + } + + @Test + public void testSeek() throws Exception { + final var topic = "persistent://my-property/my-ns/test-seek-" + System.currentTimeMillis(); + @Cleanup final var producer = pulsarClient.newProducer(Schema.INT32).topic(topic).create(); + final var msgIds = new ArrayList(); + for (int i = 0; i < 10; i++) { + msgIds.add(new SimpleMessageIdImpl((MessageIdAdv) producer.send(i))); + } + @Cleanup final var consumer = pulsarClient.newConsumer(Schema.INT32) + .topic(topic).subscriptionName("sub").subscribe(); + consumer.seek(msgIds.get(6)); + final var msg = consumer.receive(3, TimeUnit.SECONDS); + assertNotNull(msg); + assertEquals(msg.getValue(), 7); + } + + @Test(dataProvider = "enableBatching") + public void testAcknowledgment(boolean enableBatching) throws Exception { + final var topic = "persistent://my-property/my-ns/test-ack-" + + enableBatching + System.currentTimeMillis(); + final var producer = pulsarClient.newProducer(Schema.INT32) + .topic(topic) + .enableBatching(enableBatching) + .batchingMaxMessages(10) + .batchingMaxPublishDelay(300, TimeUnit.MILLISECONDS) + .create(); + final var consumer = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName("sub") + .enableBatchIndexAcknowledgment(true) + .isAckReceiptEnabled(true) + .subscribe(); + for (int i = 0; i < 10; i++) { + producer.sendAsync(i); + } + final var msgIds = new ArrayList(); + for (int i = 0; i < 10; i++) { + final var msg = consumer.receive(); + final var msgId = new SimpleMessageIdImpl((MessageIdAdv) msg.getMessageId()); + msgIds.add(msgId); + if (enableBatching) { + assertTrue(msgId.getBatchIndex() >= 0 && msgId.getBatchSize() > 0); + } else { + assertFalse(msgId.getBatchIndex() >= 0 && msgId.getBatchSize() > 0); + } + } + consumer.acknowledgeCumulative(msgIds.get(8)); + consumer.redeliverUnacknowledgedMessages(); + final var msg = consumer.receive(3, TimeUnit.SECONDS); + assertNotNull(msg); + assertEquals(msg.getValue(), 9); + } + + private record SimpleMessageIdImpl(long ledgerId, long entryId, int batchIndex, int batchSize) + implements MessageIdAdv { + + public SimpleMessageIdImpl(MessageIdAdv msgId) { + this(msgId.getLedgerId(), msgId.getEntryId(), msgId.getBatchIndex(), msgId.getBatchSize()); + } + + @Override + public byte[] toByteArray() { + return new byte[0]; // never used + } + + @Override + public long getLedgerId() { + return ledgerId; + } + + @Override + public long getEntryId() { + return entryId; + } + + @Override + public int getBatchIndex() { + return batchIndex; + } + + @Override + public int getBatchSize() { + return batchSize; + } + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PartitionedProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PartitionedProducerConsumerTest.java index cd384e587898d5..13ae991a0e89c5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PartitionedProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PartitionedProducerConsumerTest.java @@ -767,7 +767,7 @@ public void testMessageIdForSubscribeToSinglePartition() throws Exception { for (int i = 0; i < totalMessages; i ++) { msg = consumer1.receive(5, TimeUnit.SECONDS); - Assert.assertEquals(MessageIdImpl.convertToMessageIdImpl(msg.getMessageId()).getPartitionIndex(), 2); + Assert.assertEquals(((MessageIdAdv) msg.getMessageId()).getPartitionIndex(), 2); consumer1.acknowledge(msg); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConsumerAckTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConsumerAckTest.java index 42da60906483cb..a83283bc267b5b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConsumerAckTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConsumerAckTest.java @@ -176,10 +176,10 @@ private AckTestData prepareDataForAck(String topic) throws PulsarClientException messageIds.add(message.getMessageId()); } MessageId firstEntryMessageId = messageIds.get(0); - MessageId secondEntryMessageId = ((BatchMessageIdImpl) messageIds.get(1)).toMessageIdImpl(); + MessageId secondEntryMessageId = MessageIdAdvUtils.discardBatch(messageIds.get(1)); // Verify messages 2 to N must be in the same entry for (int i = 2; i < messageIds.size(); i++) { - assertEquals(((BatchMessageIdImpl) messageIds.get(i)).toMessageIdImpl(), secondEntryMessageId); + assertEquals(MessageIdAdvUtils.discardBatch(messageIds.get(i)), secondEntryMessageId); } assertTrue(interceptor.individualAckedMessageIdList.isEmpty()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageIdTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageIdTest.java index ceb5c51e6aa77f..375bbff8a4df49 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageIdTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageIdTest.java @@ -118,9 +118,6 @@ public void producerSendAsync(TopicType topicType) throws PulsarClientException, Message message = consumer.receive(); assertEquals(new String(message.getData()), messagePrefix + i); MessageId messageId = message.getMessageId(); - if (topicType == TopicType.PARTITIONED) { - messageId = MessageIdImpl.convertToMessageIdImpl(messageId); - } assertTrue(messageIds.remove(messageId), "Failed to receive message"); } log.info("Remaining message IDs = {}", messageIds); diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessageIdAdv.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessageIdAdv.java new file mode 100644 index 00000000000000..73ecfed0ad059d --- /dev/null +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessageIdAdv.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.api; + +import java.util.BitSet; + +/** + * The {@link MessageId} interface provided for advanced users. + *

+ * All built-in MessageId implementations should be able to be cast to MessageIdAdv. + *

+ */ +public interface MessageIdAdv extends MessageId { + + /** + * Get the ledger ID. + * + * @return the ledger ID + */ + long getLedgerId(); + + /** + * Get the entry ID. + * + * @return the entry ID + */ + long getEntryId(); + + /** + * Get the partition index. + * + * @return -1 if the message is from a non-partitioned topic, otherwise the non-negative partition index + */ + default int getPartitionIndex() { + return -1; + } + + /** + * Get the batch index. + * + * @return -1 if the message is not in a batch + */ + default int getBatchIndex() { + return -1; + } + + /** + * Get the batch size. + * + * @return 0 if the message is not in a batch + */ + default int getBatchSize() { + return 0; + } + + /** + * Get the BitSet that indicates which messages in the batch. + * + * @implNote The message IDs of a batch should share a BitSet. For example, given 3 messages in the same batch whose + * size is 3, all message IDs of them should return "111" (i.e. a BitSet whose size is 3 and all bits are 1). If the + * 1st message has been acknowledged, the returned BitSet should become "011" (i.e. the 1st bit become 0). + * + * @return null if the message is a non-batched message + */ + default BitSet getAckSet() { + return null; + } + + /** + * Get the message ID of the first chunk if the current message ID represents the position of a chunked message. + * + * @implNote A chunked message is distributed across different BookKeeper entries. The message ID of a chunked + * message is composed of two message IDs that represent positions of the first and the last chunk. The message ID + * itself represents the position of the last chunk. + * + * @return null if the message is not a chunked message + */ + default MessageIdAdv getFirstChunkMessageId() { + return null; + } + + /** + * The default implementation of {@link Comparable#compareTo(Object)}. + */ + default int compareTo(MessageId o) { + if (!(o instanceof MessageIdAdv)) { + throw new UnsupportedOperationException("Unknown MessageId type: " + + ((o != null) ? o.getClass().getName() : "null")); + } + final MessageIdAdv other = (MessageIdAdv) o; + int result = Long.compare(this.getLedgerId(), other.getLedgerId()); + if (result != 0) { + return result; + } + result = Long.compare(this.getEntryId(), other.getEntryId()); + if (result != 0) { + return result; + } + // TODO: Correct the following compare logics, see https://github.com/apache/pulsar/pull/18981 + result = Integer.compare(this.getPartitionIndex(), other.getPartitionIndex()); + if (result != 0) { + return result; + } + return Integer.compare(this.getBatchIndex(), other.getBatchIndex()); + } +} diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/TopicMessageId.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/TopicMessageId.java index f6109d5f8e87e7..b70267bb0fb8b1 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/TopicMessageId.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/TopicMessageId.java @@ -18,6 +18,8 @@ */ package org.apache.pulsar.client.api; +import java.util.BitSet; + /** * The MessageId used for a consumer that subscribes multiple topics or partitioned topics. * @@ -49,13 +51,13 @@ static TopicMessageId create(String topic, MessageId messageId) { /** * The simplest implementation of a TopicMessageId interface. */ - class Impl implements TopicMessageId { + class Impl implements MessageIdAdv, TopicMessageId { private final String topic; - private final MessageId messageId; + private final MessageIdAdv messageId; public Impl(String topic, MessageId messageId) { this.topic = topic; - this.messageId = messageId; + this.messageId = (MessageIdAdv) messageId; } @Override @@ -68,6 +70,41 @@ public String getOwnerTopic() { return topic; } + @Override + public long getLedgerId() { + return messageId.getLedgerId(); + } + + @Override + public long getEntryId() { + return messageId.getEntryId(); + } + + @Override + public int getPartitionIndex() { + return messageId.getPartitionIndex(); + } + + @Override + public int getBatchIndex() { + return messageId.getBatchIndex(); + } + + @Override + public int getBatchSize() { + return messageId.getBatchSize(); + } + + @Override + public BitSet getAckSet() { + return messageId.getAckSet(); + } + + @Override + public MessageIdAdv getFirstChunkMessageId() { + return messageId.getFirstChunkMessageId(); + } + @Override public int compareTo(MessageId o) { return messageId.compareTo(o); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AcknowledgmentsGroupingTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AcknowledgmentsGroupingTracker.java index d46af1a99e7f06..60d7135e5e4ae0 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AcknowledgmentsGroupingTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AcknowledgmentsGroupingTracker.java @@ -31,7 +31,7 @@ public interface AcknowledgmentsGroupingTracker extends AutoCloseable { boolean isDuplicate(MessageId messageId); - CompletableFuture addAcknowledgment(MessageIdImpl msgId, AckType ackType, Map properties); + CompletableFuture addAcknowledgment(MessageId msgId, AckType ackType, Map properties); CompletableFuture addListAcknowledgment(List messageIds, AckType ackType, Map properties); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageAcker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageAcker.java deleted file mode 100644 index 1c9b66fd2bad5c..00000000000000 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageAcker.java +++ /dev/null @@ -1,95 +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.impl; - -import java.util.BitSet; - -public class BatchMessageAcker { - - private BatchMessageAcker() { - this.bitSet = new BitSet(); - this.batchSize = 0; - } - - static BatchMessageAcker newAcker(int batchSize) { - BitSet bitSet = new BitSet(batchSize); - bitSet.set(0, batchSize); - return new BatchMessageAcker(bitSet, batchSize); - } - - // Use the param bitSet as the BatchMessageAcker's bitSet, don't care about the batchSize. - static BatchMessageAcker newAcker(BitSet bitSet) { - return new BatchMessageAcker(bitSet, -1); - } - - // bitset shared across messages in the same batch. - private final int batchSize; - private final BitSet bitSet; - private volatile boolean prevBatchCumulativelyAcked = false; - - BatchMessageAcker(BitSet bitSet, int batchSize) { - this.bitSet = bitSet; - this.batchSize = batchSize; - } - - BitSet getBitSet() { - return bitSet; - } - - public synchronized int getBatchSize() { - return batchSize; - } - - public synchronized boolean ackIndividual(int batchIndex) { - bitSet.clear(batchIndex); - return bitSet.isEmpty(); - } - - public synchronized int getBitSetSize() { - return bitSet.size(); - } - - public synchronized boolean ackCumulative(int batchIndex) { - // +1 since to argument is exclusive - bitSet.clear(0, batchIndex + 1); - return bitSet.isEmpty(); - } - - // debug purpose - public synchronized int getOutstandingAcks() { - return bitSet.cardinality(); - } - - public void setPrevBatchCumulativelyAcked(boolean acked) { - this.prevBatchCumulativelyAcked = acked; - } - - public boolean isPrevBatchCumulativelyAcked() { - return prevBatchCumulativelyAcked; - } - - @Override - public String toString() { - return "BatchMessageAcker{" - + "batchSize=" + batchSize - + ", bitSet=" + bitSet - + ", prevBatchCumulativelyAcked=" + prevBatchCumulativelyAcked - + '}'; - } -} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageAckerDisabled.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageAckerDisabled.java deleted file mode 100644 index b70c928b296509..00000000000000 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageAckerDisabled.java +++ /dev/null @@ -1,50 +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.impl; - -import java.util.BitSet; - -class BatchMessageAckerDisabled extends BatchMessageAcker { - - static final BatchMessageAckerDisabled INSTANCE = new BatchMessageAckerDisabled(); - - private BatchMessageAckerDisabled() { - super(new BitSet(), 0); - } - - @Override - public synchronized int getBatchSize() { - return 0; - } - - @Override - public boolean ackIndividual(int batchIndex) { - return true; - } - - @Override - public boolean ackCumulative(int batchIndex) { - return true; - } - - @Override - public int getOutstandingAcks() { - return 0; - } -} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageIdImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageIdImpl.java index ed28082ff6a30f..4332aebcc6992b 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageIdImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageIdImpl.java @@ -18,20 +18,16 @@ */ package org.apache.pulsar.client.impl; -import javax.annotation.Nonnull; -import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.TopicMessageId; +import java.util.BitSet; +import org.apache.pulsar.client.api.MessageIdAdv; -/** - */ public class BatchMessageIdImpl extends MessageIdImpl { private static final long serialVersionUID = 1L; - static final int NO_BATCH = -1; private final int batchIndex; private final int batchSize; - private final transient BatchMessageAcker acker; + private final BitSet ackSet; // Private constructor used only for json deserialization @SuppressWarnings("unused") @@ -40,59 +36,35 @@ private BatchMessageIdImpl() { } public BatchMessageIdImpl(long ledgerId, long entryId, int partitionIndex, int batchIndex) { - this(ledgerId, entryId, partitionIndex, batchIndex, 0, BatchMessageAckerDisabled.INSTANCE); + this(ledgerId, entryId, partitionIndex, batchIndex, 0, null); } public BatchMessageIdImpl(long ledgerId, long entryId, int partitionIndex, int batchIndex, int batchSize, - BatchMessageAcker acker) { + BitSet ackSet) { super(ledgerId, entryId, partitionIndex); this.batchIndex = batchIndex; this.batchSize = batchSize; - this.acker = acker; + this.ackSet = ackSet; } - public BatchMessageIdImpl(MessageIdImpl other) { - super(other.ledgerId, other.entryId, other.partitionIndex); - if (other instanceof BatchMessageIdImpl) { - BatchMessageIdImpl otherId = (BatchMessageIdImpl) other; - this.batchIndex = otherId.batchIndex; - this.batchSize = otherId.batchSize; - this.acker = otherId.acker; - } else { - this.batchIndex = NO_BATCH; - this.batchSize = 0; - this.acker = BatchMessageAckerDisabled.INSTANCE; - } + public BatchMessageIdImpl(MessageIdAdv other) { + this(other.getLedgerId(), other.getEntryId(), other.getPartitionIndex(), + other.getBatchIndex(), other.getBatchSize(), other.getAckSet()); } + @Override public int getBatchIndex() { return batchIndex; } - @Override - public int compareTo(@Nonnull MessageId o) { - if (o instanceof MessageIdImpl) { - MessageIdImpl other = (MessageIdImpl) o; - int batchIndex = (o instanceof BatchMessageIdImpl) ? ((BatchMessageIdImpl) o).batchIndex : NO_BATCH; - return messageIdCompare( - this.ledgerId, this.entryId, this.partitionIndex, this.batchIndex, - other.ledgerId, other.entryId, other.partitionIndex, batchIndex - ); - } else if (o instanceof TopicMessageId) { - return compareTo(MessageIdImpl.convertToMessageIdImpl(o)); - } else { - throw new UnsupportedOperationException("Unknown MessageId type: " + o.getClass().getName()); - } - } - @Override public int hashCode() { - return messageIdHashCode(ledgerId, entryId, partitionIndex, batchIndex); + return MessageIdAdvUtils.hashCode(this); } @Override public boolean equals(Object o) { - return super.equals(o); + return MessageIdAdvUtils.equals(this, o); } @Override @@ -106,39 +78,19 @@ public byte[] toByteArray() { return toByteArray(batchIndex, batchSize); } - public boolean ackIndividual() { - return acker.ackIndividual(batchIndex); - } - - public boolean ackCumulative() { - return acker.ackCumulative(batchIndex); - } - - public int getOutstandingAcksInSameBatch() { - return acker.getOutstandingAcks(); - } - + @Override public int getBatchSize() { - return acker.getBatchSize(); - } - - public int getOriginalBatchSize() { - return this.batchSize; + return batchSize; } - public MessageIdImpl prevBatchMessageId() { - return new MessageIdImpl( - ledgerId, entryId - 1, partitionIndex); - } - - // MessageIdImpl is widely used as the key of a hash map, in this case, we should convert the batch message id to - // have the correct hash code. - public MessageIdImpl toMessageIdImpl() { - return new MessageIdImpl(ledgerId, entryId, partitionIndex); + @Override + public BitSet getAckSet() { + return ackSet; } - public BatchMessageAcker getAcker() { - return acker; + static BitSet newAckSet(int batchSize) { + final BitSet ackSet = new BitSet(batchSize); + ackSet.set(0, batchSize); + return ackSet; } - } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ChunkMessageIdImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ChunkMessageIdImpl.java index 28d5047c8ef25e..29ce160442a3b1 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ChunkMessageIdImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ChunkMessageIdImpl.java @@ -21,10 +21,10 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import java.util.Objects; -import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.common.api.proto.MessageIdData; -public class ChunkMessageIdImpl extends MessageIdImpl implements MessageId { +public class ChunkMessageIdImpl extends MessageIdImpl { private final MessageIdImpl firstChunkMsgId; public ChunkMessageIdImpl(MessageIdImpl firstChunkMsgId, MessageIdImpl lastChunkMsgId) { @@ -32,11 +32,12 @@ public ChunkMessageIdImpl(MessageIdImpl firstChunkMsgId, MessageIdImpl lastChunk this.firstChunkMsgId = firstChunkMsgId; } - public MessageIdImpl getFirstChunkMessageId() { + @Override + public MessageIdAdv getFirstChunkMessageId() { return firstChunkMsgId; } - public MessageIdImpl getLastChunkMessageId() { + public MessageIdAdv getLastChunkMessageId() { return this; } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java index 6988adcccf1831..2c39647fa7bd39 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java @@ -48,6 +48,7 @@ import org.apache.pulsar.client.api.ConsumerEventListener; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.client.api.MessageListener; import org.apache.pulsar.client.api.Messages; import org.apache.pulsar.client.api.PulsarClientException; @@ -83,7 +84,7 @@ public abstract class ConsumerBase extends HandlerState implements Consumer> incomingMessages; - protected ConcurrentOpenHashMap unAckedChunkedMessageIdSequenceMap; + protected ConcurrentOpenHashMap unAckedChunkedMessageIdSequenceMap; protected final ConcurrentLinkedQueue>> pendingReceives; protected final int maxReceiverQueueSize; private volatile int currentReceiverQueueSize; @@ -129,7 +130,7 @@ protected ConsumerBase(PulsarClientImpl client, String topic, ConsumerConfigurat // Always use growable queue since items can exceed the advertised size this.incomingMessages = new GrowableArrayBlockingQueue<>(); this.unAckedChunkedMessageIdSequenceMap = - ConcurrentOpenHashMap.newBuilder().build(); + ConcurrentOpenHashMap.newBuilder().build(); this.executorProvider = executorProvider; this.externalPinnedExecutor = executorProvider.getExecutor(); this.internalPinnedExecutor = client.getInternalExecutorService(); @@ -224,14 +225,6 @@ protected void trackUnAckedMsgIfNoListener(MessageId messageId, int redeliveryCo } } - protected MessageId normalizeMessageId(MessageId messageId) { - if (messageId instanceof BatchMessageIdImpl) { - // do not add each item in batch message into tracker - return ((BatchMessageIdImpl) messageId).toMessageIdImpl(); - } - return messageId; - } - protected void reduceCurrentReceiverQueueSize() { if (!conf.isAutoScaledReceiverQueueSizeEnabled()) { return; @@ -1132,7 +1125,7 @@ protected void callMessageListener(Message msg) { ? ((TopicMessageImpl) msg).getMessage() : msg)); MessageId id; if (this instanceof ConsumerImpl) { - id = normalizeMessageId(msg.getMessageId()); + id = MessageIdAdvUtils.discardBatch(msg.getMessageId()); } else { id = msg.getMessageId(); } 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 eecd51c788f72f..523fe3c9d677b4 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 @@ -35,6 +35,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.BitSet; import java.util.Collections; import java.util.HashMap; import java.util.LinkedList; @@ -71,6 +72,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.MessageIdAdv; import org.apache.pulsar.client.api.Messages; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClientException; @@ -153,12 +155,12 @@ public class ConsumerImpl extends ConsumerBase implements ConnectionHandle @Getter(AccessLevel.PACKAGE) private final int priorityLevel; private final SubscriptionMode subscriptionMode; - private volatile BatchMessageIdImpl startMessageId; + private volatile MessageIdAdv startMessageId; - private volatile BatchMessageIdImpl seekMessageId; + private volatile MessageIdAdv seekMessageId; private final AtomicBoolean duringSeek; - private final BatchMessageIdImpl initialStartMessageId; + private final MessageIdAdv initialStartMessageId; private final long startMessageRollbackDurationInSec; @@ -177,7 +179,7 @@ public class ConsumerImpl extends ConsumerBase implements ConnectionHandle private final TopicName topicName; private final String topicNameWithoutPartition; - private final Map>> possibleSendToDeadLetterTopicMessages; + private final Map>> possibleSendToDeadLetterTopicMessages; private final DeadLetterPolicy deadLetterPolicy; @@ -257,12 +259,8 @@ protected ConsumerImpl(PulsarClientImpl client, String topic, ConsumerConfigurat this.consumerId = client.newConsumerId(); this.subscriptionMode = conf.getSubscriptionMode(); if (startMessageId != null) { - if (startMessageId instanceof ChunkMessageIdImpl) { - this.startMessageId = new BatchMessageIdImpl( - ((ChunkMessageIdImpl) startMessageId).getFirstChunkMessageId()); - } else { - this.startMessageId = new BatchMessageIdImpl((MessageIdImpl) startMessageId); - } + MessageIdAdv firstChunkMessageId = ((MessageIdAdv) startMessageId).getFirstChunkMessageId(); + this.startMessageId = (firstChunkMessageId == null) ? (MessageIdAdv) startMessageId : firstChunkMessageId; } this.initialStartMessageId = this.startMessageId; this.startMessageRollbackDurationInSec = startMessageRollbackDurationInSec; @@ -534,7 +532,6 @@ protected CompletableFuture> internalBatchReceiveAsync() { protected CompletableFuture doAcknowledge(MessageId messageId, AckType ackType, Map properties, TransactionImpl txn) { - messageId = MessageIdImpl.convertToMessageIdImpl(messageId); if (getState() != State.Ready && getState() != State.Connecting) { stats.incrementNumAcksFailed(); PulsarClientException exception = new PulsarClientException("Consumer not ready. State: " + getState()); @@ -550,16 +547,12 @@ protected CompletableFuture doAcknowledge(MessageId messageId, AckType ack return doTransactionAcknowledgeForResponse(messageId, ackType, null, properties, new TxnID(txn.getTxnIdMostBits(), txn.getTxnIdLeastBits())); } - return acknowledgmentsGroupingTracker.addAcknowledgment((MessageIdImpl) messageId, ackType, properties); + return acknowledgmentsGroupingTracker.addAcknowledgment(messageId, ackType, properties); } @Override protected CompletableFuture doAcknowledge(List messageIdList, AckType ackType, Map properties, TransactionImpl txn) { - - for (MessageId messageId : messageIdList) { - checkArgument(messageId instanceof MessageIdImpl); - } if (getState() != State.Ready && getState() != State.Connecting) { stats.incrementNumAcksFailed(); PulsarClientException exception = new PulsarClientException("Consumer not ready. State: " + getState()); @@ -571,7 +564,7 @@ protected CompletableFuture doAcknowledge(List messageIdList, A return FutureUtil.failedFuture(exception); } if (txn != null) { - return doTransactionAcknowledgeForResponse(messageIdList, ackType, null, + return doTransactionAcknowledgeForResponse(messageIdList, ackType, properties, new TxnID(txn.getTxnIdMostBits(), txn.getTxnIdLeastBits())); } else { return this.acknowledgmentsGroupingTracker.addListAcknowledgment(messageIdList, ackType, properties); @@ -591,7 +584,6 @@ protected CompletableFuture doReconsumeLater(Message message, AckType a .InvalidMessageException("Cannot handle message with null messageId")); } - messageId = MessageIdImpl.convertToMessageIdImpl(messageId); if (getState() != State.Ready && getState() != State.Connecting) { stats.incrementNumAcksFailed(); PulsarClientException exception = new PulsarClientException("Consumer not ready. State: " + getState()); @@ -916,7 +908,7 @@ protected void consumerIsReconnectedToBroker(ClientCnx cnx, int currentQueueSize * Clear the internal receiver queue and returns the message id of what was the 1st message in the queue that was * not seen by the application. */ - private BatchMessageIdImpl clearReceiverQueue() { + private MessageIdAdv clearReceiverQueue() { List> currentMessageQueue = new ArrayList<>(incomingMessages.size()); incomingMessages.drainTo(currentMessageQueue); resetIncomingMessageSize(); @@ -928,17 +920,16 @@ private BatchMessageIdImpl clearReceiverQueue() { } if (!currentMessageQueue.isEmpty()) { - MessageIdImpl nextMessageInQueue = (MessageIdImpl) currentMessageQueue.get(0).getMessageId(); - BatchMessageIdImpl previousMessage; - if (nextMessageInQueue instanceof BatchMessageIdImpl) { + MessageIdAdv nextMessageInQueue = (MessageIdAdv) currentMessageQueue.get(0).getMessageId(); + MessageIdAdv previousMessage; + if (MessageIdAdvUtils.isBatch(nextMessageInQueue)) { // Get on the previous message within the current batch previousMessage = new BatchMessageIdImpl(nextMessageInQueue.getLedgerId(), nextMessageInQueue.getEntryId(), nextMessageInQueue.getPartitionIndex(), - ((BatchMessageIdImpl) nextMessageInQueue).getBatchIndex() - 1); + nextMessageInQueue.getBatchIndex() - 1); } else { // Get on previous message in previous entry - previousMessage = new BatchMessageIdImpl(nextMessageInQueue.getLedgerId(), - nextMessageInQueue.getEntryId() - 1, nextMessageInQueue.getPartitionIndex(), -1); + previousMessage = MessageIdAdvUtils.prevMessageId(nextMessageInQueue); } // release messages if they are pooled messages currentMessageQueue.forEach(Message::release); @@ -1108,7 +1099,7 @@ protected MessageImpl newSingleMessage(final int index, final Schema schema, final boolean containMetadata, final BitSetRecyclable ackBitSet, - final BatchMessageAcker acker, + final BitSet ackSetInMessageId, final int redeliveryCount, final long consumerEpoch) { if (log.isDebugEnabled()) { @@ -1143,7 +1134,7 @@ protected MessageImpl newSingleMessage(final int index, } BatchMessageIdImpl batchMessageIdImpl = new BatchMessageIdImpl(messageId.getLedgerId(), - messageId.getEntryId(), getPartitionIndex(), index, numMessages, acker); + messageId.getEntryId(), getPartitionIndex(), index, numMessages, ackSetInMessageId); final ByteBuf payloadBuffer = (singleMessagePayload != null) ? singleMessagePayload : payload; final MessageImpl message = MessageImpl.create(topicName.toString(), batchMessageIdImpl, @@ -1507,7 +1498,7 @@ void receiveIndividualMessagesFromBatch(BrokerEntryMetadata brokerEntryMetadata, possibleToDeadLetter = new ArrayList<>(); } - BatchMessageAcker acker = BatchMessageAcker.newAcker(batchSize); + BitSet ackSetInMessageId = BatchMessageIdImpl.newAckSet(batchSize); BitSetRecyclable ackBitSet = null; if (ackSet != null && ackSet.size() > 0) { ackBitSet = BitSetRecyclable.valueOf(SafeCollectionUtils.longListToArray(ackSet)); @@ -1519,7 +1510,7 @@ void receiveIndividualMessagesFromBatch(BrokerEntryMetadata brokerEntryMetadata, for (int i = 0; i < batchSize; ++i) { final MessageImpl message = newSingleMessage(i, batchSize, brokerEntryMetadata, msgMetadata, singleMessageMetadata, uncompressedPayload, batchMessage, schema, true, - ackBitSet, acker, redeliveryCount, consumerEpoch); + ackBitSet, ackSetInMessageId, redeliveryCount, consumerEpoch); if (message == null) { skippedMessages++; continue; @@ -1616,7 +1607,7 @@ protected void trackMessage(MessageId messageId) { protected void trackMessage(MessageId messageId, int redeliveryCount) { if (conf.getAckTimeoutMillis() > 0 && messageId instanceof MessageIdImpl) { - MessageId id = normalizeMessageId(messageId); + MessageId id = MessageIdAdvUtils.discardBatch(messageId); if (hasParentConsumer) { //TODO: check parent consumer here // we should no longer track this message, TopicsConsumer will take care from now onwards @@ -1913,8 +1904,6 @@ public void redeliverUnacknowledgedMessages(Set messageIds) { return; } - checkArgument(messageIds.stream().findFirst().get() instanceof MessageIdImpl); - if (conf.getSubscriptionType() != SubscriptionType.Shared && conf.getSubscriptionType() != SubscriptionType.Key_Shared) { // We cannot redeliver single messages if subscription type is not Shared @@ -1924,11 +1913,7 @@ public void redeliverUnacknowledgedMessages(Set messageIds) { ClientCnx cnx = cnx(); if (isConnected() && cnx.getRemoteEndpointProtocolVersion() >= ProtocolVersion.v2.getValue()) { int messagesFromQueue = removeExpiredMessagesFromQueue(messageIds); - Iterable> batches = Iterables.partition( - messageIds.stream() - .map(messageId -> (MessageIdImpl) messageId) - .collect(Collectors.toSet()), MAX_REDELIVER_UNACKNOWLEDGED); - batches.forEach(ids -> { + Iterables.partition(messageIds, MAX_REDELIVER_UNACKNOWLEDGED).forEach(ids -> { getRedeliveryMessageIdData(ids).thenAccept(messageIdData -> { if (!messageIdData.isEmpty()) { ByteBuf cmd = Commands.newRedeliverUnacknowledgedMessages(consumerId, messageIdData); @@ -1967,11 +1952,12 @@ protected void completeOpBatchReceive(OpBatchReceive op) { notifyPendingBatchReceivedCallBack(op.future); } - private CompletableFuture> getRedeliveryMessageIdData(List messageIds) { + private CompletableFuture> getRedeliveryMessageIdData(List messageIds) { if (messageIds == null || messageIds.isEmpty()) { return CompletableFuture.completedFuture(Collections.emptyList()); } - List> futures = messageIds.stream().map(messageId -> { + List> futures = messageIds.stream().map(originalMessageId -> { + final MessageIdAdv messageId = (MessageIdAdv) originalMessageId; CompletableFuture future = processPossibleToDLQ(messageId); return future.thenApply(sendToDLQ -> { if (!sendToDLQ) { @@ -1987,20 +1973,15 @@ private CompletableFuture> getRedeliveryMessageIdData(List processPossibleToDLQ(MessageIdImpl messageId) { + private CompletableFuture processPossibleToDLQ(MessageIdAdv messageId) { List> deadLetterMessages = null; if (possibleSendToDeadLetterTopicMessages != null) { - if (messageId instanceof BatchMessageIdImpl) { - messageId = new MessageIdImpl(messageId.getLedgerId(), messageId.getEntryId(), - getPartitionIndex()); - } - deadLetterMessages = possibleSendToDeadLetterTopicMessages.get(messageId); + deadLetterMessages = possibleSendToDeadLetterTopicMessages.get(MessageIdAdvUtils.discardBatch(messageId)); } CompletableFuture result = new CompletableFuture<>(); if (deadLetterMessages != null) { initDeadLetterProducerIfNeeded(); List> finalDeadLetterMessages = deadLetterMessages; - MessageIdImpl finalMessageId = messageId; deadLetterProducer.thenAcceptAsync(producerDLQ -> { for (MessageImpl message : finalDeadLetterMessages) { String originMessageIdStr = message.getMessageId().toString(); @@ -2014,12 +1995,12 @@ private CompletableFuture processPossibleToDLQ(MessageIdImpl messageId) } typedMessageBuilderNew.sendAsync() .thenAccept(messageIdInDLQ -> { - possibleSendToDeadLetterTopicMessages.remove(finalMessageId); - acknowledgeAsync(finalMessageId).whenComplete((v, ex) -> { + possibleSendToDeadLetterTopicMessages.remove(messageId); + acknowledgeAsync(messageId).whenComplete((v, ex) -> { if (ex != null) { log.warn("[{}] [{}] [{}] Failed to acknowledge the message {} of the original" + " topic but send to the DLQ successfully.", - topicName, subscription, consumerName, finalMessageId, ex); + topicName, subscription, consumerName, messageId, ex); result.complete(false); } else { result.complete(true); @@ -2028,10 +2009,10 @@ private CompletableFuture processPossibleToDLQ(MessageIdImpl messageId) }).exceptionally(ex -> { if (ex instanceof PulsarClientException.ProducerQueueIsFullError) { log.warn("[{}] [{}] [{}] Failed to send DLQ message to {} for message id {}: {}", - topicName, subscription, consumerName, finalMessageId, ex.getMessage()); + topicName, subscription, consumerName, messageId, ex.getMessage()); } else { log.warn("[{}] [{}] [{}] Failed to send DLQ message to {} for message id {}", - topicName, subscription, consumerName, finalMessageId, ex); + topicName, subscription, consumerName, messageId, ex); } result.complete(false); return null; @@ -2134,8 +2115,8 @@ private CompletableFuture seekAsyncInternal(long requestId, ByteBuf seek, final CompletableFuture seekFuture = new CompletableFuture<>(); ClientCnx cnx = cnx(); - BatchMessageIdImpl originSeekMessageId = seekMessageId; - seekMessageId = new BatchMessageIdImpl((MessageIdImpl) seekId); + MessageIdAdv originSeekMessageId = seekMessageId; + seekMessageId = (MessageIdAdv) seekId; duringSeek.set(true); log.info("[{}][{}] Seeking subscription to {}", topic, subscription, seekBy); @@ -2173,29 +2154,28 @@ public CompletableFuture seekAsync(long timestamp) { } @Override - public CompletableFuture seekAsync(MessageId originalMessageId) { - final MessageIdImpl messageId = MessageIdImpl.convertToMessageIdImpl(originalMessageId); + public CompletableFuture seekAsync(MessageId messageId) { String seekBy = String.format("the message %s", messageId.toString()); return seekAsyncCheckState(seekBy).orElseGet(() -> { long requestId = client.newRequestId(); - ByteBuf seek = null; - if (messageId instanceof BatchMessageIdImpl) { - BatchMessageIdImpl msgId = (BatchMessageIdImpl) messageId; - // Initialize ack set - BitSetRecyclable ackSet = BitSetRecyclable.create(); - ackSet.set(0, msgId.getBatchSize()); - ackSet.clear(0, Math.max(msgId.getBatchIndex(), 0)); - long[] ackSetArr = ackSet.toLongArray(); - ackSet.recycle(); - - seek = Commands.newSeek(consumerId, requestId, msgId.getLedgerId(), msgId.getEntryId(), ackSetArr); - } else if (messageId instanceof ChunkMessageIdImpl) { - ChunkMessageIdImpl msgId = (ChunkMessageIdImpl) messageId; - seek = Commands.newSeek(consumerId, requestId, msgId.getFirstChunkMessageId().getLedgerId(), - msgId.getFirstChunkMessageId().getEntryId(), new long[0]); + final MessageIdAdv msgId = (MessageIdAdv) messageId; + final MessageIdAdv firstChunkMsgId = msgId.getFirstChunkMessageId(); + final ByteBuf seek; + if (msgId.getFirstChunkMessageId() != null) { + seek = Commands.newSeek(consumerId, requestId, firstChunkMsgId.getLedgerId(), + firstChunkMsgId.getEntryId(), new long[0]); } else { - seek = Commands.newSeek( - consumerId, requestId, messageId.getLedgerId(), messageId.getEntryId(), new long[0]); + final long[] ackSetArr; + if (MessageIdAdvUtils.isBatch(msgId)) { + final BitSetRecyclable ackSet = BitSetRecyclable.create(); + ackSet.set(0, msgId.getBatchSize()); + ackSet.clear(0, Math.max(msgId.getBatchIndex(), 0)); + ackSetArr = ackSet.toLongArray(); + ackSet.recycle(); + } else { + ackSetArr = new long[0]; + } + seek = Commands.newSeek(consumerId, requestId, msgId.getLedgerId(), msgId.getEntryId(), ackSetArr); } return seekAsyncInternal(requestId, seek, messageId, seekBy); }); @@ -2227,9 +2207,8 @@ public CompletableFuture hasMessageAvailableAsync() { } future.thenAccept(response -> { - MessageIdImpl lastMessageId = MessageIdImpl.convertToMessageIdImpl(response.lastMessageId); - MessageIdImpl markDeletePosition = MessageIdImpl - .convertToMessageIdImpl(response.markDeletePosition); + MessageIdAdv lastMessageId = (MessageIdAdv) response.lastMessageId; + MessageIdAdv markDeletePosition = (MessageIdAdv) response.markDeletePosition; if (markDeletePosition != null && !(markDeletePosition.getEntryId() < 0 && markDeletePosition.getLedgerId() > lastMessageId.getLedgerId())) { @@ -2414,16 +2393,6 @@ private void internalGetLastMessageIdAsync(final Backoff backoff, } } - private MessageIdImpl getMessageIdImpl(Message msg) { - MessageIdImpl messageId = (MessageIdImpl) msg.getMessageId(); - if (messageId instanceof BatchMessageIdImpl) { - // messageIds contain MessageIdImpl, not BatchMessageIdImpl - messageId = new MessageIdImpl(messageId.getLedgerId(), messageId.getEntryId(), getPartitionIndex()); - } - return messageId; - } - - private boolean isMessageUndecryptable(MessageMetadata msgMetadata) { return (msgMetadata.getEncryptionKeysCount() > 0 && conf.getCryptoKeyReader() == null && conf.getCryptoFailureAction() == ConsumerCryptoFailureAction.CONSUME); @@ -2466,7 +2435,7 @@ private int removeExpiredMessagesFromQueue(Set messageIds) { int messagesFromQueue = 0; Message peek = incomingMessages.peek(); if (peek != null) { - MessageIdImpl messageId = getMessageIdImpl(peek); + MessageIdAdv messageId = MessageIdAdvUtils.discardBatch(peek.getMessageId()); if (!messageIds.contains(messageId)) { // first message is not expired, then no message is expired in queue. return 0; @@ -2477,7 +2446,7 @@ private int removeExpiredMessagesFromQueue(Set messageIds) { while (message != null) { decreaseIncomingMessageSize(message); messagesFromQueue++; - MessageIdImpl id = getMessageIdImpl(message); + MessageIdAdv id = MessageIdAdvUtils.discardBatch(message.getMessageId()); if (!messageIds.contains(id)) { messageIds.add(id); break; @@ -2671,33 +2640,26 @@ private void removeChunkMessage(String msgUUID, ChunkedMessageCtx chunkedMsgCtx, private CompletableFuture doTransactionAcknowledgeForResponse(MessageId messageId, AckType ackType, ValidationError validationError, Map properties, TxnID txnID) { - BitSetRecyclable bitSetRecyclable = null; - long ledgerId; - long entryId; - ByteBuf cmd; long requestId = client.newRequestId(); - if (messageId instanceof BatchMessageIdImpl) { - BatchMessageIdImpl batchMessageId = (BatchMessageIdImpl) messageId; - bitSetRecyclable = BitSetRecyclable.create(); - ledgerId = batchMessageId.getLedgerId(); - entryId = batchMessageId.getEntryId(); + final MessageIdAdv messageIdAdv = (MessageIdAdv) messageId; + final long ledgerId = messageIdAdv.getLedgerId(); + final long entryId = messageIdAdv.getEntryId(); + final ByteBuf cmd; + if (MessageIdAdvUtils.isBatch(messageIdAdv)) { + BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create(); + bitSetRecyclable.set(0, messageIdAdv.getBatchSize()); if (ackType == AckType.Cumulative) { - batchMessageId.ackCumulative(); - bitSetRecyclable.set(0, batchMessageId.getBatchSize()); - bitSetRecyclable.clear(0, batchMessageId.getBatchIndex() + 1); + MessageIdAdvUtils.acknowledge(messageIdAdv, false); + bitSetRecyclable.clear(0, messageIdAdv.getBatchIndex() + 1); } else { - bitSetRecyclable.set(0, batchMessageId.getBatchSize()); - bitSetRecyclable.clear(batchMessageId.getBatchIndex()); + bitSetRecyclable.clear(messageIdAdv.getBatchIndex()); } cmd = Commands.newAck(consumerId, ledgerId, entryId, bitSetRecyclable, ackType, validationError, properties, - txnID.getLeastSigBits(), txnID.getMostSigBits(), requestId, batchMessageId.getBatchSize()); + txnID.getLeastSigBits(), txnID.getMostSigBits(), requestId, messageIdAdv.getBatchSize()); bitSetRecyclable.recycle(); } else { - MessageIdImpl singleMessage = (MessageIdImpl) messageId; - ledgerId = singleMessage.getLedgerId(); - entryId = singleMessage.getEntryId(); - cmd = Commands.newAck(consumerId, ledgerId, entryId, bitSetRecyclable, ackType, - validationError, properties, txnID.getLeastSigBits(), txnID.getMostSigBits(), requestId); + cmd = Commands.newAck(consumerId, ledgerId, entryId, null, ackType, validationError, properties, + txnID.getLeastSigBits(), txnID.getMostSigBits(), requestId); } if (ackType == AckType.Cumulative) { @@ -2716,58 +2678,42 @@ private CompletableFuture doTransactionAcknowledgeForResponse(MessageId me } private CompletableFuture doTransactionAcknowledgeForResponse(List messageIds, AckType ackType, - ValidationError validationError, Map properties, TxnID txnID) { - BitSetRecyclable bitSetRecyclable = null; - long ledgerId; - long entryId; - ByteBuf cmd; long requestId = client.newRequestId(); List messageIdDataList = new LinkedList<>(); for (MessageId messageId : messageIds) { - if (messageId instanceof BatchMessageIdImpl) { - BatchMessageIdImpl batchMessageId = (BatchMessageIdImpl) messageId; - bitSetRecyclable = BitSetRecyclable.create(); + final MessageIdAdv messageIdAdv = (MessageIdAdv) messageId; + final MessageIdData messageIdData = new MessageIdData(); + messageIdData.setLedgerId(messageIdAdv.getLedgerId()); + messageIdData.setEntryId(messageIdAdv.getEntryId()); + if (MessageIdAdvUtils.isBatch(messageIdAdv)) { + final BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create(); + bitSetRecyclable.set(0, messageIdAdv.getBatchSize()); if (ackType == AckType.Cumulative) { - batchMessageId.ackCumulative(); - bitSetRecyclable.set(0, batchMessageId.getBatchSize()); - bitSetRecyclable.clear(0, batchMessageId.getBatchIndex() + 1); + MessageIdAdvUtils.acknowledge(messageIdAdv, false); + bitSetRecyclable.clear(0, messageIdAdv.getBatchIndex() + 1); } else { - bitSetRecyclable.set(0, batchMessageId.getBatchSize()); - bitSetRecyclable.clear(batchMessageId.getBatchIndex()); + bitSetRecyclable.clear(messageIdAdv.getBatchIndex()); } - MessageIdData messageIdData = new MessageIdData(); - messageIdData.setLedgerId(batchMessageId.getLedgerId()); - messageIdData.setEntryId(batchMessageId.getEntryId()); - messageIdData.setBatchSize(batchMessageId.getBatchSize()); - long[] as = bitSetRecyclable.toLongArray(); - for (int i = 0; i < as.length; i++) { - messageIdData.addAckSet(as[i]); + for (long x : bitSetRecyclable.toLongArray()) { + messageIdData.addAckSet(x); } bitSetRecyclable.recycle(); - messageIdDataList.add(messageIdData); - } else { - MessageIdImpl singleMessage = (MessageIdImpl) messageId; - ledgerId = singleMessage.getLedgerId(); - entryId = singleMessage.getEntryId(); - MessageIdData messageIdData = new MessageIdData(); - messageIdData.setLedgerId(ledgerId); - messageIdData.setEntryId(entryId); - messageIdDataList.add(messageIdData); } + messageIdDataList.add(messageIdData); if (ackType == AckType.Cumulative) { unAckedMessageTracker.removeMessagesTill(messageId); } else { unAckedMessageTracker.remove(messageId); } } - cmd = Commands.newAck(consumerId, messageIdDataList, ackType, validationError, properties, + final ByteBuf cmd = Commands.newAck(consumerId, messageIdDataList, ackType, null, properties, txnID.getLeastSigBits(), txnID.getMostSigBits(), requestId); return cnx().newAckForReceipt(cmd, requestId); } - public Map>> getPossibleSendToDeadLetterTopicMessages() { + public Map>> getPossibleSendToDeadLetterTopicMessages() { return possibleSendToDeadLetterTopicMessages; } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageIdAdvUtils.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageIdAdvUtils.java new file mode 100644 index 00000000000000..c8b18524ec0520 --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageIdAdvUtils.java @@ -0,0 +1,74 @@ +/* + * 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.BitSet; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.MessageIdAdv; + +public class MessageIdAdvUtils { + + static int hashCode(MessageIdAdv msgId) { + return (int) (31 * (msgId.getLedgerId() + 31 * msgId.getEntryId()) + + (31 * (long) msgId.getPartitionIndex()) + msgId.getBatchIndex()); + } + + static boolean equals(MessageIdAdv lhs, Object o) { + if (!(o instanceof MessageIdAdv)) { + return false; + } + final MessageIdAdv rhs = (MessageIdAdv) o; + return lhs.getLedgerId() == rhs.getLedgerId() + && lhs.getEntryId() == rhs.getEntryId() + && lhs.getPartitionIndex() == rhs.getPartitionIndex() + && lhs.getBatchIndex() == rhs.getBatchIndex(); + } + + static boolean acknowledge(MessageIdAdv msgId, boolean individual) { + if (!isBatch(msgId)) { + return true; + } + final BitSet ackSet = msgId.getAckSet(); + if (ackSet == null) { + // The internal MessageId implementation should never reach here. If users have implemented their own + // MessageId and getAckSet() is not override, return false to avoid acknowledge current entry. + return false; + } + int batchIndex = msgId.getBatchIndex(); + if (individual) { + ackSet.clear(batchIndex); + } else { + ackSet.clear(0, batchIndex + 1); + } + return ackSet.isEmpty(); + } + + static boolean isBatch(MessageIdAdv msgId) { + return msgId.getBatchIndex() >= 0 && msgId.getBatchSize() > 0; + } + + static MessageIdAdv discardBatch(MessageId messageId) { + MessageIdAdv msgId = (MessageIdAdv) messageId; + return new MessageIdImpl(msgId.getLedgerId(), msgId.getEntryId(), msgId.getPartitionIndex()); + } + + static MessageIdAdv prevMessageId(MessageIdAdv msgId) { + return new MessageIdImpl(msgId.getLedgerId(), msgId.getEntryId() - 1, msgId.getPartitionIndex()); + } +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageIdImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageIdImpl.java index 1a0f491a6a7bba..83ee762578390b 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageIdImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageIdImpl.java @@ -18,21 +18,17 @@ */ package org.apache.pulsar.client.impl; -import static org.apache.pulsar.client.impl.BatchMessageIdImpl.NO_BATCH; -import com.google.common.collect.ComparisonChain; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import io.netty.util.concurrent.FastThreadLocal; import java.io.IOException; import java.util.Objects; -import javax.annotation.Nonnull; import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.TopicMessageId; +import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.common.api.proto.MessageIdData; -import org.apache.pulsar.common.classification.InterfaceStability; import org.apache.pulsar.common.naming.TopicName; -public class MessageIdImpl implements MessageId { +public class MessageIdImpl implements MessageIdAdv { protected final long ledgerId; protected final long entryId; protected final int partitionIndex; @@ -49,28 +45,29 @@ public MessageIdImpl(long ledgerId, long entryId, int partitionIndex) { this.partitionIndex = partitionIndex; } + @Override public long getLedgerId() { return ledgerId; } + @Override public long getEntryId() { return entryId; } + @Override public int getPartitionIndex() { return partitionIndex; } @Override public int hashCode() { - return messageIdHashCode(ledgerId, entryId, partitionIndex, NO_BATCH); + return MessageIdAdvUtils.hashCode(this); } @Override public boolean equals(Object o) { - return (o instanceof MessageId) - && !(o instanceof MultiMessageIdImpl) - && (compareTo((MessageId) o) == 0); + return MessageIdAdvUtils.equals(this, o); } @Override @@ -100,7 +97,7 @@ public static MessageId fromByteArray(byte[] data) throws IOException { if (idData.hasBatchIndex()) { if (idData.hasBatchSize()) { messageId = new BatchMessageIdImpl(idData.getLedgerId(), idData.getEntryId(), idData.getPartition(), - idData.getBatchIndex(), idData.getBatchSize(), BatchMessageAcker.newAcker(idData.getBatchSize())); + idData.getBatchIndex(), idData.getBatchSize(), BatchMessageIdImpl.newAckSet(idData.getBatchSize())); } else { messageId = new BatchMessageIdImpl(idData.getLedgerId(), idData.getEntryId(), idData.getPartition(), idData.getBatchIndex()); @@ -118,22 +115,6 @@ public static MessageId fromByteArray(byte[] data) throws IOException { return messageId; } - @InterfaceStability.Unstable - public static MessageIdImpl convertToMessageIdImpl(MessageId messageId) { - if (messageId instanceof TopicMessageId) { - if (messageId instanceof TopicMessageIdImpl) { - return (MessageIdImpl) ((TopicMessageIdImpl) messageId).getInnerMessageId(); - } else { - try { - return (MessageIdImpl) MessageId.fromByteArray(messageId.toByteArray()); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - } - return (MessageIdImpl) messageId; - } - public static MessageId fromByteArrayWithTopic(byte[] data, String topicName) throws IOException { return fromByteArrayWithTopic(data, TopicName.get(topicName)); } @@ -152,10 +133,10 @@ public static MessageId fromByteArrayWithTopic(byte[] data, TopicName topicName) if (idData.hasBatchSize()) { messageId = new BatchMessageIdImpl(idData.getLedgerId(), idData.getEntryId(), idData.getPartition(), idData.getBatchIndex(), idData.getBatchSize(), - BatchMessageAcker.newAcker(idData.getBatchSize())); + BatchMessageIdImpl.newAckSet(idData.getBatchSize())); } else { messageId = new BatchMessageIdImpl(idData.getLedgerId(), idData.getEntryId(), idData.getPartition(), - idData.getBatchIndex(), 0, BatchMessageAckerDisabled.INSTANCE); + idData.getBatchIndex(), 0, null); } } else { messageId = new MessageIdImpl(idData.getLedgerId(), idData.getEntryId(), idData.getPartition()); @@ -207,36 +188,4 @@ public byte[] toByteArray() { // there is no message batch so we pass -1 return toByteArray(-1, 0); } - - @Override - public int compareTo(@Nonnull MessageId o) { - if (o instanceof MessageIdImpl) { - MessageIdImpl other = (MessageIdImpl) o; - int batchIndex = (o instanceof BatchMessageIdImpl) ? ((BatchMessageIdImpl) o).getBatchIndex() : NO_BATCH; - return messageIdCompare( - this.ledgerId, this.entryId, this.partitionIndex, NO_BATCH, - other.ledgerId, other.entryId, other.partitionIndex, batchIndex - ); - } else if (o instanceof TopicMessageId) { - return compareTo(convertToMessageIdImpl(o)); - } else { - throw new UnsupportedOperationException("Unknown MessageId type: " + o.getClass().getName()); - } - } - - static int messageIdHashCode(long ledgerId, long entryId, int partitionIndex, int batchIndex) { - return (int) (31 * (ledgerId + 31 * entryId) + (31 * (long) partitionIndex) + batchIndex); - } - - static int messageIdCompare( - long ledgerId1, long entryId1, int partitionIndex1, int batchIndex1, - long ledgerId2, long entryId2, int partitionIndex2, int batchIndex2 - ) { - return ComparisonChain.start() - .compare(ledgerId1, ledgerId2) - .compare(entryId1, entryId2) - .compare(partitionIndex1, partitionIndex2) - .compare(batchIndex1, batchIndex2) - .result(); - } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessageImpl.java index 0b6fb608ee62b6..d369d639a73a0f 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 @@ -40,6 +40,7 @@ import lombok.Getter; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SchemaSerializationException; import org.apache.pulsar.client.impl.schema.AbstractSchema; @@ -714,9 +715,10 @@ public boolean hasIndex() { @Override public Optional getIndex() { if (brokerEntryMetadata != null && brokerEntryMetadata.hasIndex()) { - if (msgMetadata.hasNumMessagesInBatch() && messageId instanceof BatchMessageIdImpl) { - int batchSize = ((BatchMessageIdImpl) messageId).getBatchSize(); - int batchIndex = ((BatchMessageIdImpl) messageId).getBatchIndex(); + MessageIdAdv messageIdAdv = (MessageIdAdv) messageId; + if (msgMetadata.hasNumMessagesInBatch() && MessageIdAdvUtils.isBatch(messageIdAdv)) { + int batchSize = messageIdAdv.getBatchSize(); + int batchIndex = messageIdAdv.getBatchIndex(); return Optional.of(brokerEntryMetadata.getIndex() - batchSize + batchIndex + 1); } return Optional.of(brokerEntryMetadata.getIndex()); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadContextImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadContextImpl.java index dcae86bd01a3b2..f4c9aa27074773 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadContextImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MessagePayloadContextImpl.java @@ -21,6 +21,7 @@ import static org.apache.pulsar.common.protocol.Commands.DEFAULT_CONSUMER_EPOCH; import io.netty.buffer.ByteBuf; import io.netty.util.Recycler; +import java.util.BitSet; import java.util.List; import lombok.NonNull; import org.apache.pulsar.client.api.Message; @@ -50,7 +51,7 @@ protected MessagePayloadContextImpl newObject(Handle private MessageIdImpl messageId; private ConsumerImpl consumer; private int redeliveryCount; - private BatchMessageAcker acker; + private BitSet ackSetInMessageId; private BitSetRecyclable ackBitSet; private long consumerEpoch; @@ -73,7 +74,7 @@ public static MessagePayloadContextImpl get(final BrokerEntryMetadata brokerEntr context.messageId = messageId; context.consumer = consumer; context.redeliveryCount = redeliveryCount; - context.acker = BatchMessageAcker.newAcker(context.getNumMessages()); + context.ackSetInMessageId = BatchMessageIdImpl.newAckSet(context.getNumMessages()); context.ackBitSet = (ackSet != null && ackSet.size() > 0) ? BitSetRecyclable.valueOf(SafeCollectionUtils.longListToArray(ackSet)) : null; @@ -88,7 +89,7 @@ public void recycle() { consumer = null; redeliveryCount = 0; consumerEpoch = DEFAULT_CONSUMER_EPOCH; - acker = null; + ackSetInMessageId = null; if (ackBitSet != null) { ackBitSet.recycle(); ackBitSet = null; @@ -134,7 +135,7 @@ public Message getMessageAt(int index, schema, containMetadata, ackBitSet, - acker, + ackSetInMessageId, redeliveryCount, consumerEpoch); } finally { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java index 341a91e97348e6..861a7e3cd7ed5f 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java @@ -54,6 +54,7 @@ import org.apache.pulsar.client.api.ConsumerStats; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.client.api.Messages; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.PulsarClientException.NotSupportedException; @@ -99,7 +100,7 @@ public class MultiTopicsConsumerImpl extends ConsumerBase { private final MultiTopicConsumerStatsRecorderImpl stats; private final ConsumerConfigurationData internalConfig; - private volatile BatchMessageIdImpl startMessageId = null; + private volatile MessageIdAdv startMessageId; private final long startMessageRollbackDurationInSec; MultiTopicsConsumerImpl(PulsarClientImpl client, ConsumerConfigurationData conf, ExecutorProvider executorProvider, CompletableFuture> subscribeFuture, Schema schema, @@ -138,9 +139,7 @@ public class MultiTopicsConsumerImpl extends ConsumerBase { this.consumers = new ConcurrentHashMap<>(); this.pausedConsumers = new ConcurrentLinkedQueue<>(); this.allTopicPartitionsNumber = new AtomicInteger(0); - this.startMessageId = startMessageId != null - ? new BatchMessageIdImpl(MessageIdImpl.convertToMessageIdImpl(startMessageId)) - : null; + this.startMessageId = (MessageIdAdv) startMessageId; this.startMessageRollbackDurationInSec = startMessageRollbackDurationInSec; this.paused = conf.isStartPaused(); @@ -453,18 +452,15 @@ protected CompletableFuture doAcknowledge(MessageId messageId, AckType ack return FutureUtil.failedFuture(new PulsarClientException("Consumer already closed")); } - TopicMessageId topicMessageId = (TopicMessageId) messageId; - ConsumerImpl consumer = consumers.get(topicMessageId.getOwnerTopic()); + ConsumerImpl consumer = consumers.get(((TopicMessageId) messageId).getOwnerTopic()); if (consumer == null) { return FutureUtil.failedFuture(new PulsarClientException.NotConnectedException()); } - MessageId innerMessageId = MessageIdImpl.convertToMessageIdImpl(topicMessageId); if (ackType == AckType.Cumulative) { - return consumer.acknowledgeCumulativeAsync(innerMessageId); + return consumer.acknowledgeCumulativeAsync(messageId); } else { - return consumer.doAcknowledgeWithTxn(innerMessageId, ackType, properties, txnImpl) - .thenRun(() -> - unAckedMessageTracker.remove(topicMessageId)); + return consumer.doAcknowledgeWithTxn(messageId, ackType, properties, txnImpl) + .thenRun(() -> unAckedMessageTracker.remove(messageId)); } } @@ -489,10 +485,9 @@ protected CompletableFuture doAcknowledge(List messageIdList, } Map> topicToMessageIdMap = new HashMap<>(); for (MessageId messageId : messageIdList) { - TopicMessageId topicMessageId = (TopicMessageId) messageId; - topicToMessageIdMap.putIfAbsent(topicMessageId.getOwnerTopic(), new ArrayList<>()); - topicToMessageIdMap.get(topicMessageId.getOwnerTopic()) - .add(MessageIdImpl.convertToMessageIdImpl(topicMessageId)); + String ownerTopic = ((TopicMessageId) messageId).getOwnerTopic(); + topicToMessageIdMap.putIfAbsent(ownerTopic, new ArrayList<>()); + topicToMessageIdMap.get(ownerTopic).add(messageId); } topicToMessageIdMap.forEach((topicPartitionName, messageIds) -> { ConsumerImpl consumer = consumers.get(topicPartitionName); @@ -540,10 +535,8 @@ protected CompletableFuture doReconsumeLater(Message message, AckType a @Override public void negativeAcknowledge(MessageId messageId) { checkArgument(messageId instanceof TopicMessageId); - TopicMessageId topicMessageId = (TopicMessageId) messageId; - - ConsumerImpl consumer = consumers.get(topicMessageId.getOwnerTopic()); - consumer.negativeAcknowledge(MessageIdImpl.convertToMessageIdImpl(topicMessageId)); + ConsumerImpl consumer = consumers.get(((TopicMessageId) messageId).getOwnerTopic()); + consumer.negativeAcknowledge(messageId); } @Override @@ -696,12 +689,11 @@ public void redeliverUnacknowledgedMessages(Set messageIds) { return; } removeExpiredMessagesFromQueue(messageIds); - messageIds.stream().map(messageId -> (TopicMessageId) messageId) - .collect(Collectors.groupingBy(TopicMessageId::getOwnerTopic, Collectors.toSet())) - .forEach((topicName, messageIds1) -> - consumers.get(topicName) - .redeliverUnacknowledgedMessages(messageIds1.stream() - .map(MessageIdImpl::convertToMessageIdImpl).collect(Collectors.toSet()))); + messageIds.stream() + .collect(Collectors.groupingBy( + msgId -> ((TopicMessageIdImpl) msgId).getOwnerTopic(), Collectors.toSet())) + .forEach((topicName, messageIds1) -> + consumers.get(topicName).redeliverUnacknowledgedMessages(messageIds1)); resumeReceivingFromPausedConsumersIfNeeded(); } @@ -1499,7 +1491,7 @@ public CompletableFuture getLastMessageIdAsync() { public static boolean isIllegalMultiTopicsMessageId(MessageId messageId) { //only support earliest/latest - return !MessageId.earliest.equals(messageId) && !MessageId.latest.equals(messageId); + return !messageId.equals(MessageId.earliest) && !messageId.equals(MessageId.latest); } public void tryAcknowledgeMessage(Message msg) { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java index 70d57db3bb6910..37f58a02180912 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NegativeAcksTracker.java @@ -95,14 +95,6 @@ public synchronized void add(Message message) { } private synchronized void add(MessageId messageId, int redeliveryCount) { - messageId = MessageIdImpl.convertToMessageIdImpl(messageId); - - if (messageId instanceof BatchMessageIdImpl) { - BatchMessageIdImpl batchMessageId = (BatchMessageIdImpl) messageId; - messageId = new MessageIdImpl(batchMessageId.getLedgerId(), batchMessageId.getEntryId(), - batchMessageId.getPartitionIndex()); - } - if (nackedMessages == null) { nackedMessages = new HashMap<>(); } @@ -113,7 +105,7 @@ private synchronized void add(MessageId messageId, int redeliveryCount) { } else { backoffNs = nackDelayNanos; } - nackedMessages.put(messageId, System.nanoTime() + backoffNs); + nackedMessages.put(MessageIdAdvUtils.discardBatch(messageId), System.nanoTime() + backoffNs); if (this.timeout == null) { // Schedule a task and group all the redeliveries for same period. Leave a small buffer to allow for diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NonPersistentAcknowledgmentGroupingTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NonPersistentAcknowledgmentGroupingTracker.java index 32f8fb922304bf..e8951cd3d1692f 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NonPersistentAcknowledgmentGroupingTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NonPersistentAcknowledgmentGroupingTracker.java @@ -43,7 +43,7 @@ public boolean isDuplicate(MessageId messageId) { return false; } - public CompletableFuture addAcknowledgment(MessageIdImpl msgId, AckType ackType, Map addAcknowledgment(MessageId msgId, AckType ackType, Map properties) { // no-op return CompletableFuture.completedFuture(null); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java index fef0bcb8906f1e..9086ccc4ef0e0c 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java @@ -23,6 +23,7 @@ import io.netty.channel.EventLoopGroup; import io.netty.util.concurrent.FastThreadLocal; import java.util.ArrayList; +import java.util.BitSet; import java.util.Collections; import java.util.HashSet; import java.util.Iterator; @@ -43,6 +44,7 @@ import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.tuple.Triple; import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; import org.apache.pulsar.client.util.TimedCompletableFuture; @@ -79,8 +81,8 @@ public class PersistentAcknowledgmentsGroupingTracker implements Acknowledgments * This is a set of all the individual acks that the application has issued and that were not already sent to * broker. */ - private final ConcurrentSkipListSet pendingIndividualAcks; - private final ConcurrentHashMap pendingIndividualBatchIndexAcks; + private final ConcurrentSkipListSet pendingIndividualAcks; + private final ConcurrentHashMap pendingIndividualBatchIndexAcks; private final ScheduledFuture scheduledTask; private final boolean batchIndexAckEnabled; @@ -113,18 +115,16 @@ public PersistentAcknowledgmentsGroupingTracker(ConsumerImpl consumer, Consum */ @Override public boolean isDuplicate(MessageId messageId) { - if (!(messageId instanceof MessageIdImpl)) { + if (!(messageId instanceof MessageIdAdv)) { throw new IllegalArgumentException("isDuplicated cannot accept " + messageId.getClass().getName() + ": " + messageId); } - if (lastCumulativeAck.compareTo(messageId) >= 0) { + final MessageIdAdv messageIdAdv = (MessageIdAdv) messageId; + if (lastCumulativeAck.compareTo(messageIdAdv) >= 0) { // Already included in a cumulative ack return true; } else { - final MessageIdImpl messageIdImpl = (messageId instanceof BatchMessageIdImpl) - ? ((BatchMessageIdImpl) messageId).toMessageIdImpl() - : (MessageIdImpl) messageId; - return pendingIndividualAcks.contains(messageIdImpl); + return pendingIndividualAcks.contains(MessageIdAdvUtils.discardBatch(messageIdAdv)); } } @@ -135,10 +135,10 @@ public CompletableFuture addListAcknowledgment(List messageIds, if (consumer.isAckReceiptEnabled()) { Set> completableFutureSet = new HashSet<>(); messageIds.forEach(messageId -> - completableFutureSet.add(addAcknowledgment((MessageIdImpl) messageId, ackType, properties))); + completableFutureSet.add(addAcknowledgment(messageId, ackType, properties))); return FutureUtil.waitForAll(new ArrayList<>(completableFutureSet)); } else { - messageIds.forEach(messageId -> addAcknowledgment((MessageIdImpl) messageId, ackType, properties)); + messageIds.forEach(messageId -> addAcknowledgment(messageId, ackType, properties)); return CompletableFuture.completedFuture(null); } } else { @@ -162,46 +162,43 @@ public CompletableFuture addListAcknowledgment(List messageIds, private void addListAcknowledgment(List messageIds) { for (MessageId messageId : messageIds) { - if (messageId instanceof BatchMessageIdImpl) { - BatchMessageIdImpl batchMessageId = (BatchMessageIdImpl) messageId; - addIndividualAcknowledgment(batchMessageId.toMessageIdImpl(), - batchMessageId, + MessageIdAdv messageIdAdv = (MessageIdAdv) messageId; + if (MessageIdAdvUtils.isBatch(messageIdAdv)) { + addIndividualAcknowledgment(MessageIdAdvUtils.discardBatch(messageIdAdv), + messageIdAdv, this::doIndividualAckAsync, this::doIndividualBatchAckAsync); - } else if (messageId instanceof MessageIdImpl) { - addIndividualAcknowledgment((MessageIdImpl) messageId, + } else { + addIndividualAcknowledgment(messageIdAdv, null, this::doIndividualAckAsync, this::doIndividualBatchAckAsync); - } else { - throw new IllegalStateException("Unsupported message id type in addListAcknowledgement: " - + messageId.getClass().getCanonicalName()); } } } @Override - public CompletableFuture addAcknowledgment(MessageIdImpl msgId, AckType ackType, + public CompletableFuture addAcknowledgment(MessageId msgId, AckType ackType, Map properties) { - if (msgId instanceof BatchMessageIdImpl) { - BatchMessageIdImpl batchMessageId = (BatchMessageIdImpl) msgId; - return addAcknowledgment(batchMessageId.toMessageIdImpl(), ackType, properties, batchMessageId); + MessageIdAdv msgIdAdv = (MessageIdAdv) msgId; + if (MessageIdAdvUtils.isBatch(msgIdAdv)) { + return addAcknowledgment(MessageIdAdvUtils.discardBatch(msgId), ackType, properties, msgIdAdv); } else { - return addAcknowledgment(msgId, ackType, properties, null); + return addAcknowledgment(msgIdAdv, ackType, properties, null); } } private CompletableFuture addIndividualAcknowledgment( - MessageIdImpl msgId, - @Nullable BatchMessageIdImpl batchMessageId, - Function> individualAckFunction, - Function> batchAckFunction) { + MessageIdAdv msgId, + @Nullable MessageIdAdv batchMessageId, + Function> individualAckFunction, + Function> batchAckFunction) { if (batchMessageId != null) { consumer.onAcknowledge(batchMessageId, null); } else { consumer.onAcknowledge(msgId, null); } - if (batchMessageId == null || batchMessageId.ackIndividual()) { + if (batchMessageId == null || MessageIdAdvUtils.acknowledge(batchMessageId, true)) { consumer.getStats().incrementNumAcksSent((batchMessageId != null) ? batchMessageId.getBatchSize() : 1); consumer.getUnAckedMessageTracker().remove(msgId); if (consumer.getPossibleSendToDeadLetterTopicMessages() != null) { @@ -215,10 +212,10 @@ private CompletableFuture addIndividualAcknowledgment( } } - private CompletableFuture addAcknowledgment(MessageIdImpl msgId, + private CompletableFuture addAcknowledgment(MessageIdAdv msgId, AckType ackType, Map properties, - @Nullable BatchMessageIdImpl batchMessageId) { + @Nullable MessageIdAdv batchMessageId) { switch (ackType) { case Individual: return addIndividualAcknowledgment(msgId, @@ -231,15 +228,12 @@ private CompletableFuture addAcknowledgment(MessageIdImpl msgId, } else { consumer.onAcknowledgeCumulative(msgId, null); } - if (batchMessageId == null || batchMessageId.ackCumulative()) { + if (batchMessageId == null || MessageIdAdvUtils.acknowledge(batchMessageId, false)) { return doCumulativeAck(msgId, properties, null); } else if (batchIndexAckEnabled) { return doCumulativeBatchIndexAck(batchMessageId, properties); } else { - if (!batchMessageId.getAcker().isPrevBatchCumulativelyAcked()) { - doCumulativeAck(batchMessageId.prevBatchMessageId(), properties, null); - batchMessageId.getAcker().setPrevBatchCumulativelyAcked(true); - } + doCumulativeAck(MessageIdAdvUtils.prevMessageId(batchMessageId), properties, null); return CompletableFuture.completedFuture(null); } default: @@ -247,7 +241,7 @@ private CompletableFuture addAcknowledgment(MessageIdImpl msgId, } } - private CompletableFuture doIndividualAck(MessageIdImpl messageId, Map properties) { + private CompletableFuture doIndividualAck(MessageIdAdv messageId, Map properties) { if (acknowledgementGroupTimeMicros == 0 || (properties != null && !properties.isEmpty())) { // We cannot group acks if the delay is 0 or when there are properties attached to it. Fortunately that's an // uncommon condition since it's only used for the compaction subscription. @@ -267,13 +261,13 @@ private CompletableFuture doIndividualAck(MessageIdImpl messageId, Map doIndividualAckAsync(MessageIdImpl messageId) { + private CompletableFuture doIndividualAckAsync(MessageIdAdv messageId) { pendingIndividualAcks.add(messageId); pendingIndividualBatchIndexAcks.remove(messageId); return CompletableFuture.completedFuture(null); } - private CompletableFuture doIndividualBatchAck(BatchMessageIdImpl batchMessageId, + private CompletableFuture doIndividualBatchAck(MessageIdAdv batchMessageId, Map properties) { if (acknowledgementGroupTimeMicros == 0 || (properties != null && !properties.isEmpty())) { return doImmediateBatchIndexAck(batchMessageId, batchMessageId.getBatchIndex(), @@ -283,7 +277,7 @@ private CompletableFuture doIndividualBatchAck(BatchMessageIdImpl batchMes } } - private CompletableFuture doIndividualBatchAck(BatchMessageIdImpl batchMessageId) { + private CompletableFuture doIndividualBatchAck(MessageIdAdv batchMessageId) { Optional readLock = acquireReadLock(); try { doIndividualBatchAckAsync(batchMessageId); @@ -296,7 +290,7 @@ private CompletableFuture doIndividualBatchAck(BatchMessageIdImpl batchMes } } - private CompletableFuture doCumulativeAck(MessageIdImpl messageId, Map properties, + private CompletableFuture doCumulativeAck(MessageIdAdv messageId, Map properties, BitSetRecyclable bitSet) { consumer.getStats().incrementNumAcksSent(consumer.getUnAckedMessageTracker().removeMessagesTill(messageId)); if (acknowledgementGroupTimeMicros == 0 || (properties != null && !properties.isEmpty())) { @@ -314,29 +308,29 @@ private CompletableFuture doCumulativeAck(MessageIdImpl messageId, Map doIndividualBatchAckAsync(BatchMessageIdImpl batchMessageId) { + private CompletableFuture doIndividualBatchAckAsync(MessageIdAdv msgId) { ConcurrentBitSetRecyclable bitSet = pendingIndividualBatchIndexAcks.computeIfAbsent( - batchMessageId.toMessageIdImpl(), __ -> { - ConcurrentBitSetRecyclable value; - if (batchMessageId.getAcker() != null - && !(batchMessageId.getAcker() instanceof BatchMessageAckerDisabled)) { - value = ConcurrentBitSetRecyclable.create(batchMessageId.getAcker().getBitSet()); + MessageIdAdvUtils.discardBatch(msgId), __ -> { + final BitSet ackSet = msgId.getAckSet(); + final ConcurrentBitSetRecyclable value; + if (ackSet != null && !ackSet.isEmpty()) { + value = ConcurrentBitSetRecyclable.create(ackSet); } else { value = ConcurrentBitSetRecyclable.create(); - value.set(0, batchMessageId.getOriginalBatchSize()); + value.set(0, msgId.getBatchSize()); } return value; }); - bitSet.clear(batchMessageId.getBatchIndex()); + bitSet.clear(msgId.getBatchIndex()); return CompletableFuture.completedFuture(null); } - private void doCumulativeAckAsync(MessageIdImpl msgId, BitSetRecyclable bitSet) { + private void doCumulativeAckAsync(MessageIdAdv msgId, BitSetRecyclable bitSet) { // Handle concurrent updates from different threads lastCumulativeAck.update(msgId, bitSet); } - private CompletableFuture doCumulativeBatchIndexAck(BatchMessageIdImpl batchMessageId, + private CompletableFuture doCumulativeBatchIndexAck(MessageIdAdv batchMessageId, Map properties) { if (acknowledgementGroupTimeMicros == 0 || (properties != null && !properties.isEmpty())) { return doImmediateBatchIndexAck(batchMessageId, batchMessageId.getBatchIndex(), @@ -349,7 +343,7 @@ private CompletableFuture doCumulativeBatchIndexAck(BatchMessageIdImpl bat } } - private CompletableFuture doImmediateAck(MessageIdImpl msgId, AckType ackType, Map properties, + private CompletableFuture doImmediateAck(MessageIdAdv msgId, AckType ackType, Map properties, BitSetRecyclable bitSet) { ClientCnx cnx = consumer.getClientCnx(); @@ -360,7 +354,7 @@ private CompletableFuture doImmediateAck(MessageIdImpl msgId, AckType ackT return newImmediateAckAndFlush(consumer.consumerId, msgId, bitSet, ackType, properties, cnx); } - private CompletableFuture doImmediateBatchIndexAck(BatchMessageIdImpl msgId, int batchIndex, int batchSize, + private CompletableFuture doImmediateBatchIndexAck(MessageIdAdv msgId, int batchIndex, int batchSize, AckType ackType, Map properties) { ClientCnx cnx = consumer.getClientCnx(); @@ -369,8 +363,8 @@ private CompletableFuture doImmediateBatchIndexAck(BatchMessageIdImpl msgI .ConnectException("Consumer connect fail! consumer state:" + consumer.getState())); } BitSetRecyclable bitSet; - if (msgId.getAcker() != null && !(msgId.getAcker() instanceof BatchMessageAckerDisabled)) { - bitSet = BitSetRecyclable.valueOf(msgId.getAcker().getBitSet().toLongArray()); + if (msgId.getAckSet() != null) { + bitSet = BitSetRecyclable.valueOf(msgId.getAckSet().toLongArray()); } else { bitSet = BitSetRecyclable.create(); bitSet.set(0, batchSize); @@ -382,7 +376,7 @@ private CompletableFuture doImmediateBatchIndexAck(BatchMessageIdImpl msgI } CompletableFuture completableFuture = newMessageAckCommandAndWrite(cnx, consumer.consumerId, - msgId.ledgerId, msgId.entryId, bitSet, ackType, properties, true, null, null); + msgId.getLedgerId(), msgId.getEntryId(), bitSet, ackType, properties, true, null, null); bitSet.recycle(); return completableFuture; } @@ -414,7 +408,7 @@ private void flushAsync(ClientCnx cnx) { boolean shouldFlush = false; if (lastCumulativeAckToFlush != null) { shouldFlush = true; - final MessageIdImpl messageId = lastCumulativeAckToFlush.getMessageId(); + final MessageIdAdv messageId = lastCumulativeAckToFlush.getMessageId(); newMessageAckCommandAndWrite(cnx, consumer.consumerId, messageId.getLedgerId(), messageId.getEntryId(), lastCumulativeAckToFlush.getBitSetRecyclable(), AckType.Cumulative, Collections.emptyMap(), false, @@ -429,7 +423,7 @@ private void flushAsync(ClientCnx cnx) { if (Commands.peerSupportsMultiMessageAcknowledgment(cnx.getRemoteEndpointProtocolVersion())) { // We can send 1 single protobuf command with all individual acks while (true) { - MessageIdImpl msgId = pendingIndividualAcks.pollFirst(); + MessageIdAdv msgId = pendingIndividualAcks.pollFirst(); if (msgId == null) { break; } @@ -452,7 +446,7 @@ private void flushAsync(ClientCnx cnx) { } else { // When talking to older brokers, send the acknowledgements individually while (true) { - MessageIdImpl msgId = pendingIndividualAcks.pollFirst(); + MessageIdAdv msgId = pendingIndividualAcks.pollFirst(); if (msgId == null) { break; } @@ -465,12 +459,13 @@ private void flushAsync(ClientCnx cnx) { } if (!pendingIndividualBatchIndexAcks.isEmpty()) { - Iterator> iterator = + Iterator> iterator = pendingIndividualBatchIndexAcks.entrySet().iterator(); while (iterator.hasNext()) { - Map.Entry entry = iterator.next(); - entriesToAck.add(Triple.of(entry.getKey().ledgerId, entry.getKey().entryId, entry.getValue())); + Map.Entry entry = iterator.next(); + entriesToAck.add(Triple.of( + entry.getKey().getLedgerId(), entry.getKey().getEntryId(), entry.getValue())); iterator.remove(); } } @@ -509,7 +504,7 @@ public void close() { } } - private CompletableFuture newImmediateAckAndFlush(long consumerId, MessageIdImpl msgId, + private CompletableFuture newImmediateAckAndFlush(long consumerId, MessageIdAdv msgId, BitSetRecyclable bitSet, AckType ackType, Map map, ClientCnx cnx) { MessageIdImpl[] chunkMsgIds = this.consumer.unAckedChunkedMessageIdSequenceMap.remove(msgId); @@ -535,7 +530,7 @@ private CompletableFuture newImmediateAckAndFlush(long consumerId, Message completableFuture = CompletableFuture.completedFuture(null); } } else { - completableFuture = newMessageAckCommandAndWrite(cnx, consumerId, msgId.ledgerId, msgId.getEntryId(), + completableFuture = newMessageAckCommandAndWrite(cnx, consumerId, msgId.getLedgerId(), msgId.getEntryId(), bitSet, ackType, map, true, null, null); } return completableFuture; @@ -621,13 +616,13 @@ protected LastCumulativeAck initialValue() { return new LastCumulativeAck(); } }; - public static final MessageIdImpl DEFAULT_MESSAGE_ID = (MessageIdImpl) MessageIdImpl.earliest; + public static final MessageIdAdv DEFAULT_MESSAGE_ID = (MessageIdAdv) MessageId.earliest; - private volatile MessageIdImpl messageId = DEFAULT_MESSAGE_ID; + private volatile MessageIdAdv messageId = DEFAULT_MESSAGE_ID; private BitSetRecyclable bitSetRecyclable = null; private boolean flushRequired = false; - public synchronized void update(final MessageIdImpl messageId, final BitSetRecyclable bitSetRecyclable) { + public synchronized void update(final MessageIdAdv messageId, final BitSetRecyclable bitSetRecyclable) { if (compareTo(messageId) < 0) { if (this.bitSetRecyclable != null && this.bitSetRecyclable != bitSetRecyclable) { this.bitSetRecyclable.recycle(); @@ -662,25 +657,22 @@ public synchronized void reset() { flushRequired = false; } - public synchronized int compareTo(MessageId messageId) { - if (this.messageId instanceof BatchMessageIdImpl && (!(messageId instanceof BatchMessageIdImpl))) { - final BatchMessageIdImpl lhs = (BatchMessageIdImpl) this.messageId; - final MessageIdImpl rhs = (MessageIdImpl) messageId; - return MessageIdImpl.messageIdCompare( - lhs.getLedgerId(), lhs.getEntryId(), lhs.getPartitionIndex(), lhs.getBatchIndex(), - rhs.getLedgerId(), rhs.getEntryId(), rhs.getPartitionIndex(), Integer.MAX_VALUE); - } else if (messageId instanceof BatchMessageIdImpl && (!(this.messageId instanceof BatchMessageIdImpl))){ - final MessageIdImpl lhs = this.messageId; - final BatchMessageIdImpl rhs = (BatchMessageIdImpl) messageId; - return MessageIdImpl.messageIdCompare( - lhs.getLedgerId(), lhs.getEntryId(), lhs.getPartitionIndex(), Integer.MAX_VALUE, - rhs.getLedgerId(), rhs.getEntryId(), rhs.getPartitionIndex(), rhs.getBatchIndex()); - } else { - return this.messageId.compareTo(messageId); + public synchronized int compareTo(MessageIdAdv messageId) { + int result = Long.compare(this.messageId.getLedgerId(), messageId.getLedgerId()); + if (result != 0) { + return result; + } + result = Long.compare(this.messageId.getEntryId(), messageId.getEntryId()); + if (result != 0) { + return result; } + return Integer.compare( + (this.messageId.getBatchIndex() >= 0) ? this.messageId.getBatchIndex() : Integer.MAX_VALUE, + (messageId.getBatchIndex() >= 0) ? messageId.getBatchIndex() : Integer.MAX_VALUE + ); } - private synchronized void set(final MessageIdImpl messageId, final BitSetRecyclable bitSetRecyclable) { + private synchronized void set(final MessageIdAdv messageId, final BitSetRecyclable bitSetRecyclable) { this.messageId = messageId; this.bitSetRecyclable = bitSetRecyclable; } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ResetCursorData.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ResetCursorData.java index 06f79024c4f247..1c4230470dbc29 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ResetCursorData.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ResetCursorData.java @@ -22,6 +22,8 @@ import lombok.Data; import lombok.NoArgsConstructor; import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.MessageIdAdv; +import org.apache.pulsar.client.api.TopicMessageId; @Data @NoArgsConstructor @@ -67,18 +69,12 @@ private ResetCursorData(String position) { } public ResetCursorData(MessageId messageId) { - if (messageId instanceof BatchMessageIdImpl) { - BatchMessageIdImpl batchMessageId = (BatchMessageIdImpl) messageId; - this.ledgerId = batchMessageId.getLedgerId(); - this.entryId = batchMessageId.getEntryId(); - this.batchIndex = batchMessageId.getBatchIndex(); - this.partitionIndex = batchMessageId.partitionIndex; - } else if (messageId instanceof MessageIdImpl) { - MessageIdImpl messageIdImpl = (MessageIdImpl) messageId; - this.ledgerId = messageIdImpl.getLedgerId(); - this.entryId = messageIdImpl.getEntryId(); - this.partitionIndex = messageIdImpl.partitionIndex; - } else if (messageId instanceof TopicMessageIdImpl) { + MessageIdAdv messageIdAdv = (MessageIdAdv) messageId; + this.ledgerId = messageIdAdv.getLedgerId(); + this.entryId = messageIdAdv.getEntryId(); + this.batchIndex = messageIdAdv.getBatchIndex(); + this.partitionIndex = messageIdAdv.getPartitionIndex(); + if (messageId instanceof TopicMessageId) { throw new IllegalArgumentException("Not supported operation on partitioned-topic"); } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicMessageIdImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicMessageIdImpl.java index 941f18cf65a2c6..405fa96c6246e3 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicMessageIdImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicMessageIdImpl.java @@ -21,16 +21,12 @@ import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.TopicMessageId; -public class TopicMessageIdImpl implements TopicMessageId { +public class TopicMessageIdImpl extends TopicMessageId.Impl { - /** This topicPartitionName is get from ConsumerImpl, it contains partition part. */ - private final String topicPartitionName; private final String topicName; - private final MessageId messageId; public TopicMessageIdImpl(String topicPartitionName, String topicName, MessageId messageId) { - this.messageId = messageId; - this.topicPartitionName = topicPartitionName; + super(topicPartitionName, messageId); this.topicName = topicName; } @@ -49,40 +45,16 @@ public String getTopicName() { */ @Deprecated public String getTopicPartitionName() { - return this.topicPartitionName; - } - - public MessageId getInnerMessageId() { - return messageId; - } - - @Override - public String toString() { - return messageId.toString(); - } - - @Override - public byte[] toByteArray() { - return messageId.toByteArray(); - } - - @Override - public int hashCode() { - return messageId.hashCode(); + return getOwnerTopic(); } @Override public boolean equals(Object obj) { - return messageId.equals(obj); + return super.equals(obj); } @Override - public int compareTo(MessageId o) { - return messageId.compareTo(o); - } - - @Override - public String getOwnerTopic() { - return topicPartitionName; + public int hashCode() { + return super.hashCode(); } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicMessageImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicMessageImpl.java index c3fcb0a16a383d..c1223ee4904e9f 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicMessageImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicMessageImpl.java @@ -68,11 +68,6 @@ public MessageId getMessageId() { return messageId; } - @Deprecated - public MessageId getInnerMessageId() { - return MessageIdImpl.convertToMessageIdImpl(messageId); - } - @Override public Map getProperties() { return msg.getProperties(); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ZeroQueueConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ZeroQueueConsumerImpl.java index 42eb197d632d37..ae874b4da6d6b3 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ZeroQueueConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ZeroQueueConsumerImpl.java @@ -174,7 +174,8 @@ private void triggerZeroQueueSizeListener(final Message message) { } waitingOnListenerForZeroQueueSize = true; trackMessage(message); - unAckedMessageTracker.add(normalizeMessageId(message.getMessageId()), message.getRedeliveryCount()); + unAckedMessageTracker.add( + MessageIdAdvUtils.discardBatch(message.getMessageId()), message.getRedeliveryCount()); listener.received(ZeroQueueConsumerImpl.this, beforeConsume(message)); } catch (Throwable t) { log.error("[{}][{}] Message listener error in processing unqueued message: {}", topic, subscription, diff --git a/pulsar-client/src/main/resources/findbugsExclude.xml b/pulsar-client/src/main/resources/findbugsExclude.xml index e5f8babe841b8b..92ec9e934ee1ee 100644 --- a/pulsar-client/src/main/resources/findbugsExclude.xml +++ b/pulsar-client/src/main/resources/findbugsExclude.xml @@ -1007,4 +1007,9 @@ + + + + + diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AcknowledgementsGroupingTrackerTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AcknowledgementsGroupingTrackerTest.java index ddca6951e49e15..0418a54c772ccc 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AcknowledgementsGroupingTrackerTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AcknowledgementsGroupingTrackerTest.java @@ -38,6 +38,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; import org.apache.pulsar.client.util.TimedCompletableFuture; @@ -61,7 +62,7 @@ public void setup() throws NoSuchFieldException, IllegalAccessException { eventLoopGroup = new NioEventLoopGroup(1); consumer = mock(ConsumerImpl.class); consumer.unAckedChunkedMessageIdSequenceMap = - ConcurrentOpenHashMap.newBuilder().build(); + ConcurrentOpenHashMap.newBuilder().build(); cnx = spy(new ClientCnxTest(new ClientConfigurationData(), eventLoopGroup)); PulsarClientImpl client = mock(PulsarClientImpl.class); doReturn(client).when(consumer).getClient(); @@ -391,21 +392,21 @@ public void testBatchAckTrackerMultiAck(boolean isNeedReceipt) throws Exception public void testDoIndividualBatchAckAsync() throws Exception{ ConsumerConfigurationData conf = new ConsumerConfigurationData<>(); AcknowledgmentsGroupingTracker tracker = new PersistentAcknowledgmentsGroupingTracker(consumer, conf, eventLoopGroup); - MessageId messageId1 = new BatchMessageIdImpl(5, 1, 0, 3, 10, BatchMessageAckerDisabled.INSTANCE); + MessageId messageId1 = new BatchMessageIdImpl(5, 1, 0, 3, 10, null); BitSet bitSet = new BitSet(20); for(int i = 0; i < 20; i ++) { bitSet.set(i, true); } - MessageId messageId2 = new BatchMessageIdImpl(3, 2, 0, 5, 20, BatchMessageAcker.newAcker(bitSet)); + MessageId messageId2 = new BatchMessageIdImpl(3, 2, 0, 5, 20, bitSet); Method doIndividualBatchAckAsync = PersistentAcknowledgmentsGroupingTracker.class - .getDeclaredMethod("doIndividualBatchAckAsync", BatchMessageIdImpl.class); + .getDeclaredMethod("doIndividualBatchAckAsync", MessageIdAdv.class); doIndividualBatchAckAsync.setAccessible(true); doIndividualBatchAckAsync.invoke(tracker, messageId1); doIndividualBatchAckAsync.invoke(tracker, messageId2); Field pendingIndividualBatchIndexAcks = PersistentAcknowledgmentsGroupingTracker.class.getDeclaredField("pendingIndividualBatchIndexAcks"); pendingIndividualBatchIndexAcks.setAccessible(true); - ConcurrentHashMap batchIndexAcks = - (ConcurrentHashMap) pendingIndividualBatchIndexAcks.get(tracker); + ConcurrentHashMap batchIndexAcks = + (ConcurrentHashMap) pendingIndividualBatchIndexAcks.get(tracker); MessageIdImpl position1 = new MessageIdImpl(5, 1, 0); MessageIdImpl position2 = new MessageIdImpl(3, 2, 0); assertTrue(batchIndexAcks.containsKey(position1)); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BatchMessageAckerDisabledTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BatchMessageAckerDisabledTest.java deleted file mode 100644 index 1b3795d878cd8a..00000000000000 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BatchMessageAckerDisabledTest.java +++ /dev/null @@ -1,47 +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.impl; - -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertTrue; - -import org.testng.annotations.Test; - -public class BatchMessageAckerDisabledTest { - - @Test - public void testAckIndividual() { - for (int i = 0; i < 10; i++) { - assertTrue(BatchMessageAckerDisabled.INSTANCE.ackIndividual(i)); - } - } - - @Test - public void testAckCumulative() { - for (int i = 0; i < 10; i++) { - assertTrue(BatchMessageAckerDisabled.INSTANCE.ackCumulative(i)); - } - } - - @Test - public void testGetOutstandingAcks() { - assertEquals(0, BatchMessageAckerDisabled.INSTANCE.getOutstandingAcks()); - } - -} diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BatchMessageAckerTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BatchMessageAckerTest.java deleted file mode 100644 index d31fd18cba9710..00000000000000 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BatchMessageAckerTest.java +++ /dev/null @@ -1,83 +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.impl; - -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertFalse; -import static org.testng.Assert.assertTrue; - -import org.testng.Assert; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; - -import java.util.BitSet; - -public class BatchMessageAckerTest { - - private static final int BATCH_SIZE = 10; - - private BatchMessageAcker acker; - - @BeforeMethod - public void setup() { - acker = BatchMessageAcker.newAcker(10); - } - - @Test - public void testAckers() { - assertEquals(BATCH_SIZE, acker.getOutstandingAcks()); - assertEquals(BATCH_SIZE, acker.getBatchSize()); - - assertFalse(acker.ackIndividual(4)); - for (int i = 0; i < BATCH_SIZE; i++) { - if (4 == i) { - assertFalse(acker.getBitSet().get(i)); - } else { - assertTrue(acker.getBitSet().get(i)); - } - } - - assertFalse(acker.ackCumulative(6)); - for (int i = 0; i < BATCH_SIZE; i++) { - if (i <= 6) { - assertFalse(acker.getBitSet().get(i)); - } else { - assertTrue(acker.getBitSet().get(i)); - } - } - - for (int i = BATCH_SIZE - 1; i >= 8; i--) { - assertFalse(acker.ackIndividual(i)); - assertFalse(acker.getBitSet().get(i)); - } - - assertTrue(acker.ackIndividual(7)); - assertEquals(0, acker.getOutstandingAcks()); - } - - @Test - public void testBitSetAcker() { - BitSet bitSet = BitSet.valueOf(acker.getBitSet().toLongArray()); - BatchMessageAcker bitSetAcker = BatchMessageAcker.newAcker(bitSet); - - Assert.assertEquals(acker.getBitSet(), bitSetAcker.getBitSet()); - Assert.assertEquals(acker.getOutstandingAcks(), bitSetAcker.getOutstandingAcks()); - } - -} diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BatchMessageIdImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BatchMessageIdImplTest.java index 6bf9cd943483f1..10d805cdc4db31 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BatchMessageIdImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BatchMessageIdImplTest.java @@ -20,13 +20,8 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotEquals; -import static org.testng.Assert.assertTrue; -import static org.testng.Assert.fail; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectWriter; import java.io.IOException; import java.util.Collections; -import org.apache.pulsar.common.util.ObjectMapperFactory; import org.testng.annotations.Test; public class BatchMessageIdImplTest { @@ -123,36 +118,10 @@ public void hashCodeUnbatchedTest() { assertEquals(batchMsgId2.hashCode(), msgId2.hashCode()); } - @Test - public void deserializationTest() { - // initialize BitSet with null - BatchMessageAcker ackerDisabled = new BatchMessageAcker(null, 0); - BatchMessageIdImpl batchMsgId = new BatchMessageIdImpl(0, 0, 0, 0, 0, ackerDisabled); - - ObjectWriter writer = ObjectMapperFactory.create().writerWithDefaultPrettyPrinter(); - - try { - writer.writeValueAsString(batchMsgId); - fail("Shouldn't be deserialized"); - } catch (JsonProcessingException e) { - // expected - assertTrue(e.getCause() instanceof NullPointerException); - } - - // use the default BatchMessageAckerDisabled - BatchMessageIdImpl batchMsgIdToDeserialize = new BatchMessageIdImpl(0, 0, 0, 0); - - try { - writer.writeValueAsString(batchMsgIdToDeserialize); - } catch (JsonProcessingException e) { - fail("Should be successful"); - } - } - @Test public void serializeAndDeserializeTest() throws IOException { BatchMessageIdImpl batchMessageId = new BatchMessageIdImpl(1, 1, 0, - 1, 10, BatchMessageAcker.newAcker(10)); + 1, 10, BatchMessageIdImpl.newAckSet(10)); byte[] serialized = batchMessageId.toByteArray(); BatchMessageIdImpl deserialized = (BatchMessageIdImpl) MessageIdImpl.fromByteArray(serialized); assertEquals(deserialized.getBatchSize(), batchMessageId.getBatchSize()); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/MessageIdSerializationTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/MessageIdSerializationTest.java index 7f029635241de2..4173d6439b9319 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/MessageIdSerializationTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/MessageIdSerializationTest.java @@ -43,8 +43,7 @@ public void testProtobufSerialization2() throws Exception { @Test public void testBatchSizeNotSet() throws Exception { - MessageId id = new BatchMessageIdImpl(1L, 2L, 3, 4, -1, - BatchMessageAckerDisabled.INSTANCE); + MessageId id = new BatchMessageIdImpl(1L, 2L, 3, 4, -1, null); byte[] serialized = id.toByteArray(); assertEquals(MessageId.fromByteArray(serialized), id); assertEquals(MessageId.fromByteArrayWithTopic(serialized, "my-topic"), id); diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionCommon.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionCommon.java index bda99a39478a39..43261ee76860d2 100644 --- a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionCommon.java +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionCommon.java @@ -45,6 +45,7 @@ import net.jodah.typetools.TypeResolver; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.common.functions.FunctionConfig; @@ -314,7 +315,7 @@ public static String getFullyQualifiedInstanceId(String tenant, String namespace } public static final long getSequenceId(MessageId messageId) { - MessageIdImpl msgId = MessageIdImpl.convertToMessageIdImpl(messageId); + MessageIdAdv msgId = (MessageIdAdv) messageId; long ledgerId = msgId.getLedgerId(); long entryId = msgId.getEntryId(); diff --git a/pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/KafkaConnectSink.java b/pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/KafkaConnectSink.java index 37d0987e610239..171f2542a51a5e 100644 --- a/pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/KafkaConnectSink.java +++ b/pulsar-io/kafka-connect-adaptor/src/main/java/org/apache/pulsar/io/kafka/connect/KafkaConnectSink.java @@ -26,6 +26,7 @@ import com.google.common.collect.Maps; import com.google.common.util.concurrent.ThreadFactoryBuilder; import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -365,6 +366,23 @@ static class BatchMessageSequenceRef { int batchIdx; } + private static Method getMethodOfMessageId(MessageId messageId, String name) throws NoSuchMethodException { + Class clazz = messageId.getClass(); + NoSuchMethodException firstException = null; + while (clazz != null) { + try { + return clazz.getDeclaredMethod(name); + } catch (NoSuchMethodException e) { + if (firstException == null) { + firstException = e; + } + clazz = clazz.getSuperclass(); + } + } + assert firstException != null; + throw firstException; + } + @VisibleForTesting static BatchMessageSequenceRef getMessageSequenceRefForBatchMessage(MessageId messageId) { long ledgerId; @@ -372,23 +390,17 @@ static BatchMessageSequenceRef getMessageSequenceRefForBatchMessage(MessageId me int batchIdx; try { try { - messageId = (MessageId) messageId.getClass().getDeclaredMethod("getInnerMessageId").invoke(messageId); - } catch (NoSuchMethodException noSuchMethodException) { - // not a TopicMessageIdImpl - } - - try { - batchIdx = (int) messageId.getClass().getDeclaredMethod("getBatchIndex").invoke(messageId); + batchIdx = (int) getMethodOfMessageId(messageId, "getBatchIndex").invoke(messageId); + if (batchIdx < 0) { + return null; + } } catch (NoSuchMethodException noSuchMethodException) { // not a BatchMessageIdImpl, returning null to use the standard sequenceId return null; } - // if getBatchIndex exists it means messageId is a 'BatchMessageIdImpl' instance. - final Class messageIdImplClass = messageId.getClass().getSuperclass(); - - ledgerId = (long) messageIdImplClass.getDeclaredMethod("getLedgerId").invoke(messageId); - entryId = (long) messageIdImplClass.getDeclaredMethod("getEntryId").invoke(messageId); + ledgerId = (long) getMethodOfMessageId(messageId, "getLedgerId").invoke(messageId); + entryId = (long) getMethodOfMessageId(messageId, "getEntryId").invoke(messageId); } catch (IllegalAccessException | NoSuchMethodException | InvocationTargetException ex) { log.error("Unexpected error while retrieving sequenceId, messageId class: {}, error: {}", messageId.getClass().getName(), ex.getMessage(), ex);