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 B4F37200CE0 for ; Fri, 25 Aug 2017 15:24:07 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id B346416BA39; Fri, 25 Aug 2017 13:24:07 +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 9E0D616BA35 for ; Fri, 25 Aug 2017 15:24:06 +0200 (CEST) Received: (qmail 80046 invoked by uid 500); 25 Aug 2017 13:24:05 -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 80037 invoked by uid 99); 25 Aug 2017 13:24:05 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd3-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 25 Aug 2017 13:24:05 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd3-us-west.apache.org (ASF Mail Server at spamd3-us-west.apache.org) with ESMTP id 63220180C37 for ; Fri, 25 Aug 2017 13:24:05 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd3-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-eu.apache.org ([10.40.0.8]) by localhost (spamd3-us-west.apache.org [10.40.0.10]) (amavisd-new, port 10024) with ESMTP id 11BeL7qoI2kp for ; Fri, 25 Aug 2017 13:24:03 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTP id 884B15FE5F for ; Fri, 25 Aug 2017 13:24:01 +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 894D2E099F for ; Fri, 25 Aug 2017 13:24:00 +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 38B9625382 for ; Fri, 25 Aug 2017 13:24:00 +0000 (UTC) Date: Fri, 25 Aug 2017 13:24:00 +0000 (UTC) From: "Daniel Harper (JIRA)" To: commits@beam.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Comment Edited] (BEAM-2807) NullPointerException during checkpoint on FlinkRunner MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Fri, 25 Aug 2017 13:24:07 -0000 [ https://issues.apache.org/jira/browse/BEAM-2807?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16141601#comment-16141601 ] Daniel Harper edited comment on BEAM-2807 at 8/25/17 1:23 PM: -------------------------------------------------------------- {{getCanonicalName}} will return null on anonymous classes, so the {{AtomicCoder}} returned by [Count.java#170|https://github.com/apache/beam/blob/2040e2bd4203f81ea63966110e4eef3a1ff72393/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java#L170] will always return null when you call this method https://docs.oracle.com/javase/8/docs/api/java/lang/Class.html#getCanonicalName {quote} Returns the canonical name of the underlying class as defined by the Java Language Specification. Returns null if the underlying class does not have a canonical name (i.e., if it is a local or anonymous class or an array whose component type does not have a canonical name). {quote} was (Author: djharper): {{getCanonicalName}} will return null on anonymous classes, so the {{AtomicCoder}} returned by will always return null when you call this method https://docs.oracle.com/javase/8/docs/api/java/lang/Class.html#getCanonicalName {quote} Returns the canonical name of the underlying class as defined by the Java Language Specification. Returns null if the underlying class does not have a canonical name (i.e., if it is a local or anonymous class or an array whose component type does not have a canonical name). {quote} > NullPointerException during checkpoint on FlinkRunner > ----------------------------------------------------- > > Key: BEAM-2807 > URL: https://issues.apache.org/jira/browse/BEAM-2807 > Project: Beam > Issue Type: Bug > Components: beam-model > Affects Versions: 2.1.0 > Reporter: Daniel Harper > Assignee: Kenneth Knowles > Priority: Blocker > > *Beam version:* 2.1.0 > *Runner:* FlinkRunner > We're seeing the following exception when checkpointing, which is causing our job to restart > {code} > 2017-08-25 09:42:17,658 INFO org.apache.flink.runtime.taskmanager.Task - Combine.perKey(Count) -> ParMultiDo(ToConcurrentStreamResult) -> ParMultiDo(Anonymous) -> ParMultiDo(ApplyShardingKey) -> ToKeyedWorkItem (7/32) (f00a31b722a31030f18d83ac613de21d) switched from RUNNING to FAILED. > AsynchronousException{java.lang.Exception: Could not materialize checkpoint 2 for operator Combine.perKey(Count) -> ParMultiDo(ToConcurrentStreamResult) -> ParMultiDo(Anonymous) -> ParMultiDo(ApplyShardingKey) -> ToKeyedWorkItem (7/32).} > at org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:966) > at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at java.lang.Thread.run(Thread.java:748) > Caused by: java.lang.Exception: Could not materialize checkpoint 2 for operator Combine.perKey(Count) -> ParMultiDo(ToConcurrentStreamResult) -> ParMultiDo(Anonymous) -> ParMultiDo(ApplyShardingKey) -> ToKeyedWorkItem (7/32). > ... 6 more > Caused by: java.util.concurrent.ExecutionException: java.lang.NullPointerException > at java.util.concurrent.FutureTask.report(FutureTask.java:122) > at java.util.concurrent.FutureTask.get(FutureTask.java:192) > at org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:43) > at org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:893) > ... 5 more > Suppressed: java.lang.Exception: Could not properly cancel managed keyed state future. > at org.apache.flink.streaming.api.operators.OperatorSnapshotResult.cancel(OperatorSnapshotResult.java:90) > at org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.cleanup(StreamTask.java:1018) > at org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:957) > ... 5 more > Caused by: java.util.concurrent.ExecutionException: java.lang.NullPointerException > at java.util.concurrent.FutureTask.report(FutureTask.java:122) > at java.util.concurrent.FutureTask.get(FutureTask.java:192) > at org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:43) > at org.apache.flink.runtime.state.StateUtil.discardStateFuture(StateUtil.java:85) > at org.apache.flink.streaming.api.operators.OperatorSnapshotResult.cancel(OperatorSnapshotResult.java:88) > ... 7 more > Caused by: java.lang.NullPointerException > at java.io.DataOutputStream.writeUTF(DataOutputStream.java:347) > at java.io.DataOutputStream.writeUTF(DataOutputStream.java:323) > at org.apache.beam.runners.flink.translation.types.CoderTypeSerializer$CoderTypeSerializerConfigSnapshot.write(CoderTypeSerializer.java:189) > at org.apache.flink.api.common.typeutils.TypeSerializerSerializationUtil$TypeSerializerConfigSnapshotSerializationProxy.write(TypeSerializerSerializationUtil.java:413) > at org.apache.flink.api.common.typeutils.TypeSerializerSerializationUtil.writeSerializerConfigSnapshot(TypeSerializerSerializationUtil.java:229) > at org.apache.flink.api.common.typeutils.TypeSerializerSerializationUtil.writeSerializersAndConfigsWithResilience(TypeSerializerSerializationUtil.java:151) > at org.apache.flink.runtime.state.KeyedBackendStateMetaInfoSnapshotReaderWriters$KeyedBackendStateMetaInfoWriterV3.writeStateMetaInfo(KeyedBackendStateMetaInfoSnapshotReaderWriters.java:107) > at org.apache.flink.runtime.state.KeyedBackendSerializationProxy.write(KeyedBackendSerializationProxy.java:104) > at org.apache.flink.runtime.state.heap.HeapKeyedStateBackend$1.performOperation(HeapKeyedStateBackend.java:293) > at org.apache.flink.runtime.state.heap.HeapKeyedStateBackend$1.performOperation(HeapKeyedStateBackend.java:286) > at org.apache.flink.runtime.io.async.AbstractAsyncIOCallable.call(AbstractAsyncIOCallable.java:72) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at org.apache.flink.runtime.state.heap.HeapKeyedStateBackend.snapshot(HeapKeyedStateBackend.java:329) > at org.apache.flink.streaming.api.operators.AbstractStreamOperator.snapshotState(AbstractStreamOperator.java:397) > at org.apache.flink.streaming.runtime.tasks.StreamTask$CheckpointingOperation.checkpointStreamOperator(StreamTask.java:1157) > at org.apache.flink.streaming.runtime.tasks.StreamTask$CheckpointingOperation.executeCheckpointing(StreamTask.java:1089) > at org.apache.flink.streaming.runtime.tasks.StreamTask.checkpointState(StreamTask.java:653) > at org.apache.flink.streaming.runtime.tasks.StreamTask.performCheckpoint(StreamTask.java:589) > at org.apache.flink.streaming.runtime.tasks.StreamTask.triggerCheckpointOnBarrier(StreamTask.java:542) > at org.apache.flink.streaming.runtime.io.BarrierTracker.notifyCheckpoint(BarrierTracker.java:263) > at org.apache.flink.streaming.runtime.io.BarrierTracker.processBarrier(BarrierTracker.java:178) > at org.apache.flink.streaming.runtime.io.BarrierTracker.getNextNonBlocked(BarrierTracker.java:97) > at org.apache.flink.streaming.runtime.io.StreamInputProcessor.processInput(StreamInputProcessor.java:213) > at org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.run(OneInputStreamTask.java:69) > at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:262) > at org.apache.flink.runtime.taskmanager.Task.run(Task.java:702) > ... 1 more > {code} > From debugging locally I've narrowed it down to here > {code} > Caused by: java.lang.NullPointerException > at java.io.DataOutputStream.writeUTF(DataOutputStream.java:347) > at java.io.DataOutputStream.writeUTF(DataOutputStream.java:323) > at org.apache.beam.runners.flink.translation.types.CoderTypeSerializer$CoderTypeSerializerConfigSnapshot.write(CoderTypeSerializer.java:189) > at org.apache.flink.api.common.typeutils.TypeSerializerSerializationUtil$TypeSerializerConfigSnapshotSerializationProxy.write(TypeSerializerSerializationUtil.java:413) > {code} > Specifically in [CoderTypeSerializer.java#189|https://github.com/apache/beam/blob/609016d700c84800cf942482fb7cd2ddaa420b00/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java#L189], when it calls {{DataOutputStream.writeUTF(String)}}, there is some logic in the {{writeUTF}} method that gets the string length. This is what is causing the NPE as the {{coderName}} field is null. > I think this stems from the [constructor|https://github.com/apache/beam/blob/609016d700c84800cf942482fb7cd2ddaa420b00/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java#L164] which sets the {{coderName}} by calling {{.getClass().getCanonicalName();}} on the {{coder}} that is passed into the constructor > On debugging I've noticed this returns {{null}} when calling {{.getClass().getCanonicalName();}} on an instance of [Count$CountFn|https://github.com/apache/beam/blob/2040e2bd4203f81ea63966110e4eef3a1ff72393/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Count.java#L134] -- This message was sent by Atlassian JIRA (v6.4.14#64029)