2929import java .util .concurrent .ScheduledExecutorService ;
3030import java .util .concurrent .Semaphore ;
3131import java .util .concurrent .TimeUnit ;
32+ import java .util .function .BiFunction ;
3233import java .util .function .BiPredicate ;
3334import org .apache .bookkeeper .client .BKException ;
3435import org .apache .bookkeeper .client .BookKeeper ;
3839import org .apache .pulsar .broker .ServiceConfiguration ;
3940import org .apache .pulsar .client .api .MessageId ;
4041import org .apache .pulsar .client .api .PulsarClient ;
42+ import org .apache .pulsar .client .api .PulsarClientException ;
4143import org .apache .pulsar .client .api .RawMessage ;
4244import org .apache .pulsar .client .api .RawReader ;
4345import org .apache .pulsar .client .impl .MessageIdImpl ;
4446import org .apache .pulsar .client .impl .RawBatchConverter ;
4547import org .apache .pulsar .common .api .proto .MessageMetadata ;
4648import org .apache .pulsar .common .protocol .Commands ;
4749import org .apache .pulsar .common .protocol .Markers ;
50+ import org .apache .pulsar .common .util .Backoff ;
4851import org .apache .pulsar .common .util .FutureUtil ;
4952import org .slf4j .Logger ;
5053import org .slf4j .LoggerFactory ;
@@ -62,6 +65,9 @@ public abstract class AbstractTwoPhaseCompactor<T> extends Compactor {
6265
6366 @ VisibleForTesting
6467 static Runnable injectionAfterSeekInPhaseTwo = () -> {};
68+ @ VisibleForTesting
69+ static BiFunction <RawReader , MessageId , CompletableFuture <Void >> injectionPhaseTwoSeek =
70+ RawReader ::seekAsync ;
6571 private static final Logger log = LoggerFactory .getLogger (AbstractTwoPhaseCompactor .class );
6672 protected static final int MAX_OUTSTANDING = 500 ;
6773 protected final Duration phaseOneLoopReadTimeout ;
@@ -190,7 +196,7 @@ private CompletableFuture<Long> phaseTwoSeekThenLoop(RawReader reader, MessageId
190196 LedgerHandle ledger ) {
191197 CompletableFuture <Long > promise = new CompletableFuture <>();
192198
193- reader . seekAsync ( from ).thenCompose ((v ) -> {
199+ phaseTwoSeekWithRetry ( reader , from ).thenCompose ((v ) -> {
194200 injectionAfterSeekInPhaseTwo .run ();
195201 Semaphore outstanding = new Semaphore (MAX_OUTSTANDING );
196202 CompletableFuture <Void > loopPromise = new CompletableFuture <>();
@@ -215,6 +221,53 @@ private CompletableFuture<Long> phaseTwoSeekThenLoop(RawReader reader, MessageId
215221 return promise ;
216222 }
217223
224+ /**
225+ * Seek the compaction subscription to {@code from}, retrying on transient
226+ * {@link PulsarClientException.ConnectException}.
227+ *
228+ * <p>Server-side, {@code PersistentSubscription.resetCursorInternal} disconnects the compaction
229+ * consumer before resetting the managed cursor, then sends the success response. This races with
230+ * the client: {@code channelInactive} fires on the consumer's {@code ClientCnx} and fails the
231+ * in-flight seek future with {@code ConnectException} before the broker's success response
232+ * arrives. The seek is idempotent and the cursor is already repositioned server-side, so
233+ * retrying after a short backoff lets the client reconnect and the next seek complete normally.
234+ * Non-transient failures propagate immediately.
235+ */
236+ private CompletableFuture <Void > phaseTwoSeekWithRetry (RawReader reader , MessageId from ) {
237+ CompletableFuture <Void > promise = new CompletableFuture <>();
238+ Backoff backoff = Backoff .builder ()
239+ .initialDelay (Duration .ofMillis (100 ))
240+ .maxBackoff (Duration .ofSeconds (1 ))
241+ .mandatoryStop (Duration .ofSeconds (10 ))
242+ .build ();
243+ attemptPhaseTwoSeek (reader , from , backoff , promise );
244+ return promise ;
245+ }
246+
247+ private void attemptPhaseTwoSeek (RawReader reader , MessageId from , Backoff backoff ,
248+ CompletableFuture <Void > promise ) {
249+ injectionPhaseTwoSeek .apply (reader , from ).whenComplete ((v , ex ) -> {
250+ if (ex == null ) {
251+ promise .complete (null );
252+ return ;
253+ }
254+ Throwable cause = FutureUtil .unwrapCompletionException (ex );
255+ if (!(cause instanceof PulsarClientException .ConnectException )) {
256+ promise .completeExceptionally (cause );
257+ return ;
258+ }
259+ long nextMs = backoff .next ().toMillis ();
260+ if (backoff .isMandatoryStopMade ()) {
261+ promise .completeExceptionally (cause );
262+ return ;
263+ }
264+ log .warn ("Phase two seek failed transiently, will retry. topic={} from={} nextMs={}" ,
265+ reader .getTopic (), from , nextMs , cause );
266+ scheduler .schedule (() -> attemptPhaseTwoSeek (reader , from , backoff , promise ),
267+ nextMs , TimeUnit .MILLISECONDS );
268+ });
269+ }
270+
218271 private void phaseTwoLoop (RawReader reader , MessageId to , Map <String , MessageId > latestForKey ,
219272 LedgerHandle lh , Semaphore outstanding , CompletableFuture <Void > promise ,
220273 MessageId lastCompactedMessageId ) {
0 commit comments