Repository: incubator-beam Updated Branches: refs/heads/python-sdk 6e6d89d48 -> 8e1793caf
Fix tests expecting list from AsIter. Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/66b4c2f1 Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/66b4c2f1 Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/66b4c2f1 Branch: refs/heads/python-sdk Commit: 66b4c2f160b5fdce6aca9d5ea50b406fcb07bb51 Parents: 29a7378 Author: Robert Bradshaw <rober...@google.com> Authored: Thu Oct 13 15:15:06 2016 -0700 Committer: Robert Bradshaw <rober...@google.com> Committed: Tue Oct 18 12:17:15 2016 -0700 ---------------------------------------------------------------------- .../python/apache_beam/examples/cookbook/bigquery_side_input.py | 2 +- sdks/python/apache_beam/transforms/util.py | 5 +++-- sdks/python/apache_beam/transforms/write_ptransform_test.py | 2 +- 3 files changed, 5 insertions(+), 4 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/66b4c2f1/sdks/python/apache_beam/examples/cookbook/bigquery_side_input.py ---------------------------------------------------------------------- diff --git a/sdks/python/apache_beam/examples/cookbook/bigquery_side_input.py b/sdks/python/apache_beam/examples/cookbook/bigquery_side_input.py index 2099e48..ffba786 100644 --- a/sdks/python/apache_beam/examples/cookbook/bigquery_side_input.py +++ b/sdks/python/apache_beam/examples/cookbook/bigquery_side_input.py @@ -72,7 +72,7 @@ def create_groups(group_ids, corpus, word, ignore_corpus, ignore_word): | beam.FlatMap( 'attach word', attach_word_fn, - AsIter(word), + AsList(word), AsSingleton(ignore_word))) http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/66b4c2f1/sdks/python/apache_beam/transforms/util.py ---------------------------------------------------------------------- diff --git a/sdks/python/apache_beam/transforms/util.py b/sdks/python/apache_beam/transforms/util.py index aeac0d9..ebe6ba9 100644 --- a/sdks/python/apache_beam/transforms/util.py +++ b/sdks/python/apache_beam/transforms/util.py @@ -20,7 +20,7 @@ from __future__ import absolute_import -from apache_beam.pvalue import AsIter as AllOf +from apache_beam.pvalue import AsList from apache_beam.transforms import core from apache_beam.transforms import window from apache_beam.transforms.core import CombinePerKey, Create, Flatten, GroupByKey, Map @@ -192,6 +192,7 @@ def equal_to(expected): def is_empty(): def _empty(actual): + actual = list(actual) if actual: raise DataflowAssertException( 'Failed assert: [] == %r' % actual) @@ -224,7 +225,7 @@ def assert_that(actual, matcher, label='assert_that'): def apply(self, pipeline): return pipeline | 'singleton' >> Create([None]) | Map( match, - AllOf(actual | core.WindowInto(window.GlobalWindows()))) + AsList(actual | core.WindowInto(window.GlobalWindows()))) def default_label(self): return label http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/66b4c2f1/sdks/python/apache_beam/transforms/write_ptransform_test.py ---------------------------------------------------------------------- diff --git a/sdks/python/apache_beam/transforms/write_ptransform_test.py b/sdks/python/apache_beam/transforms/write_ptransform_test.py index af3668c..e7cdbd4 100644 --- a/sdks/python/apache_beam/transforms/write_ptransform_test.py +++ b/sdks/python/apache_beam/transforms/write_ptransform_test.py @@ -100,7 +100,7 @@ class WriteTest(unittest.TestCase): write_to_test_sink = WriteToTestSink(return_init_result, return_write_results) p = Pipeline(options=PipelineOptions([])) - result = p | 'start' >> beam.Create(data) | write_to_test_sink + result = p | beam.Create(data) | write_to_test_sink | beam.Map(list) assert_that(result, is_empty()) p.run()