[ 
https://issues.apache.org/jira/browse/BEAM-11028?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17209661#comment-17209661
 ] 

Luke Cwik commented on BEAM-11028:
----------------------------------

{noformat}
org.apache.beam.runners.core.SplittableParDoViaKeyedWorkItems$ProcessFn 
processElement
INFO: before: KV{null, null}
{noformat}

Earlier logging shows that we have set this from state and then set a timer and 
then failed to load the data:
{noformat}
INFO: Setting: 
//:713916102:element:ValueInGlobalWindow{value=UnboundedEventSource(0, 100000), 
pane=PaneInfo.NO_FIRING}
INFO: Setting: 
//:713916102:restriction:UnboundedSourceRestriction{source=UnboundedEventSource(33000,
 34000), checkpoint=GeneratorCheckpoint{numEvents=1000, 
wallclockBaseTime=1600734382646}, watermark=294247-01-10T04:00:54.775Z}
INFO: Reading: 
//:713916102:element:ValueInGlobalWindow{value=UnboundedEventSource(0, 100000), 
pane=PaneInfo.NO_FIRING}
INFO: Reading: 
//:713916102:restriction:UnboundedSourceRestriction{source=UnboundedEventSource(33000,
 34000), checkpoint=GeneratorCheckpoint{numEvents=1000, 
wallclockBaseTime=1600734382646}, watermark=294247-01-10T04:00:54.775Z}
INFO: Setting: //:713916102:watermarkEstimatorState:294247-01-10T04:00:54.775Z
INFO: Setting timer: 713916102 1:1600734383019// at 1600734383019 with output 
time 1600734383019
INFO: Reading: //:713916102:element:null
INFO: Reading: //:713916102:restriction:null
INFO: Reading: //:713916102:watermarkEstimatorState:null
{noformat}
for byte[] key with hash 713916102 in the global window namespace.

Note that in a rerun the keys will be different since they are UUIDs but the 
pattern is the same.

To rerun:
checkout https://github.com/lukecwik/incubator-beam/tree/beam10670.2 for 
additional log output
setup gradle task:
with task:
{noformat}
:sdks:java:testing:nexmark:run
{noformat}
with args:
{noformat}
-Pnexmark.runner=":runners:flink:1.11"
-Pnexmark.args="     --runner=FlinkRunner     --suite=SMOKE     
--streamTimeout=60     --streaming=true     --manageResources=false     
--monitorJobs=true     --flinkMaster=[local] --numEvents=100000 --query=4 
--checkpointingInterval=3000 --shutdownSourcesAfterIdleMs=60000"
{noformat}

Note, you may need to increase your console size (Editor>General>Console) to 
capture enough output before it fails. I usually add a breakpoint for the 
UserCodeException constructor to pause the program and inspect the state at the 
point in time.

> 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
>             Fix For: 2.25.0
>
>
> 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)
>   



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

Reply via email to