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

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

                Author: ASF GitHub Bot
            Created on: 20/Nov/19 01:18
            Start Date: 20/Nov/19 01:18
    Worklog Time Spent: 10m 
      Work Description: robertwb commented on pull request #10143: [BEAM-8645] 
To test state backed iterable coder in py sdk.
URL: https://github.com/apache/beam/pull/10143#discussion_r348249978
 
 

 ##########
 File path: sdks/python/apache_beam/runners/portability/fn_api_runner_test.py
 ##########
 @@ -1579,6 +1580,27 @@ def test_lull_logging(self):
         '.*There has been a processing lull of over.*',
         'Unable to find a lull logged for this job.')
 
+@attr('ValidatesRunner')
+class FnApiBasedStateBackedCoderTest(unittest.TestCase):
+  def create_pipeline(self):
+    return beam.Pipeline(
+        runner=fn_api_runner.FnApiRunner(use_state_iterables=True))
+
+  def test_state_backed_coder(self):
+    class MyDoFn(beam.DoFn):
+      def process(self, gbk_result):
+        value_list = gbk_result[1]
+        return (gbk_result[0], sum(value_list))
+
+    with self.create_pipeline() as p:
+      # The number of integers could be a knob to test against
+      # different runners' default settings on page size.
+      main = (p | 'main' >> beam.Create([('a', 1) for _ in range(0, 20000)])
 
 Review comment:
   Rather than create all the values in memory, I'd create these with a DoFn. 
E.g.
   
   beam.Create([None]) | beam.FlatMap(lambda x: ((x, 1) for _ in range(20000))) 
| ...
 
----------------------------------------------------------------
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: 346400)
    Time Spent: 6h  (was: 5h 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: 6h
>  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