You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@beam.apache.org by "Ismaël Mejía (Jira)" <ji...@apache.org> on 2020/10/06 22:15:00 UTC

[jira] [Updated] (BEAM-11028) NullPointerException when running Flink Nexmark tests on Streaming after switch to SDF based translation

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

Ismaël Mejía updated BEAM-11028:
--------------------------------
    Status: Open  (was: Triage Needed)

> NullPointerException when running Flink Nexmark tests on Streaming after switch to SDF based translation
> --------------------------------------------------------------------------------------------------------
>
>                 Key: BEAM-11028
>                 URL: https://issues.apache.org/jira/browse/BEAM-11028
>             Project: Beam
>          Issue Type: Test
>          Components: runner-core, runner-flink, testing-nexmark
>    Affects Versions: 2.25.0
>            Reporter: Ismaël Mejía
>            Assignee: Luke Cwik
>            Priority: P2
>
> When running Nexmark on Streaming mode with Flink locally via:
>  
> /gradlew :sdks:java:testing:nexmark:run \
>       -Pnexmark.runner=":runners:flink:1.10" \
>       -Pnexmark.args=" \
>           --runner=FlinkRunner \
>           --streaming=true \
>           --suite=SMOKE \
>           --manageResources=false \
>           --monitorJobs=true \
>           --enforceEncodability=true \
>           --enforceImmutability=true"
>  
> I see the following error and get no results (this works ok in Beam 2.24.0)
>  
> Caused by: org.apache.beam.sdk.util.UserCodeException: java.lang.NullPointerException
>         at org.apache.beam.sdk.util.UserCodeException.wrap(UserCodeException.java:36)
>         at org.apache.beam.sdk.io.Read$UnboundedSourceAsSDFWrapperFn$DoFnInvoker.invokeNewWatermarkEstimator(Unknown Source)
>         at org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems$ProcessFn.processElement(SplittableParDoViaKeyedWorkItems.java:439)
>         at org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems$ProcessFn$DoFnInvoker.invokeProcessElement(Unknown Source)
>         at org.apache.beam.runners.core.SimpleDoFnRunner.invokeProcessElement(SimpleDoFnRunner.java:227)
>         at org.apache.beam.runners.core.SimpleDoFnRunner.processElement(SimpleDoFnRunner.java:183)
>         at org.apache.beam.runners.flink.metrics.DoFnRunnerWithMetricsUpdate.processElement(DoFnRunnerWithMetricsUpdate.java:62)
>         at org.apache.beam.runners.flink.translation.wrappers.streaming.SplittableDoFnOperator.fireTimer(SplittableDoFnOperator.java:171)
>         at org.apache.beam.runners.flink.translation.wrappers.streaming.DoFnOperator$FlinkTimerInternals.processPendingProcessingTimeTimers(DoFnOperator.java:1317)
>         at org.apache.beam.runners.flink.translation.wrappers.streaming.DoFnOperator.close(DoFnOperator.java:575)
>         at org.apache.beam.runners.flink.translation.wrappers.streaming.SplittableDoFnOperator.close(SplittableDoFnOperator.java:179)
>         at org.apache.flink.streaming.runtime.tasks.StreamTask.closeAllOperators(StreamTask.java:618)
>         at org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$afterInvoke$1(StreamTask.java:498)
>         at org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$SynchronizedStreamTaskActionExecutor.runThrowing(StreamTaskActionExecutor.java:94)
>         at org.apache.flink.streaming.runtime.tasks.StreamTask.afterInvoke(StreamTask.java:496)
>         at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:477)
>         at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:708)
>         at org.apache.flink.runtime.taskmanager.Task.run(Task.java:533)
>         at java.lang.Thread.run(Thread.java:748)
> Caused by: java.lang.NullPointerException
>         at org.apache.beam.sdk.io.Read$UnboundedSourceAsSDFWrapperFn.ensureTimestampWithinBounds(Read.java:541)
>         at org.apache.beam.sdk.io.Read$UnboundedSourceAsSDFWrapperFn.newWatermarkEstimator(Read.java:552)
>  
> I just wanted to tell you in case you want to take a look, this is happening in current master and also in 2.25.0 branch so I suppose is related to the recent changes. I have not issued a JIRA yet
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)