Thanks a lot Luke..

Regards
Mohil

On Tue, Aug 4, 2020 at 12:01 PM Luke Cwik <lc...@google.com> wrote:

> BEAM-6855 is still open and I updated it linking to this thread that a
> user is still being impacted.
>
> On Tue, Aug 4, 2020 at 10:20 AM Mohil Khare <mo...@prosimo.io> wrote:
>
>> yeah .. looks like a bug still exists.
>>
>> So how does this work ? Shall I open a new Jira ?
>>
>> Thanks and regards
>> Mohil
>>
>> On Thu, Jul 30, 2020 at 10:39 PM Reuven Lax <re...@google.com> wrote:
>>
>>> I believe that the person trying to fix BEAM-6855 was unable to
>>> reproduce it in test, and therefore assumed that the bug was fixed. However
>>> it appears that the bug still exists.
>>>
>>> On Wed, Jul 29, 2020 at 10:36 AM Kenneth Knowles <k...@apache.org>
>>> wrote:
>>>
>>>> Hi Mohil,
>>>>
>>>> It helps also to tell us what version of Beam you are using and some
>>>> more details. This looks related to
>>>> https://issues.apache.org/jira/browse/BEAM-6855 which claims to be
>>>> resolved in 2.17.0
>>>>
>>>> Kenn
>>>>
>>>> On Mon, Jul 27, 2020 at 11:47 PM Mohil Khare <mo...@prosimo.io> wrote:
>>>>
>>>>> Hello all,
>>>>>
>>>>> I think I found the reason for the issue.  Since the exception was
>>>>> thrown by StreamingSideInputDoFnRunner.java, I realize that I recently
>>>>> added side input to one of my ParDo that does stateful transformations.
>>>>> It looks like there is some issue when you add side input (My side
>>>>> input was coming via Global window to ParDo in a Fixed Window) to stateful
>>>>> DoFn.
>>>>>
>>>>> As a work around, instead of adding side input to stateful ParDo, I
>>>>> introduced another ParDo  that enriches streaming data with side input
>>>>> before flowing into stateful DoFn. That seems to have fixed the problem.
>>>>>
>>>>>
>>>>> Thanks and regards
>>>>> Mohil
>>>>>
>>>>>
>>>>>
>>>>> On Mon, Jul 27, 2020 at 10:50 AM Mohil Khare <mo...@prosimo.io> wrote:
>>>>>
>>>>>> Hello All,
>>>>>>
>>>>>> Any idea how to debug this and find out which stage, which DoFn or
>>>>>> which side input is causing the problem?
>>>>>> Do I need to override OnTimer with every DoFn to avoid this problem?
>>>>>> I thought that some uncaught exceptions were causing this and added
>>>>>> various checks and exception handling in all DoFn and still seeing this
>>>>>> issue.
>>>>>> It has been driving me nuts. And now forget DRAIN, it happens during
>>>>>> normal functioning as well. Any help would be appreciated.
>>>>>>
>>>>>> java.lang.UnsupportedOperationException: Attempt to deliver a timer
>>>>>> to a DoFn, but timers are not supported in Dataflow.
>>>>>>
>>>>>>    1.
>>>>>>       1. at org.apache.beam.runners.dataflow.worker.
>>>>>>       StreamingSideInputDoFnRunner.onTimer (
>>>>>>       StreamingSideInputDoFnRunner.java:86
>>>>>>       
>>>>>> <https://console.cloud.google.com/debug/fromlog?appModule=Dataflow%20Jobs&appVersion=2020-07-26_20_35_51-7756840794032391970&file=org%2Fapache.beam.runners.dataflow.worker%2FStreamingSideInputDoFnRunner.java&line=86&project=prosimo-test>
>>>>>>       )
>>>>>>       2. at org.apache.beam.runners.dataflow.worker.
>>>>>>       SimpleParDoFn.processUserTimer (SimpleParDoFn.java:360
>>>>>>       
>>>>>> <https://console.cloud.google.com/debug/fromlog?appModule=Dataflow%20Jobs&appVersion=2020-07-26_20_35_51-7756840794032391970&file=org%2Fapache.beam.runners.dataflow.worker%2FSimpleParDoFn.java&line=360&project=prosimo-test>
>>>>>>       )
>>>>>>       3. at org.apache.beam.runners.dataflow.worker.
>>>>>>       SimpleParDoFn.access$600 (SimpleParDoFn.java:73
>>>>>>       
>>>>>> <https://console.cloud.google.com/debug/fromlog?appModule=Dataflow%20Jobs&appVersion=2020-07-26_20_35_51-7756840794032391970&file=org%2Fapache.beam.runners.dataflow.worker%2FSimpleParDoFn.java&line=73&project=prosimo-test>
>>>>>>       )
>>>>>>       4. at org.apache.beam.runners.dataflow.worker.
>>>>>>       SimpleParDoFn$TimerType$1.processTimer (SimpleParDoFn.java:444
>>>>>>       
>>>>>> <https://console.cloud.google.com/debug/fromlog?appModule=Dataflow%20Jobs&appVersion=2020-07-26_20_35_51-7756840794032391970&file=org%2Fapache.beam.runners.dataflow.worker%2FSimpleParDoFn.java&line=444&project=prosimo-test>
>>>>>>       )
>>>>>>       5. at org.apache.beam.runners.dataflow.worker.
>>>>>>       SimpleParDoFn.processTimers (SimpleParDoFn.java:473
>>>>>>       
>>>>>> <https://console.cloud.google.com/debug/fromlog?appModule=Dataflow%20Jobs&appVersion=2020-07-26_20_35_51-7756840794032391970&file=org%2Fapache.beam.runners.dataflow.worker%2FSimpleParDoFn.java&line=473&project=prosimo-test>
>>>>>>       )
>>>>>>       6. at org.apache.beam.runners.dataflow.worker.
>>>>>>       SimpleParDoFn.processTimers (SimpleParDoFn.java:353
>>>>>>       
>>>>>> <https://console.cloud.google.com/debug/fromlog?appModule=Dataflow%20Jobs&appVersion=2020-07-26_20_35_51-7756840794032391970&file=org%2Fapache.beam.runners.dataflow.worker%2FSimpleParDoFn.java&line=353&project=prosimo-test>
>>>>>>       )
>>>>>>       7. at
>>>>>>       org.apache.beam.runners.dataflow.worker.util.common.worker.
>>>>>>       ParDoOperation.finish (ParDoOperation.java:52
>>>>>>       
>>>>>> <https://console.cloud.google.com/debug/fromlog?appModule=Dataflow%20Jobs&appVersion=2020-07-26_20_35_51-7756840794032391970&file=org%2Fapache.beam.runners.dataflow.worker.util.common.worker%2FParDoOperation.java&line=52&project=prosimo-test>
>>>>>>       )
>>>>>>       8. at
>>>>>>       org.apache.beam.runners.dataflow.worker.util.common.worker.
>>>>>>       MapTaskExecutor.execute (MapTaskExecutor.java:85
>>>>>>       
>>>>>> <https://console.cloud.google.com/debug/fromlog?appModule=Dataflow%20Jobs&appVersion=2020-07-26_20_35_51-7756840794032391970&file=org%2Fapache.beam.runners.dataflow.worker.util.common.worker%2FMapTaskExecutor.java&line=85&project=prosimo-test>
>>>>>>       )
>>>>>>       9. at org.apache.beam.runners.dataflow.worker.
>>>>>>       StreamingDataflowWorker.process (
>>>>>>       StreamingDataflowWorker.java:1350
>>>>>>       
>>>>>> <https://console.cloud.google.com/debug/fromlog?appModule=Dataflow%20Jobs&appVersion=2020-07-26_20_35_51-7756840794032391970&file=org%2Fapache.beam.runners.dataflow.worker%2FStreamingDataflowWorker.java&line=1350&project=prosimo-test>
>>>>>>       )
>>>>>>       10. at org.apache.beam.runners.dataflow.worker.
>>>>>>       StreamingDataflowWorker.access$1100 (
>>>>>>       StreamingDataflowWorker.java:152
>>>>>>       
>>>>>> <https://console.cloud.google.com/debug/fromlog?appModule=Dataflow%20Jobs&appVersion=2020-07-26_20_35_51-7756840794032391970&file=org%2Fapache.beam.runners.dataflow.worker%2FStreamingDataflowWorker.java&line=152&project=prosimo-test>
>>>>>>       )
>>>>>>    2.
>>>>>>       1. at org.apache.beam.runners.dataflow.worker.
>>>>>>       StreamingDataflowWorker$7.run (
>>>>>>       StreamingDataflowWorker.java:1073)
>>>>>>       2. at java.util.concurrent.ThreadPoolExecutor.runWorker (
>>>>>>       ThreadPoolExecutor.java:1149)
>>>>>>       3. at java.util.concurrent.ThreadPoolExecutor$Worker.run (
>>>>>>       ThreadPoolExecutor.java:624)
>>>>>>       4. at java.lang.Thread.run (Thread.java:748)
>>>>>>
>>>>>> Thanks
>>>>>> Mohil
>>>>>>
>>>>>>
>>>>>> On Sun, Jul 26, 2020 at 1:50 PM Mohil Khare <mo...@prosimo.io> wrote:
>>>>>>
>>>>>>> and it seems be due  to TimerType User
>>>>>>>
>>>>>>> Thanks
>>>>>>> Mohil
>>>>>>>
>>>>>>> On Sun, Jul 26, 2020 at 1:42 PM Mohil Khare <mo...@prosimo.io>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Hello,
>>>>>>>>
>>>>>>>> I was looking at source code of
>>>>>>>> https://github.com/apache/beam/blob/master/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunner.java
>>>>>>>> ,
>>>>>>>>
>>>>>>>> It seems the default implementation of OnTimer is to throw (Attempt
>>>>>>>> to deliver a timer to a DoFn, but timers are not supported in 
>>>>>>>> Dataflow).
>>>>>>>>
>>>>>>>> Do you know under what circumstances, My code might be throwing
>>>>>>>> this ?  Not sure if its some issue in 2.19 which might have  been 
>>>>>>>> fixed now
>>>>>>>> with 2.22
>>>>>>>>
>>>>>>>> Thanks and Regards
>>>>>>>> Mohil
>>>>>>>>
>>>>>>>>
>>>>>>>> On Fri, Jul 24, 2020 at 5:21 PM Mohil Khare <mo...@prosimo.io>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Actually NOT JUST DRAIN, seeing it during regular RUN as well.
>>>>>>>>> It's getting flooded with this exception
>>>>>>>>>
>>>>>>>>> Attempt to deliver a timer to a DoFn, but timers are not supported
>>>>>>>>> in Dataflow.
>>>>>>>>> The changes that I did are the following:
>>>>>>>>> 1. Read one set of logs from one of the kafka topics and create
>>>>>>>>> KV(ID, log1)
>>>>>>>>> 2. Read 2nd set of logs from another kafka topic and create KV(ID,
>>>>>>>>> log2)
>>>>>>>>> 3. Put above in session window
>>>>>>>>> 4. CoGroupByKey both logs
>>>>>>>>>
>>>>>>>>> Apart from aforementioned exceptions, I am also seeing the
>>>>>>>>> following in worker logs. What I have noticed is that it happens when 
>>>>>>>>> the
>>>>>>>>> pipeline is silent for a while. i.e. no new logs to be read from 
>>>>>>>>> Kafka (Not
>>>>>>>>> sure if it is the actual reason).
>>>>>>>>>
>>>>>>>>> 2020-07-24 17:06:43.532 PDT
>>>>>>>>> Execution of work for P188 for key
>>>>>>>>> cloud@prosimo.ioHe8fc8079-c844-11ea-a6d5-dabe1eb9c630 failed.
>>>>>>>>> Will retry locally.
>>>>>>>>>
>>>>>>>>> <https://console.cloud.google.com/logs/viewer?advancedFilter=insertId%3D%224729149996009968456:27267:0:42419165%22%20resource.type%3D%22dataflow_step%22%20resource.labels.job_id%3D%222020-07-24_16_23_20-4320159532106629306%22%20(logName%3D%22projects%2Fprosimo-test%2Flogs%2Fdataflow.googleapis.com%252Fworker%22)%20severity%3E%3DERROR%0A(timestamp%3E%222020-07-25T00:06:33.224Z%22%20OR%20(insertId%3E%224729149996009968456:27267:0:42264272%22%20AND%20timestamp%3D%222020-07-25T00:06:33.224Z%22))&expandAll=true&interval=JUMP_TO_TIME&dateRangeStart=2020-07-25T00:06:43.532Z&dateRangeEnd=2020-07-25T00:06:43.532Z&project=prosimo-test>
>>>>>>>>> 2020-07-24 17:06:53.863 PDT
>>>>>>>>> Uncaught exception:
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Fri, Jul 24, 2020 at 2:59 PM Mohil Khare <mo...@prosimo.io>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Hello,
>>>>>>>>>>
>>>>>>>>>> I am on java sdk 2.19 and using dataflow for beam job. I use
>>>>>>>>>> Timers for my stateful transformations, but recently I started 
>>>>>>>>>> seeing the
>>>>>>>>>> following exception on DRAINING a job. It used to work fine and not 
>>>>>>>>>> sure
>>>>>>>>>> what changed.
>>>>>>>>>>
>>>>>>>>>> *java.lang.UnsupportedOperationException:*
>>>>>>>>>>
>>>>>>>>> *Attempt to deliver a timer to a DoFn, but timers are not
>>>>>>>>> supported in Dataflow. *
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>    1.
>>>>>>>>>>       1. atorg.apache.beam.runners.dataflow.worker.
>>>>>>>>>>       StreamingSideInputDoFnRunner.onTimer (
>>>>>>>>>>       StreamingSideInputDoFnRunner.java:86
>>>>>>>>>>       
>>>>>>>>>> <https://console.cloud.google.com/debug/fromlog?appModule=Dataflow%20Jobs&appVersion=2020-07-24_14_14_42-5619042994908733980&file=org%2Fapache.beam.runners.dataflow.worker%2FStreamingSideInputDoFnRunner.java&line=86&project=prosimo-test>
>>>>>>>>>>       )
>>>>>>>>>>       2. atorg.apache.beam.runners.dataflow.worker.
>>>>>>>>>>       SimpleParDoFn.processUserTimer (SimpleParDoFn.java:360
>>>>>>>>>>       
>>>>>>>>>> <https://console.cloud.google.com/debug/fromlog?appModule=Dataflow%20Jobs&appVersion=2020-07-24_14_14_42-5619042994908733980&file=org%2Fapache.beam.runners.dataflow.worker%2FSimpleParDoFn.java&line=360&project=prosimo-test>
>>>>>>>>>>       )
>>>>>>>>>>       3. atorg.apache.beam.runners.dataflow.worker.
>>>>>>>>>>       SimpleParDoFn.access$600 (SimpleParDoFn.java:73
>>>>>>>>>>       
>>>>>>>>>> <https://console.cloud.google.com/debug/fromlog?appModule=Dataflow%20Jobs&appVersion=2020-07-24_14_14_42-5619042994908733980&file=org%2Fapache.beam.runners.dataflow.worker%2FSimpleParDoFn.java&line=73&project=prosimo-test>
>>>>>>>>>>       )
>>>>>>>>>>       4. atorg.apache.beam.runners.dataflow.worker.
>>>>>>>>>>       SimpleParDoFn$TimerType$1.processTimer (
>>>>>>>>>>       SimpleParDoFn.java:444
>>>>>>>>>>       
>>>>>>>>>> <https://console.cloud.google.com/debug/fromlog?appModule=Dataflow%20Jobs&appVersion=2020-07-24_14_14_42-5619042994908733980&file=org%2Fapache.beam.runners.dataflow.worker%2FSimpleParDoFn.java&line=444&project=prosimo-test>
>>>>>>>>>>       )
>>>>>>>>>>       5. atorg.apache.beam.runners.dataflow.worker.
>>>>>>>>>>       SimpleParDoFn.processTimers (SimpleParDoFn.java:473
>>>>>>>>>>       
>>>>>>>>>> <https://console.cloud.google.com/debug/fromlog?appModule=Dataflow%20Jobs&appVersion=2020-07-24_14_14_42-5619042994908733980&file=org%2Fapache.beam.runners.dataflow.worker%2FSimpleParDoFn.java&line=473&project=prosimo-test>
>>>>>>>>>>       )
>>>>>>>>>>       6. atorg.apache.beam.runners.dataflow.worker.
>>>>>>>>>>       SimpleParDoFn.processTimers (SimpleParDoFn.java:353
>>>>>>>>>>       
>>>>>>>>>> <https://console.cloud.google.com/debug/fromlog?appModule=Dataflow%20Jobs&appVersion=2020-07-24_14_14_42-5619042994908733980&file=org%2Fapache.beam.runners.dataflow.worker%2FSimpleParDoFn.java&line=353&project=prosimo-test>
>>>>>>>>>>       )
>>>>>>>>>>       7. at
>>>>>>>>>>       org.apache.beam.runners.dataflow.worker.util.common.worker.
>>>>>>>>>>       ParDoOperation.finish (ParDoOperation.java:52
>>>>>>>>>>       
>>>>>>>>>> <https://console.cloud.google.com/debug/fromlog?appModule=Dataflow%20Jobs&appVersion=2020-07-24_14_14_42-5619042994908733980&file=org%2Fapache.beam.runners.dataflow.worker.util.common.worker%2FParDoOperation.java&line=52&project=prosimo-test>
>>>>>>>>>>       )
>>>>>>>>>>       8. at
>>>>>>>>>>       org.apache.beam.runners.dataflow.worker.util.common.worker.
>>>>>>>>>>       MapTaskExecutor.execute (MapTaskExecutor.java:85
>>>>>>>>>>       
>>>>>>>>>> <https://console.cloud.google.com/debug/fromlog?appModule=Dataflow%20Jobs&appVersion=2020-07-24_14_14_42-5619042994908733980&file=org%2Fapache.beam.runners.dataflow.worker.util.common.worker%2FMapTaskExecutor.java&line=85&project=prosimo-test>
>>>>>>>>>>       )
>>>>>>>>>>       9. atorg.apache.beam.runners.dataflow.worker.
>>>>>>>>>>       StreamingDataflowWorker.process (
>>>>>>>>>>       StreamingDataflowWorker.java:1350
>>>>>>>>>>       
>>>>>>>>>> <https://console.cloud.google.com/debug/fromlog?appModule=Dataflow%20Jobs&appVersion=2020-07-24_14_14_42-5619042994908733980&file=org%2Fapache.beam.runners.dataflow.worker%2FStreamingDataflowWorker.java&line=1350&project=prosimo-test>
>>>>>>>>>>       )
>>>>>>>>>>       10. atorg.apache.beam.runners.dataflow.worker.
>>>>>>>>>>       StreamingDataflowWorker.access$1100 (
>>>>>>>>>>       StreamingDataflowWorker.java:152
>>>>>>>>>>       
>>>>>>>>>> <https://console.cloud.google.com/debug/fromlog?appModule=Dataflow%20Jobs&appVersion=2020-07-24_14_14_42-5619042994908733980&file=org%2Fapache.beam.runners.dataflow.worker%2FStreamingDataflowWorker.java&line=152&project=prosimo-test>
>>>>>>>>>>       )
>>>>>>>>>>    2. Sometime back I opened jira for an issue which is related
>>>>>>>>>>    to this while doing DRAIN:
>>>>>>>>>>    https://issues.apache.org/jira/browse/BEAM-10053 (Looks like
>>>>>>>>>>    no one has taken a stab on this Jira)
>>>>>>>>>>    3. Not sure if the reason is same and due to multiple side
>>>>>>>>>>    inputs that i use in my PTransforms.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Any help would be appreciated.
>>>>>>>>>>
>>>>>>>>>> Thanks and Regards
>>>>>>>>>> Mohil
>>>>>>>>>>
>>>>>>>>>>

Reply via email to