5959import java .util .concurrent .locks .ReentrantReadWriteLock ;
6060import java .util .function .Function ;
6161import java .util .function .Predicate ;
62+ import java .util .stream .Collectors ;
6263import java .util .stream .LongStream ;
6364import org .apache .bookkeeper .client .AsyncCallback .CloseCallback ;
6465import org .apache .bookkeeper .client .AsyncCallback .OpenCallback ;
9192import org .apache .bookkeeper .mledger .ScanOutcome ;
9293import org .apache .bookkeeper .mledger .impl .MetaStore .MetaStoreCallback ;
9394import org .apache .bookkeeper .mledger .proto .MLDataFormats ;
95+ import org .apache .bookkeeper .mledger .proto .MLDataFormats .LongListMap ;
9496import org .apache .bookkeeper .mledger .proto .MLDataFormats .LongProperty ;
9597import org .apache .bookkeeper .mledger .proto .MLDataFormats .ManagedCursorInfo ;
9698import org .apache .bookkeeper .mledger .proto .MLDataFormats .ManagedLedgerInfo .LedgerInfo ;
9799import org .apache .bookkeeper .mledger .proto .MLDataFormats .MessageRange ;
98100import org .apache .bookkeeper .mledger .proto .MLDataFormats .PositionInfo ;
101+ import org .apache .bookkeeper .mledger .proto .MLDataFormats .PositionInfo .Builder ;
99102import org .apache .bookkeeper .mledger .proto .MLDataFormats .StringProperty ;
103+ import org .apache .commons .lang3 .mutable .MutableInt ;
100104import org .apache .commons .lang3 .tuple .Pair ;
101105import org .apache .pulsar .common .policies .data .ManagedLedgerInternalStats ;
102106import org .apache .pulsar .common .util .DateFormatter ;
@@ -606,9 +610,7 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac
606610 }
607611
608612 Position position = PositionFactory .create (positionInfo .getLedgerId (), positionInfo .getEntryId ());
609- if (positionInfo .getIndividualDeletedMessagesCount () > 0 ) {
610- recoverIndividualDeletedMessages (positionInfo .getIndividualDeletedMessagesList ());
611- }
613+ recoverIndividualDeletedMessages (positionInfo );
612614 if (getConfig ().isDeletionAtBatchIndexLevelEnabled ()
613615 && positionInfo .getBatchedEntryDeletionIndexInfoCount () > 0 ) {
614616 recoverBatchDeletedIndexes (positionInfo .getBatchedEntryDeletionIndexInfoList ());
@@ -627,6 +629,45 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac
627629 }
628630 }
629631
632+ public void recoverIndividualDeletedMessages (PositionInfo positionInfo ) {
633+ if (positionInfo .getIndividualDeletedMessagesCount () > 0 ) {
634+ recoverIndividualDeletedMessages (positionInfo .getIndividualDeletedMessagesList ());
635+ } else if (positionInfo .getIndividualDeletedMessageRangesCount () > 0 ) {
636+ List <LongListMap > rangeList = positionInfo .getIndividualDeletedMessageRangesList ();
637+ try {
638+ Map <Long , long []> rangeMap = rangeList .stream ().collect (Collectors .toMap (LongListMap ::getKey ,
639+ list -> list .getValuesList ().stream ().mapToLong (i -> i ).toArray ()));
640+ individualDeletedMessages .build (rangeMap );
641+ } catch (Exception e ) {
642+ log .warn ("[{}]-{} Failed to recover individualDeletedMessages from serialized data" , ledger .getName (),
643+ name , e );
644+ }
645+ }
646+ }
647+
648+ private List <LongListMap > buildLongPropertiesMap (Map <Long , long []> properties ) {
649+ if (properties .isEmpty ()) {
650+ return Collections .emptyList ();
651+ }
652+ List <LongListMap > longListMap = new ArrayList <>();
653+ MutableInt serializedSize = new MutableInt ();
654+ properties .forEach ((id , ranges ) -> {
655+ if (ranges == null || ranges .length <= 0 ) {
656+ return ;
657+ }
658+ org .apache .bookkeeper .mledger .proto .MLDataFormats .LongListMap .Builder lmBuilder = LongListMap .newBuilder ()
659+ .setKey (id );
660+ for (long range : ranges ) {
661+ lmBuilder .addValues (range );
662+ }
663+ LongListMap lm = lmBuilder .build ();
664+ longListMap .add (lm );
665+ serializedSize .add (lm .getSerializedSize ());
666+ });
667+ individualDeletedMessagesSerializedSize = serializedSize .toInteger ();
668+ return longListMap ;
669+ }
670+
630671 private void recoverIndividualDeletedMessages (List <MLDataFormats .MessageRange > individualDeletedMessagesList ) {
631672 lock .writeLock ().lock ();
632673 try {
@@ -3125,12 +3166,23 @@ private List<MLDataFormats.BatchedEntryDeletionIndexInfo> buildBatchEntryDeletio
31253166
31263167 void persistPositionToLedger (final LedgerHandle lh , MarkDeleteEntry mdEntry , final VoidCallback callback ) {
31273168 Position position = mdEntry .newPosition ;
3128- PositionInfo pi = PositionInfo .newBuilder ().setLedgerId (position .getLedgerId ())
3169+ Builder piBuilder = PositionInfo .newBuilder ().setLedgerId (position .getLedgerId ())
31293170 .setEntryId (position .getEntryId ())
3130- .addAllIndividualDeletedMessages (buildIndividualDeletedMessageRanges ())
31313171 .addAllBatchedEntryDeletionIndexInfo (buildBatchEntryDeletionIndexInfoList ())
3132- .addAllProperties (buildPropertiesMap (mdEntry .properties )). build () ;
3172+ .addAllProperties (buildPropertiesMap (mdEntry .properties ));
31333173
3174+ Map <Long , long []> internalRanges = null ;
3175+ try {
3176+ internalRanges = individualDeletedMessages .toRanges (getConfig ().getMaxUnackedRangesToPersist ());
3177+ } catch (Exception e ) {
3178+ log .warn ("[{}]-{} Failed to serialize individualDeletedMessages" , ledger .getName (), name , e );
3179+ }
3180+ if (internalRanges != null && !internalRanges .isEmpty ()) {
3181+ piBuilder .addAllIndividualDeletedMessageRanges (buildLongPropertiesMap (internalRanges ));
3182+ } else {
3183+ piBuilder .addAllIndividualDeletedMessages (buildIndividualDeletedMessageRanges ());
3184+ }
3185+ PositionInfo pi = piBuilder .build ();
31343186
31353187 if (log .isDebugEnabled ()) {
31363188 log .debug ("[{}] Cursor {} Appending to ledger={} position={}" , ledger .getName (), name , lh .getId (),
0 commit comments