@@ -440,12 +440,9 @@ public ProcessContinuation processElement(
440
440
LOG .info ("Creating Kafka consumer for process continuation for {}" , kafkaSourceDescriptor );
441
441
try (Consumer <byte [], byte []> consumer = consumerFactoryFn .apply (updatedConsumerConfig )) {
442
442
consumer .assign (ImmutableList .of (kafkaSourceDescriptor .getTopicPartition ()));
443
- long startOffset = tracker .currentRestriction ().getFrom ();
444
- long expectedOffset = startOffset ;
445
- consumer .seek (kafkaSourceDescriptor .getTopicPartition (), startOffset );
443
+ long expectedOffset = tracker .currentRestriction ().getFrom ();
444
+ consumer .seek (kafkaSourceDescriptor .getTopicPartition (), expectedOffset );
446
445
ConsumerRecords <byte [], byte []> rawRecords = ConsumerRecords .empty ();
447
- long skippedRecords = 0L ;
448
- final Stopwatch sw = Stopwatch .createStarted ();
449
446
450
447
while (true ) {
451
448
// Fetch the record size accumulator.
@@ -465,36 +462,6 @@ public ProcessContinuation processElement(
465
462
return ProcessContinuation .resume ();
466
463
}
467
464
for (ConsumerRecord <byte [], byte []> rawRecord : rawRecords ) {
468
- // If the Kafka consumer returns a record with an offset that is already processed
469
- // the record can be safely skipped. This is needed because there is a possibility
470
- // that the seek() above fails to move the offset to the desired position. In which
471
- // case poll() would return records that are already cnsumed.
472
- if (rawRecord .offset () < startOffset ) {
473
- // If the start offset is not reached even after skipping the records for 10 seconds
474
- // then the processing is stopped with a backoff to give the Kakfa server some time
475
- // catch up.
476
- if (sw .elapsed ().getSeconds () > 10L ) {
477
- LOG .error (
478
- "The expected offset ({}) was not reached even after"
479
- + " skipping consumed records for 10 seconds. The offset we could"
480
- + " reach was {}. The processing of this bundle will be attempted"
481
- + " at a later time." ,
482
- expectedOffset ,
483
- rawRecord .offset ());
484
- return ProcessContinuation .resume ()
485
- .withResumeDelay (org .joda .time .Duration .standardSeconds (10L ));
486
- }
487
- skippedRecords ++;
488
- continue ;
489
- }
490
- if (skippedRecords > 0L ) {
491
- LOG .warn (
492
- "{} records were skipped due to seek returning an"
493
- + " earlier position than requested position of {}" ,
494
- skippedRecords ,
495
- expectedOffset );
496
- skippedRecords = 0L ;
497
- }
498
465
if (!tracker .tryClaim (rawRecord .offset ())) {
499
466
return ProcessContinuation .stop ();
500
467
}
0 commit comments