@@ -177,7 +177,7 @@ public class ConsumerImpl<T> extends ConsumerBase<T> implements ConnectionHandle
177177 private final TopicName topicName ;
178178 private final String topicNameWithoutPartition ;
179179
180- private final Map <MessageIdImpl , List <MessageImpl <T >>> possibleSendToDeadLetterTopicMessages ;
180+ private final Map <MessageIdAdv , List <MessageImpl <T >>> possibleSendToDeadLetterTopicMessages ;
181181
182182 private final DeadLetterPolicy deadLetterPolicy ;
183183
@@ -530,7 +530,6 @@ protected CompletableFuture<Messages<T>> internalBatchReceiveAsync() {
530530 protected CompletableFuture <Void > doAcknowledge (MessageId messageId , AckType ackType ,
531531 Map <String , Long > properties ,
532532 TransactionImpl txn ) {
533- checkArgument (messageId instanceof MessageIdImpl );
534533 if (getState () != State .Ready && getState () != State .Connecting ) {
535534 stats .incrementNumAcksFailed ();
536535 PulsarClientException exception = new PulsarClientException ("Consumer not ready. State: " + getState ());
@@ -546,7 +545,7 @@ protected CompletableFuture<Void> doAcknowledge(MessageId messageId, AckType ack
546545 return doTransactionAcknowledgeForResponse (messageId , ackType , null , properties ,
547546 new TxnID (txn .getTxnIdMostBits (), txn .getTxnIdLeastBits ()));
548547 }
549- return acknowledgmentsGroupingTracker .addAcknowledgment ((MessageIdImpl ) messageId , ackType , properties );
548+ return acknowledgmentsGroupingTracker .addAcknowledgment ((MessageIdAdv ) messageId , ackType , properties );
550549 }
551550
552551 @ Override
@@ -587,10 +586,6 @@ protected CompletableFuture<Void> doReconsumeLater(Message<?> message, AckType a
587586 .InvalidMessageException ("Cannot handle message with null messageId" ));
588587 }
589588
590- if (messageId instanceof TopicMessageIdImpl ) {
591- messageId = ((TopicMessageIdImpl ) messageId ).getInnerMessageId ();
592- }
593- checkArgument (messageId instanceof MessageIdImpl );
594589 if (getState () != State .Ready && getState () != State .Connecting ) {
595590 stats .incrementNumAcksFailed ();
596591 PulsarClientException exception = new PulsarClientException ("Consumer not ready. State: " + getState ());
@@ -626,7 +621,7 @@ protected CompletableFuture<Void> doReconsumeLater(Message<?> message, AckType a
626621 if (retryLetterProducer != null ) {
627622 try {
628623 MessageImpl <T > retryMessage = (MessageImpl <T >) getMessageImpl (message );
629- String originMessageIdStr = getOriginMessageIdStr ( message );
624+ String originMessageIdStr = message . getMessageId (). toString ( );
630625 String originTopicNameStr = getOriginTopicNameStr (message );
631626 SortedMap <String , String > propertiesMap =
632627 getPropertiesMap (message , originMessageIdStr , originTopicNameStr );
@@ -718,15 +713,6 @@ private SortedMap<String, String> getPropertiesMap(Message<?> message,
718713 return propertiesMap ;
719714 }
720715
721- private String getOriginMessageIdStr (Message <?> message ) {
722- if (message instanceof TopicMessageImpl ) {
723- return ((TopicMessageIdImpl ) message .getMessageId ()).getInnerMessageId ().toString ();
724- } else if (message instanceof MessageImpl ) {
725- return message .getMessageId ().toString ();
726- }
727- return null ;
728- }
729-
730716 private String getOriginTopicNameStr (Message <?> message ) {
731717 if (message instanceof TopicMessageImpl ) {
732718 return ((TopicMessageIdImpl ) message .getMessageId ()).getTopicName ();
@@ -2005,7 +1991,7 @@ private CompletableFuture<Boolean> processPossibleToDLQ(MessageIdImpl messageId)
20051991 MessageIdImpl finalMessageId = messageId ;
20061992 deadLetterProducer .thenAcceptAsync (producerDLQ -> {
20071993 for (MessageImpl <T > message : finalDeadLetterMessages ) {
2008- String originMessageIdStr = getOriginMessageIdStr ( message );
1994+ String originMessageIdStr = message . getMessageId (). toString ( );
20091995 String originTopicNameStr = getOriginTopicNameStr (message );
20101996 TypedMessageBuilder <byte []> typedMessageBuilderNew =
20111997 producerDLQ .newMessage (Schema .AUTO_PRODUCE_BYTES (message .getReaderSchema ().get ()))
@@ -2178,24 +2164,24 @@ public CompletableFuture<Void> seekAsync(MessageId messageId) {
21782164 String seekBy = String .format ("the message %s" , messageId .toString ());
21792165 return seekAsyncCheckState (seekBy ).orElseGet (() -> {
21802166 long requestId = client .newRequestId ();
2181- ByteBuf seek = null ;
2182- if (messageId instanceof BatchMessageIdImpl ) {
2183- BatchMessageIdImpl msgId = (BatchMessageIdImpl ) messageId ;
2184- // Initialize ack set
2185- BitSetRecyclable ackSet = BitSetRecyclable .create ();
2186- ackSet .set (0 , msgId .getBatchSize ());
2187- ackSet .clear (0 , Math .max (msgId .getBatchIndex (), 0 ));
2188- long [] ackSetArr = ackSet .toLongArray ();
2189- ackSet .recycle ();
2190-
2191- seek = Commands .newSeek (consumerId , requestId , msgId .getLedgerId (), msgId .getEntryId (), ackSetArr );
2192- } else if (messageId instanceof ChunkMessageIdImpl ) {
2193- ChunkMessageIdImpl msgId = (ChunkMessageIdImpl ) messageId ;
2194- seek = Commands .newSeek (consumerId , requestId , msgId .getFirstChunkMessageId ().getLedgerId (),
2195- msgId .getFirstChunkMessageId ().getEntryId (), new long [0 ]);
2167+ final MessageIdAdv msgId = (MessageIdAdv ) messageId ;
2168+ final MessageIdAdv firstChunkMsgId = msgId .getFirstChunkMessageId ();
2169+ final ByteBuf seek ;
2170+ if (msgId .getFirstChunkMessageId () != null ) {
2171+ seek = Commands .newSeek (consumerId , requestId , firstChunkMsgId .getLedgerId (),
2172+ firstChunkMsgId .getEntryId (), new long [0 ]);
21962173 } else {
2197- MessageIdImpl msgId = (MessageIdImpl ) messageId ;
2198- seek = Commands .newSeek (consumerId , requestId , msgId .getLedgerId (), msgId .getEntryId (), new long [0 ]);
2174+ final long [] ackSetArr ;
2175+ if (msgId .getBatchIndex () >= 0 && msgId .getBatchSize () > 0 ) {
2176+ final BitSetRecyclable ackSet = BitSetRecyclable .create ();
2177+ ackSet .set (0 , msgId .getBatchSize ());
2178+ ackSet .clear (0 , Math .max (msgId .getBatchIndex (), 0 ));
2179+ ackSetArr = ackSet .toLongArray ();
2180+ ackSet .recycle ();
2181+ } else {
2182+ ackSetArr = new long [0 ];
2183+ }
2184+ seek = Commands .newSeek (consumerId , requestId , msgId .getLedgerId (), msgId .getEntryId (), ackSetArr );
21992185 }
22002186 return seekAsyncInternal (requestId , seek , messageId , seekBy );
22012187 });
@@ -2227,9 +2213,8 @@ public CompletableFuture<Boolean> hasMessageAvailableAsync() {
22272213 }
22282214
22292215 future .thenAccept (response -> {
2230- MessageIdImpl lastMessageId = MessageIdImpl .convertToMessageIdImpl (response .lastMessageId );
2231- MessageIdImpl markDeletePosition = MessageIdImpl
2232- .convertToMessageIdImpl (response .markDeletePosition );
2216+ MessageIdAdv lastMessageId = (MessageIdAdv ) response .lastMessageId ;
2217+ MessageIdAdv markDeletePosition = (MessageIdAdv ) response .markDeletePosition ;
22332218
22342219 if (markDeletePosition != null && !(markDeletePosition .getEntryId () < 0
22352220 && markDeletePosition .getLedgerId () > lastMessageId .getLedgerId ())) {
@@ -2766,7 +2751,7 @@ private CompletableFuture<Void> doTransactionAcknowledgeForResponse(List<Message
27662751 return cnx ().newAckForReceipt (cmd , requestId );
27672752 }
27682753
2769- public Map <MessageIdImpl , List <MessageImpl <T >>> getPossibleSendToDeadLetterTopicMessages () {
2754+ public Map <MessageIdAdv , List <MessageImpl <T >>> getPossibleSendToDeadLetterTopicMessages () {
27702755 return possibleSendToDeadLetterTopicMessages ;
27712756 }
27722757
0 commit comments