Arseniy Tashoyan created FLINK-28266:
----------------------------------------

             Summary: Kafka connector fails: Invalid negative offset
                 Key: FLINK-28266
                 URL: https://issues.apache.org/jira/browse/FLINK-28266
             Project: Flink
          Issue Type: Bug
          Components: Connectors / Kafka
    Affects Versions: 1.15.0
            Reporter: Arseniy Tashoyan
         Attachments: NegativeOffsetSpec.scala

The failure occurs when reading from a Kafka topic using new KafkaSource API.

Configure KafkaSource with starting offsets set to a timestamp. This timestamp 
should be big enough, so all records in the Kafka topic have their timestamps 
smaller.
{code:scala}
val kafkaSource = KafkaSource.builder()
        .setTopics(topic)
        .setBootstrapServers(bootstrapServers)
        .setStartingOffsets(OffsetsInitializer.timestamp(VERY_BIG_VALUE))
{code}
KafkaSource finds none records in the Kafka topic that have timestamps bigger 
than this configured timestamp. In this case we observe the failure:

*IllegalArgumentException: Invalid negative offset*

Full stack trace:
{code:none}
org.apache.flink.util.FlinkRuntimeException: Failed to initialize partition 
splits due to
        at 
org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumerator.handlePartitionSplitChanges(KafkaSourceEnumerator.java:299)
 ~[flink-connector-kafka-1.15.0.jar:1.15.0]
        at 
org.apache.flink.runtime.source.coordinator.ExecutorNotifier.lambda$null$1(ExecutorNotifier.java:83)
 ~[flink-runtime-1.15.0.jar:1.15.0]
        at 
org.apache.flink.util.ThrowableCatchingRunnable.run(ThrowableCatchingRunnable.java:40)
 [flink-core-1.15.0.jar:1.15.0]
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
[?:1.8.0_332]
        at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
[?:1.8.0_332]
        at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
 [?:1.8.0_332]
        at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
 [?:1.8.0_332]
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) 
[?:1.8.0_332]
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) 
[?:1.8.0_332]
        at java.lang.Thread.run(Thread.java:750) [?:1.8.0_332]
Caused by: java.lang.IllegalArgumentException: Invalid negative offset
        at 
org.apache.kafka.clients.consumer.OffsetAndTimestamp.<init>(OffsetAndTimestamp.java:36)
 ~[kafka-clients-2.8.1.jar:?]
        at 
org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumerator$PartitionOffsetsRetrieverImpl.lambda$offsetsForTimes$8(KafkaSourceEnumerator.java:622)
 ~[flink-connector-kafka-1.15.0.jar:1.15.0]
        at java.util.stream.Collectors.lambda$toMap$58(Collectors.java:1321) 
~[?:1.8.0_332]
        at java.util.stream.ReduceOps$3ReducingSink.accept(ReduceOps.java:169) 
~[?:1.8.0_332]
        at 
java.util.HashMap$EntrySpliterator.forEachRemaining(HashMap.java:1723) 
~[?:1.8.0_332]
        at 
java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482) 
~[?:1.8.0_332]
        at 
java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472) 
~[?:1.8.0_332]
        at 
java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:708) 
~[?:1.8.0_332]
        at 
java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) 
~[?:1.8.0_332]
        at 
java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:566) 
~[?:1.8.0_332]
        at 
org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumerator$PartitionOffsetsRetrieverImpl.offsetsForTimes(KafkaSourceEnumerator.java:615)
 ~[flink-connector-kafka-1.15.0.jar:1.15.0]
        at 
org.apache.flink.connector.kafka.source.enumerator.initializer.TimestampOffsetsInitializer.getPartitionOffsets(TimestampOffsetsInitializer.java:57)
 ~[flink-connector-kafka-1.15.0.jar:1.15.0]
        at 
org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumerator.initializePartitionSplits(KafkaSourceEnumerator.java:272)
 ~[flink-connector-kafka-1.15.0.jar:1.15.0]
        at 
org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumerator.lambda$checkPartitionChanges$0(KafkaSourceEnumerator.java:242)
 ~[flink-connector-kafka-1.15.0.jar:1.15.0]
        at 
org.apache.flink.runtime.source.coordinator.ExecutorNotifier.lambda$notifyReadyAsync$2(ExecutorNotifier.java:80)
 ~[flink-runtime-1.15.0.jar:1.15.0]
{code}

A simple reproducer based on  
[embedded-kafka|https://github.com/embeddedkafka/embedded-kafka] is attached.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

Reply via email to