You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by "Eugene Kirpichov (JIRA)" <ji...@apache.org> on 2017/10/31 00:00:03 UTC

[jira] [Closed] (BEAM-2979) Race condition between KafkaIO.UnboundedKafkaReader.getWatermark() and KafkaIO.UnboundedKafkaReader.advance()

     [ https://issues.apache.org/jira/browse/BEAM-2979?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Eugene Kirpichov closed BEAM-2979.
----------------------------------
       Resolution: Fixed
    Fix Version/s:     (was: 2.3.0)
                   2.2.0

> Race condition between KafkaIO.UnboundedKafkaReader.getWatermark() and KafkaIO.UnboundedKafkaReader.advance()
> -------------------------------------------------------------------------------------------------------------
>
>                 Key: BEAM-2979
>                 URL: https://issues.apache.org/jira/browse/BEAM-2979
>             Project: Beam
>          Issue Type: Bug
>          Components: sdk-java-core
>    Affects Versions: 2.0.0, 2.1.0
>            Reporter: Wesley Tanaka
>            Assignee: Raghu Angadi
>             Fix For: 2.2.0
>
>
> getWatermark() looks like this:
> {noformat}
>     @Override
>     public Instant getWatermark() {
>       if (curRecord == null) {
>         LOG.debug("{}: getWatermark() : no records have been read yet.", name);
>         return initialWatermark;
>       }
>       return source.spec.getWatermarkFn() != null
>           ? source.spec.getWatermarkFn().apply(curRecord) : curTimestamp;
>     }
> {noformat}
> advance() has code in it that looks like this:
> {noformat}
>           curRecord = null; // user coders below might throw.
>           // apply user deserializers.
>           // TODO: write records that can't be deserialized to a "dead-letter" additional output.
>           KafkaRecord<K, V> record = new KafkaRecord<K, V>(
>               rawRecord.topic(),
>               rawRecord.partition(),
>               rawRecord.offset(),
>               consumerSpEL.getRecordTimestamp(rawRecord),
>               keyDeserializerInstance.deserialize(rawRecord.topic(), rawRecord.key()),
>               valueDeserializerInstance.deserialize(rawRecord.topic(), rawRecord.value()));
>           curTimestamp = (source.spec.getTimestampFn() == null)
>               ? Instant.now() : source.spec.getTimestampFn().apply(record);
>           curRecord = record;
> {noformat}
> There's a race condition between these two blocks of code which is exposed at the very least in the FlinkRunner, which calls getWatermark() periodically from a timer.
> The symptom of the race condition is a stack trace that looks like this (SDK 2.0.0):
> {noformat}
> Caused by: org.apache.flink.runtime.client.JobExecutionException: Job execution failed.
> 	at org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$7.apply$mcV$sp(JobManager.scala:910)
> 	at org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$7.apply(JobManager.scala:853)
> 	at org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$7.apply(JobManager.scala:853)
> 	at scala.concurrent.impl.Future$PromiseCompletingRunnable.liftedTree1$1(Future.scala:24)
> 	at scala.concurrent.impl.Future$PromiseCompletingRunnable.run(Future.scala:24)
> 	at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:40)
> 	at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:397)
> 	at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
> 	at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
> 	at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
> 	at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
> Caused by: TimerException{java.lang.NullPointerException}
> 	at org.apache.flink.streaming.runtime.tasks.SystemProcessingTimeService$TriggerTask.run(SystemProcessingTimeService.java:220)
> 	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> 	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> 	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
> 	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> 	at java.lang.Thread.run(Thread.java:745)
> Caused by: java.lang.NullPointerException
> 	at org.apache.beam.sdk.io.kafka.KafkaIO$Read$1.apply(KafkaIO.java:568)
> 	at org.apache.beam.sdk.io.kafka.KafkaIO$Read$1.apply(KafkaIO.java:565)
> 	at org.apache.beam.sdk.io.kafka.KafkaIO$UnboundedKafkaReader.getWatermark(KafkaIO.java:1210)
> 	at org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSourceWrapper.onProcessingTime(UnboundedSourceWrapper.java:431)
> 	at org.apache.flink.streaming.runtime.tasks.SystemProcessingTimeService$TriggerTask.run(SystemProcessingTimeService.java:218)
> 	... 7 more
> {noformat}
> Based on inspecting the code, what is probably happening is that while advance() is executing:
> * Flink runner calls getWatermark()
> * getWatermark evaluates (curRecord == null) and it is false
> * advance() proceeds to set curRecord = null
> * The flink runner thread calls getWatermarkFn().apply(curRecord) which passes a null record into the custom watermark function
> * If that watermark function had been set with withWatermarkFn() (as suggested in the javadoc), then it's using the closure created in unwrapKafkaAndThen()
> * That calls record.getKV() and we get the NullPointerException



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)