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

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

                Author: ASF GitHub Bot
            Created on: 21/May/18 01:46
            Start Date: 21/May/18 01:46
    Worklog Time Spent: 10m 
      Work Description: aaltay closed pull request #5384: [BEAM-3377] Add 
validation for streaming wordcount with assert_that
URL: https://github.com/apache/beam/pull/5384
 
 
   

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/examples/streaming_wordcount_debugging.py 
b/sdks/python/apache_beam/examples/streaming_wordcount_debugging.py
new file mode 100644
index 00000000000..30e438098b1
--- /dev/null
+++ b/sdks/python/apache_beam/examples/streaming_wordcount_debugging.py
@@ -0,0 +1,184 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+"""A streaming wordcount example with debugging capabilities.
+
+It demonstrate the use of logging and assert_that in streaming mode.
+
+This workflow only works with the DirectRunner (BEAM-3377).
+
+Usage:
+python streaming_wordcount_debugging.py
+--input_topic projects/$PROJECT_ID/topics/$PUBSUB_INPUT_TOPIC
+--output_topic projects/$PROJECT_ID/topics/$PUBSUB_OUTPUT_TOPIC
+--streaming
+
+To publish messages:
+gcloud alpha pubsub topics publish $PUBSUB_INPUT_TOPIC --message '210 213 151'
+
+"""
+
+from __future__ import absolute_import
+
+import argparse
+import logging
+import re
+
+import six
+
+import apache_beam as beam
+import apache_beam.transforms.window as window
+from apache_beam.examples.wordcount import WordExtractingDoFn
+from apache_beam.options.pipeline_options import PipelineOptions
+from apache_beam.options.pipeline_options import SetupOptions
+from apache_beam.options.pipeline_options import StandardOptions
+from apache_beam.testing.util import assert_that
+from apache_beam.testing.util import equal_to_per_window
+from apache_beam.transforms.core import ParDo
+
+
+class PrintFn(beam.DoFn):
+  """A DoFn that prints label, element, its window, and its timstamp. """
+  def __init__(self, label):
+    self.label = label
+
+  def process(self, element, timestamp=beam.DoFn.TimestampParam,
+              window=beam.DoFn.WindowParam):
+    # Log at INFO level each element processed.
+    logging.info('[%s]: %s %s %s', self.label, element, window, timestamp)
+    yield element
+
+
+class AddTimestampFn(beam.DoFn):
+  """A DoFn that attaches timestamps to its elements.
+
+  It takes a string of integers and it attaches to each of them
+  a timestamp of its same value.
+
+  For example, [120, 225, 312] will result in:
+  [(120, Timestamp(120)), (225, Timestamp(225)), (312, Timestamp(312))].
+  """
+  def process(self, element):
+    for elem in element.split(' '):
+      logging.info('Adding timestamp to: %s', element)
+      yield beam.window.TimestampedValue(elem, int(elem))
+
+
+def run(argv=None):
+  """Build and run the pipeline."""
+  parser = argparse.ArgumentParser()
+  parser.add_argument(
+      '--output_topic', required=True,
+      help=('Output PubSub topic of the form '
+            '"projects/<PROJECT>/topic/<TOPIC>".'))
+  group = parser.add_mutually_exclusive_group(required=True)
+  group.add_argument(
+      '--input_topic',
+      help=('Input PubSub topic of the form '
+            '"projects/<PROJECT>/topics/<TOPIC>".'))
+  group.add_argument(
+      '--input_subscription',
+      help=('Input PubSub subscription of the form '
+            '"projects/<PROJECT>/subscriptions/<SUBSCRIPTION>."'))
+  known_args, pipeline_args = parser.parse_known_args(argv)
+
+  # We use the save_main_session option because one or more DoFn's in this
+  # workflow rely on global context (e.g., a module imported at module level).
+  pipeline_options = PipelineOptions(pipeline_args)
+  pipeline_options.view_as(SetupOptions).save_main_session = True
+  pipeline_options.view_as(StandardOptions).streaming = True
+  p = beam.Pipeline(options=pipeline_options)
+
+  # Read from PubSub into a PCollection.
+  if known_args.input_subscription:
+    lines = p | beam.io.ReadStringsFromPubSub(
+        subscription=known_args.input_subscription)
+  else:
+    lines = p | beam.io.ReadStringsFromPubSub(topic=known_args.input_topic)
+
+  # Count the occurrences of each word.
+  def count_ones(word_ones):
+    (word, ones) = word_ones
+    return (word, sum(ones))
+
+  counts = (lines
+            | 'AddTimestampFn' >> beam.ParDo(AddTimestampFn())
+            | 'After AddTimestampFn' >> ParDo(PrintFn('After AddTimestampFn'))
+            | 'Split' >> (beam.ParDo(WordExtractingDoFn())
+                          .with_output_types(six.text_type))
+            | 'PairWithOne' >> beam.Map(lambda x: (x, 1))
+            | beam.WindowInto(window.FixedWindows(5, 0))
+            | 'GroupByKey' >> beam.GroupByKey()
+            | 'CountOnes' >> beam.Map(count_ones))
+
+  # Format the counts into a PCollection of strings.
+  def format_result(word_count):
+    (word, count) = word_count
+    return '%s: %d' % (word, count)
+
+  output = counts | 'format' >> beam.Map(format_result)
+
+  # Write to PubSub.
+  # pylint: disable=expression-not-assigned
+  output | beam.io.WriteStringsToPubSub(known_args.output_topic)
+
+  def check_gbk_format():
+    # A matcher that checks that the output of GBK is of the form word: count.
+    def matcher(elements):
+      # pylint: disable=unused-variable
+      actual_elements_in_window, window = elements
+      for elm in actual_elements_in_window:
+        assert re.match(r'\S+:\s+\d+', elm) is not None
+    return matcher
+
+  # Check that the format of the output is correct.
+  assert_that(
+      output,
+      check_gbk_format(),
+      use_global_window=False,
+      label='Assert word:count format.')
+
+  # Check also that elements are ouput in the right window.
+  # This expects exactly 1 occurrence of any subset of the elements
+  # 150, 151, 152, 153, 154 in the window [150, 155)
+  # or exactly 1 occurrence of any subset of the elements
+  # 210, 211, 212, 213, 214 in the window [210, 215).
+  expected_window_to_elements = {
+      window.IntervalWindow(150, 155): [
+          ('150: 1'), ('151: 1'), ('152: 1'), ('153: 1'), ('154: 1'),
+      ],
+      window.IntervalWindow(210, 215): [
+          ('210: 1'), ('211: 1'), ('212: 1'), ('213: 1'), ('214: 1'),
+      ],
+  }
+
+  # To make it pass, publish numbers in [150-155) or [210-215) with no repeats.
+  # To make it fail, publish a repeated number in the range above range.
+  # For example: '210 213 151 213'
+  assert_that(
+      output,
+      equal_to_per_window(expected_window_to_elements),
+      use_global_window=False,
+      label='Assert correct streaming windowing.')
+
+  result = p.run()
+  result.wait_until_finish()
+
+
+if __name__ == '__main__':
+  logging.getLogger().setLevel(logging.INFO)
+  run()
diff --git a/sdks/python/apache_beam/testing/test_stream_test.py 
b/sdks/python/apache_beam/testing/test_stream_test.py
index 184a61e2a55..a907c183683 100644
--- a/sdks/python/apache_beam/testing/test_stream_test.py
+++ b/sdks/python/apache_beam/testing/test_stream_test.py
@@ -121,28 +121,6 @@ def process(self, element=beam.DoFn.ElementParam,
         ('late', timestamp.Timestamp(12)),
         ('last', timestamp.Timestamp(310)),]))
 
-    # assert per window
-    expected_window_to_elements = {
-        window.IntervalWindow(0, 15): [
-            ('a', Timestamp(10)),
-            ('b', Timestamp(10)),
-            ('c', Timestamp(10)),
-            ('late', Timestamp(12))
-        ],
-        window.IntervalWindow(15, 30): [
-            ('d', Timestamp(20)),
-            ('e', Timestamp(20))
-        ],
-        window.IntervalWindow(300, 315): [
-            ('last', Timestamp(310)),
-        ],
-    }
-    assert_that(
-        records,
-        equal_to_per_window(expected_window_to_elements),
-        custom_windowing=window.FixedWindows(15),
-        label='assert per window')
-
     p.run()
 
   def test_gbk_execution_no_triggers(self):
@@ -169,19 +147,14 @@ def test_gbk_execution_no_triggers(self):
     # TODO(BEAM-2519): timestamp assignment for elements from a GBK should
     # respect the TimestampCombiner.  The test below should also verify the
     # timestamps of the outputted elements once this is implemented.
-    assert_that(records, equal_to([
-        ('k', ['a', 'b', 'c']),
-        ('k', ['d', 'e']),
-        ('k', ['late']),
-        ('k', ['last'])]))
 
     # assert per window
     expected_window_to_elements = {
-        window.IntervalWindow(15, 30): [
+        window.IntervalWindow(0, 15): [
             ('k', ['a', 'b', 'c']),
             ('k', ['late']),
         ],
-        window.IntervalWindow(30, 45): [
+        window.IntervalWindow(15, 30): [
             ('k', ['d', 'e']),
         ],
         window.IntervalWindow(300, 315): [
@@ -191,7 +164,7 @@ def test_gbk_execution_no_triggers(self):
     assert_that(
         records,
         equal_to_per_window(expected_window_to_elements),
-        custom_windowing=window.FixedWindows(15),
+        use_global_window=False,
         label='assert per window')
 
     p.run()
@@ -228,7 +201,7 @@ def test_gbk_execution_after_watermark_trigger(self):
     assert_that(
         records,
         equal_to_per_window(expected_window_to_elements),
-        custom_windowing=window.FixedWindows(15),
+        use_global_window=False,
         label='assert per window')
 
     p.run()
@@ -261,17 +234,13 @@ def 
test_gbk_execution_after_processing_trigger_fired(self):
     # respect the TimestampCombiner.  The test below should also verify the
     # timestamps of the outputted elements once this is implemented.
 
-    assert_that(records, equal_to([
-        ('k', ['a'])]))
-
-    # assert per window
     expected_window_to_elements = {
-        window.IntervalWindow(15, 30): [('k', ['a'])],
+        window.IntervalWindow(0, 15): [('k', ['a'])],
     }
     assert_that(
         records,
         equal_to_per_window(expected_window_to_elements),
-        custom_windowing=window.FixedWindows(15),
+        use_global_window=False,
         label='assert per window')
 
     p.run()
@@ -299,19 +268,8 @@ def process(self,
     records = (main_stream     # pylint: disable=unused-variable
                | beam.ParDo(RecordFn(), beam.pvalue.AsList(side)))
 
-    # assert per window
-    expected_window_to_elements = {
-        window.IntervalWindow(0, 15): [
-            ('e', Timestamp(10), [2, 1, 4]),
-        ],
-    }
-    assert_that(
-        records,
-        equal_to_per_window(expected_window_to_elements),
-        custom_windowing=window.FixedWindows(15),
-        label='assert per window')
-
     assert_that(records, equal_to([('e', Timestamp(10), [2, 1, 4])]))
+
     p.run()
 
   def test_basic_execution_sideinputs(self):
@@ -341,19 +299,8 @@ def process(self,
     records = (main_stream        # pylint: disable=unused-variable
                | beam.ParDo(RecordFn(), beam.pvalue.AsList(side_stream)))
 
-    # assert per window
-    expected_window_to_elements = {
-        window.IntervalWindow(0, 15): [
-            ('e', Timestamp(10), [2, 1, 7, 4]),
-        ],
-    }
-    assert_that(
-        records,
-        equal_to_per_window(expected_window_to_elements),
-        custom_windowing=window.FixedWindows(15),
-        label='assert per window')
-
     assert_that(records, equal_to([('e', Timestamp(10), [2, 1, 7, 4])]))
+
     p.run()
 
   def test_basic_execution_batch_sideinputs_fixed_windows(self):
@@ -391,7 +338,7 @@ def process(self,
     assert_that(
         records,
         equal_to_per_window(expected_window_to_elements),
-        custom_windowing=window.FixedWindows(1),
+        use_global_window=False,
         label='assert per window')
 
     p.run()
@@ -445,7 +392,7 @@ def process(self,
     assert_that(
         records,
         equal_to_per_window(expected_window_to_elements),
-        custom_windowing=window.FixedWindows(1),
+        use_global_window=False,
         label='assert per window')
 
     p.run()
diff --git a/sdks/python/apache_beam/testing/util.py 
b/sdks/python/apache_beam/testing/util.py
index 5caa78ce919..26c54c6b80b 100644
--- a/sdks/python/apache_beam/testing/util.py
+++ b/sdks/python/apache_beam/testing/util.py
@@ -96,7 +96,7 @@ def matcher(elements):
         # For example, in the presence of early triggers.
         if all(elem in sorted_expected for elem in sorted_actual) is False:
           raise BeamAssertException(
-              'Failed assert: %r == %r' % (sorted_expected, sorted_actual))
+              'Failed assert: %r not in %r' % (sorted_actual, sorted_expected))
   return matcher
 
 
@@ -125,7 +125,7 @@ def _empty(actual):
   return _empty
 
 
-def assert_that(actual, matcher, custom_windowing=None,
+def assert_that(actual, matcher, use_global_window=True,
                 label='assert_that', reify_windows=False):
   """A PTransform that checks a PCollection has an expected value.
 
@@ -139,7 +139,7 @@ def assert_that(actual, matcher, custom_windowing=None,
       expectations and raises BeamAssertException if they are not met.
     label: Optional string label. This is needed in case several assert_that
       transforms are introduced in the same pipeline.
-    custom_windowing: If specified, matcher is passed a dictionary of
+    use_global_windows: If False, matcher is passed a dictionary of
       (k, v) = (window, elements in the window).
     reify_windows: If True, matcher is passed a list of TestWindowedValue.
 
@@ -167,15 +167,17 @@ def expand(self, pcoll):
         pcoll = pcoll | ParDo(ReifyTimestampWindow())
 
       keyed_singleton = pcoll.pipeline | Create([(None, None)])
-      keyed_actual = (
-          pcoll
-          | WindowInto(custom_windowing or window.GlobalWindows())
-          | "ToVoidKey" >> Map(lambda v: (None, v)))
+
+      if use_global_window:
+        pcoll = pcoll | WindowInto(window.GlobalWindows())
+
+      keyed_actual = pcoll | "ToVoidKey" >> Map(lambda v: (None, v))
+
       plain_actual = ((keyed_singleton, keyed_actual)
                       | "Group" >> CoGroupByKey()
                       | "Unkey" >> Map(lambda k_values: k_values[1][1]))
 
-      if custom_windowing:
+      if not use_global_window:
         plain_actual = plain_actual | "AddWindow" >> ParDo(AddWindow())
 
       plain_actual = plain_actual | "Match" >> Map(matcher)


 

----------------------------------------------------------------
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: 103823)
    Time Spent: 5h 50m  (was: 5h 40m)

> assert_that not working for streaming
> -------------------------------------
>
>                 Key: BEAM-3377
>                 URL: https://issues.apache.org/jira/browse/BEAM-3377
>             Project: Beam
>          Issue Type: Bug
>          Components: sdk-py-core
>    Affects Versions: 2.2.0
>            Reporter: MarĂ­a GH
>            Priority: Major
>              Labels: starter
>          Time Spent: 5h 50m
>  Remaining Estimate: 0h
>
> assert_that does not work for AfterWatermark timers.
> Easy way to reproduce: modify test_gbk_execution [1] in this form:
>  
> {code:java}
>  def test_this(self):
>     test_stream = (TestStream()
>                    .add_elements(['a', 'b', 'c'])
>                    .advance_watermark_to(20))
>     def fnc(x):
>       print 'fired_elem:', x
>       return x
>     options = PipelineOptions()
>     options.view_as(StandardOptions).streaming = True
>     p = TestPipeline(options=options)
>     records = (p
>                | test_stream
>                | beam.WindowInto(
>                    FixedWindows(15),
>                    
> trigger=trigger.AfterWatermark(early=trigger.AfterCount(2)),
>                    accumulation_mode=trigger.AccumulationMode.ACCUMULATING)
>                | beam.Map(lambda x: ('k', x))
>                | beam.GroupByKey())
>     assert_that(records, equal_to([
>         ('k', ['a', 'b', 'c'])]))
>     p.run()
> {code}
> This test will pass, but if the .advance_watermark_to(20) is removed, the 
> test will fail. However, both cases fire the same elements:
>       fired_elem: ('k', ['a', 'b', 'c'])
>       fired_elem: ('k', ['a', 'b', 'c'])
> In the passing case, they correspond to the sorted_actual inside the 
> assert_that. In the failing case:
>       sorted_actual: [('k', ['a', 'b', 'c']), ('k', ['a', 'b', 'c'])]
>       sorted_actual: []
> [1] 
> https://github.com/mariapython/incubator-beam/blob/direct-timers-show/sdks/python/apache_beam/testing/test_stream_test.py#L120



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to