beam-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Wesley Tanaka (JIRA)" <j...@apache.org>
Subject [jira] [Created] (BEAM-2979) Race condition between KafkaIO.UnboundedKafkaReader.getWatermark() and KafkaIO.UnboundedKafkaReader.advance()
Date Fri, 22 Sep 2017 01:47:00 GMT
Wesley Tanaka created BEAM-2979:
-----------------------------------

             Summary: 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.1.0, 2.0.0
            Reporter: Wesley Tanaka
            Assignee: Kenneth Knowles


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:

{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)

Mime
View raw message