[ 
https://issues.apache.org/jira/browse/BEAM-7972?focusedWorklogId=304653&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-304653
 ]

ASF GitHub Bot logged work on BEAM-7972:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 30/Aug/19 22:37
            Start Date: 30/Aug/19 22:37
    Worklog Time Spent: 10m 
      Work Description: angoenka commented on pull request #9334: [BEAM-7972] 
Always use Global window in reshuffle and then apply wind…
URL: https://github.com/apache/beam/pull/9334#discussion_r319692142
 
 

 ##########
 File path: sdks/python/apache_beam/transforms/util.py
 ##########
 @@ -612,29 +611,27 @@ def restore_timestamps(element):
             for (value, timestamp) in values]
 
     else:
-      # The linter is confused.
-      # hash(1) is used to force "runtime" selection of _IdentityWindowFn
-      # pylint: disable=abstract-class-instantiated
-      cls = hash(1) and _IdentityWindowFn
-      window_fn = cls(
-          windowing_saved.windowfn.get_window_coder())
-
-      def reify_timestamps(element, timestamp=DoFn.TimestampParam):
+      def reify_timestamps(element,
+                           timestamp=DoFn.TimestampParam,
+                           window=DoFn.WindowParam):
         key, value = element
-        return key, TimestampedValue(value, timestamp)
+        # Transport the window as part of the value and restore it later.
+        return key, windowed_value.WindowedValue(value, timestamp, [window])
 
-      def restore_timestamps(element, window=DoFn.WindowParam):
-        # Pass the current window since _IdentityWindowFn wouldn't know how
-        # to generate it.
+      def restore_timestamps(element):
         key, values = element
         return [
             windowed_value.WindowedValue(
-                (key, value.value), value.timestamp, [window])
+                (key, value.value), value.timestamp, value.windows)
             for value in values]
 
     ungrouped = pcoll | Map(reify_timestamps)
+
+    # TODO(BEAM-8104) Using global window as one of the standard window.
+    # This is to mitigate the Java Runner Harness limitation to
 
 Review comment:
   done
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 304653)
    Time Spent: 3h 50m  (was: 3h 40m)

> Portable Python Reshuffle does not work with windowed pcollection
> -----------------------------------------------------------------
>
>                 Key: BEAM-7972
>                 URL: https://issues.apache.org/jira/browse/BEAM-7972
>             Project: Beam
>          Issue Type: Bug
>          Components: sdk-py-core
>            Reporter: Ankur Goenka
>            Assignee: Ankur Goenka
>            Priority: Blocker
>             Fix For: 2.16.0
>
>          Time Spent: 3h 50m
>  Remaining Estimate: 0h
>
> Streaming pipeline gets stuck when using Reshuffle with windowed pcollection.
> The issue happen because of window function gets deserialized on java side 
> which is not possible and hence default to global window function and result 
> into window function mismatch later down the code.



--
This message was sent by Atlassian Jira
(v8.3.2#803003)

Reply via email to