[
https://issues.apache.org/jira/browse/BEAM-5250?focusedWorklogId=140999&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-140999
]
ASF GitHub Bot logged work on BEAM-5250:
----------------------------------------
Author: ASF GitHub Bot
Created on: 04/Sep/18 18:36
Start Date: 04/Sep/18 18:36
Worklog Time Spent: 10m
Work Description: lukecwik closed pull request #6315: [BEAM-5250] Fix
GlobalWindow's max timestamp in Python SDK
URL: https://github.com/apache/beam/pull/6315
This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:
As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):
diff --git a/sdks/python/apache_beam/io/iobase.py
b/sdks/python/apache_beam/io/iobase.py
index 459c3266d41..425137c8634 100644
--- a/sdks/python/apache_beam/io/iobase.py
+++ b/sdks/python/apache_beam/io/iobase.py
@@ -1034,7 +1034,8 @@ def process(self, element, init_result):
def finish_bundle(self):
if self.writer is not None:
- yield WindowedValue(self.writer.close(), window.MAX_TIMESTAMP,
+ yield WindowedValue(self.writer.close(),
+ window.GlobalWindow().max_timestamp(),
[window.GlobalWindow()])
diff --git a/sdks/python/apache_beam/transforms/window.py
b/sdks/python/apache_beam/transforms/window.py
index 067227bb3f8..e70c8ef4053 100644
--- a/sdks/python/apache_beam/transforms/window.py
+++ b/sdks/python/apache_beam/transforms/window.py
@@ -295,6 +295,11 @@ def __lt__(self, other):
class GlobalWindow(BoundedWindow):
"""The default window into which all data is placed (via GlobalWindows)."""
_instance = None
+ # The maximum timestamp for global windows is MAX_TIMESTAMP - 1 day.
+ # This is due to timers triggering when the watermark passes the trigger
+ # time, which is only possible for timestamps < MAX_TIMESTAMP.
+ # See also GlobalWindow in the Java SDK.
+ _END_OF_GLOBAL_WINDOW = MAX_TIMESTAMP - (24 * 60 * 60)
def __new__(cls):
if cls._instance is None:
@@ -302,7 +307,7 @@ def __new__(cls):
return cls._instance
def __init__(self):
- super(GlobalWindow, self).__init__(MAX_TIMESTAMP)
+ super(GlobalWindow, self).__init__(GlobalWindow._END_OF_GLOBAL_WINDOW)
self.start = MIN_TIMESTAMP
def __repr__(self):
diff --git a/sdks/python/apache_beam/transforms/window_test.py
b/sdks/python/apache_beam/transforms/window_test.py
index 77ab47e3dd8..aa575b15c64 100644
--- a/sdks/python/apache_beam/transforms/window_test.py
+++ b/sdks/python/apache_beam/transforms/window_test.py
@@ -80,6 +80,7 @@ def test_global_window(self):
IntervalWindow(MIN_TIMESTAMP, MAX_TIMESTAMP))
self.assertNotEqual(IntervalWindow(MIN_TIMESTAMP, MAX_TIMESTAMP),
GlobalWindow())
+ self.assertTrue(GlobalWindow().max_timestamp() < MAX_TIMESTAMP)
def test_fixed_windows(self):
# Test windows with offset: 2, 7, 12, 17, ...
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 140999)
Time Spent: 1h 40m (was: 1.5h)
> Python Wordcount fails with Flink portable streaming
> ----------------------------------------------------
>
> Key: BEAM-5250
> URL: https://issues.apache.org/jira/browse/BEAM-5250
> Project: Beam
> Issue Type: Improvement
> Components: runner-flink
> Reporter: Thomas Weise
> Assignee: Maximilian Michels
> Priority: Major
> Labels: portability
> Time Spent: 1h 40m
> Remaining Estimate: 0h
>
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)