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