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

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

                Author: ASF GitHub Bot
            Created on: 21/Nov/19 18:22
            Start Date: 21/Nov/19 18:22
    Worklog Time Spent: 10m 
      Work Description: robertwb commented on pull request #10081: [BEAM-8645] 
A test case for TimestampCombiner.
URL: https://github.com/apache/beam/pull/10081#discussion_r349244498
 
 

 ##########
 File path: sdks/python/apache_beam/transforms/combiners_test.py
 ##########
 @@ -480,6 +486,76 @@ def test_with_input_types_decorator_violation(self):
         pc = p | Create(l_3_tuple)
         _ = pc | beam.CombineGlobally(self.fn)
 
+#
+# Test cases for streaming.
+#
+@attr('ValidatesRunner')
+class TimestampCombinerTest(unittest.TestCase):
+
+  class RecordFn(beam.DoFn):
+    def process(self, elm=beam.DoFn.ElementParam, ts=beam.DoFn.TimestampParam):
+      yield (elm, ts)
+
+  @unittest.skip('BEAM-8657')
+  def test_combiner_earliest(self):
+    """Test TimestampCombiner with EARLIEST."""
+    options = PipelineOptions(streaming=True)
+    with TestPipeline(options=options) as p:
+      result = (p
+                | 'main TestStream' >> TestStream()
+                .add_elements([window.TimestampedValue(('k', 100), 2)])
+                .add_elements([window.TimestampedValue(('k', 400), 7)])
+                .advance_watermark_to_infinity()
+                | 'main windowInto' >> beam.WindowInto(
+                    window.FixedWindows(10),
+                    timestamp_combiner=TimestampCombiner.OUTPUT_AT_EARLIEST)
+                | 'Combine' >> beam.CombinePerKey(sum))
+
+      records = (result | beam.ParDo(self.RecordFn()))
 
 Review comment:
   Also, no need for the surrounding ()'s.
 
----------------------------------------------------------------
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: 347583)
    Time Spent: 7h  (was: 6h 50m)

> TimestampCombiner incorrect in beam python
> ------------------------------------------
>
>                 Key: BEAM-8645
>                 URL: https://issues.apache.org/jira/browse/BEAM-8645
>             Project: Beam
>          Issue Type: Bug
>          Components: sdk-py-core
>            Reporter: Ruoyun Huang
>            Priority: Major
>          Time Spent: 7h
>  Remaining Estimate: 0h
>
> When we have a TimestampValue on combine: 
> {code:java}
> main_stream = (p                   
> | 'main TestStream' >> TestStream()                   
> .add_elements([window.TimestampedValue(('k', 100), 0)])                   
> .add_elements([window.TimestampedValue(('k', 400), 9)])                   
> .advance_watermark_to_infinity()                   
> | 'main windowInto' >> beam.WindowInto(                         
> window.FixedWindows(10),                      
> timestamp_combiner=TimestampCombiner.OUTPUT_AT_LATEST)                   | 
> 'Combine' >> beam.CombinePerKey(sum))
> The expect timestamp should be:
> LATEST:    (('k', 500), Timestamp(9)),
> EARLIEST:    (('k', 500), Timestamp(0)),
> END_OF_WINDOW: (('k', 500), Timestamp(10)),
> But current py streaming gives following results: 
> LATEST:    (('k', 500), Timestamp(10)),
> EARLIEST:    (('k', 500), Timestamp(10)),
> END_OF_WINDOW: (('k', 500), Timestamp(9.99999999)),
> More details and discussions:
> https://lists.apache.org/thread.html/d3af1f2f84a2e59a747196039eae77812b78a991f0f293c717e5f4e1@%3Cdev.beam.apache.org%3E
> {code}



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

Reply via email to