Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 3975C200D29 for ; Thu, 12 Oct 2017 03:14:08 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 37DB7160BE3; Thu, 12 Oct 2017 01:14:08 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 555A21609E5 for ; Thu, 12 Oct 2017 03:14:07 +0200 (CEST) Received: (qmail 84886 invoked by uid 500); 12 Oct 2017 01:14:06 -0000 Mailing-List: contact commits-help@beam.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@beam.apache.org Delivered-To: mailing list commits@beam.apache.org Received: (qmail 84877 invoked by uid 99); 12 Oct 2017 01:14:06 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd2-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 12 Oct 2017 01:14:06 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd2-us-west.apache.org (ASF Mail Server at spamd2-us-west.apache.org) with ESMTP id B1B5A1A1220 for ; Thu, 12 Oct 2017 01:14:05 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -99.202 X-Spam-Level: X-Spam-Status: No, score=-99.202 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, RP_MATCHES_RCVD=-0.001, SPF_PASS=-0.001, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id 5P6BT6OUIra4 for ; Thu, 12 Oct 2017 01:14:04 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTP id 135555F569 for ; Thu, 12 Oct 2017 01:14:04 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id C7FA4E0217 for ; Thu, 12 Oct 2017 01:14:02 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 4B9B321EE3 for ; Thu, 12 Oct 2017 01:14:00 +0000 (UTC) Date: Thu, 12 Oct 2017 01:14:00 +0000 (UTC) From: "Raghu Angadi (JIRA)" To: commits@beam.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (BEAM-2979) Race condition between KafkaIO.UnboundedKafkaReader.getWatermark() and KafkaIO.UnboundedKafkaReader.advance() MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Thu, 12 Oct 2017 01:14:08 -0000 [ https://issues.apache.org/jira/browse/BEAM-2979?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16201300#comment-16201300 ] Raghu Angadi commented on BEAM-2979: ------------------------------------ Thanks for the detailed bug report. Sent a fix in https://github.com/apache/beam/pull/3985. > 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 > > 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 record = new KafkaRecord( > 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)