You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by ro...@apache.org on 2016/07/19 00:48:09 UTC
[1/2] incubator-beam git commit: Closes #685
Repository: incubator-beam
Updated Branches:
refs/heads/python-sdk ad7c216f4 -> 69f895a2e
Closes #685
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/69f895a2
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/69f895a2
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/69f895a2
Branch: refs/heads/python-sdk
Commit: 69f895a2e7f5a4ef39c060cfb0320c87e64ea8be
Parents: ad7c216 3bfd681
Author: Robert Bradshaw <ro...@google.com>
Authored: Mon Jul 18 17:47:53 2016 -0700
Committer: Robert Bradshaw <ro...@google.com>
Committed: Mon Jul 18 17:47:53 2016 -0700
----------------------------------------------------------------------
.../examples/complete/autocomplete_test.py | 4 +-
.../examples/complete/estimate_pi_test.py | 1 +
.../examples/cookbook/group_with_coder_test.py | 70 ++++++++++++--------
sdks/python/apache_beam/transforms/util.py | 13 ----
4 files changed, 45 insertions(+), 43 deletions(-)
----------------------------------------------------------------------
[2/2] incubator-beam git commit: Fixing broken example tests
Posted by ro...@apache.org.
Fixing broken example tests
Nose does not pick up decorated tests unless the decorator name
starts with test_. This resulted in some test being inadvertently
disabled.
Also OptionsContext
(https://github.com/aaltay/incubator-beam/blob/python-sdk/sdks/python/apache_beam/utils/options.py#L457)
is not doing what it is supposed to do. Its augment_options() method is
not called therefore using OptionsContext does not override the option
values as expected. I will remove this class and a few uses of it in
tests, in a follow up.
Finally removed contains_in_any_order in favor of equal_to. (As
discuessed in:
https://github.com/apache/incubator-beam/pull/650#discussion_r70906340
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/3bfd6813
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/3bfd6813
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/3bfd6813
Branch: refs/heads/python-sdk
Commit: 3bfd6813960eb8b6161bac79b96349e4c721bfea
Parents: ad7c216
Author: Ahmet Altay <al...@google.com>
Authored: Mon Jul 18 14:24:51 2016 -0700
Committer: Robert Bradshaw <ro...@google.com>
Committed: Mon Jul 18 17:47:53 2016 -0700
----------------------------------------------------------------------
.../examples/complete/autocomplete_test.py | 4 +-
.../examples/complete/estimate_pi_test.py | 1 +
.../examples/cookbook/group_with_coder_test.py | 70 ++++++++++++--------
sdks/python/apache_beam/transforms/util.py | 13 ----
4 files changed, 45 insertions(+), 43 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3bfd6813/sdks/python/apache_beam/examples/complete/autocomplete_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/examples/complete/autocomplete_test.py b/sdks/python/apache_beam/examples/complete/autocomplete_test.py
index 1b3ee5f..84f947b 100644
--- a/sdks/python/apache_beam/examples/complete/autocomplete_test.py
+++ b/sdks/python/apache_beam/examples/complete/autocomplete_test.py
@@ -22,7 +22,7 @@ import unittest
import apache_beam as beam
from apache_beam.examples.complete import autocomplete
from apache_beam.transforms.util import assert_that
-from apache_beam.transforms.util import contains_in_any_order
+from apache_beam.transforms.util import equal_to
class AutocompleteTest(unittest.TestCase):
@@ -35,7 +35,7 @@ class AutocompleteTest(unittest.TestCase):
result = words | autocomplete.TopPerPrefix('test', 5)
# values must be hashable for now
result = result | beam.Map(lambda (k, vs): (k, tuple(vs)))
- assert_that(result, contains_in_any_order(
+ assert_that(result, equal_to(
[
('t', ((3, 'to'), (2, 'this'), (1, 'that'))),
('to', ((3, 'to'), )),
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3bfd6813/sdks/python/apache_beam/examples/complete/estimate_pi_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/examples/complete/estimate_pi_test.py b/sdks/python/apache_beam/examples/complete/estimate_pi_test.py
index 7ca82d7..c633bb1 100644
--- a/sdks/python/apache_beam/examples/complete/estimate_pi_test.py
+++ b/sdks/python/apache_beam/examples/complete/estimate_pi_test.py
@@ -17,6 +17,7 @@
"""Test for the estimate_pi example."""
+import logging
import unittest
import apache_beam as beam
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3bfd6813/sdks/python/apache_beam/examples/cookbook/group_with_coder_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/examples/cookbook/group_with_coder_test.py b/sdks/python/apache_beam/examples/cookbook/group_with_coder_test.py
index 07211a9..fb52809 100644
--- a/sdks/python/apache_beam/examples/cookbook/group_with_coder_test.py
+++ b/sdks/python/apache_beam/examples/cookbook/group_with_coder_test.py
@@ -18,13 +18,10 @@
"""Test for the custom coders example."""
import logging
+import tempfile
import unittest
-import apache_beam as beam
from apache_beam.examples.cookbook import group_with_coder
-from apache_beam.transforms.util import assert_that
-from apache_beam.transforms.util import equal_to
-from apache_beam.utils.options import OptionsContext
# Patch group_with_coder.PlayerCoder.decode(). To test that the PlayerCoder was
@@ -39,37 +36,54 @@ class GroupWithCoderTest(unittest.TestCase):
'joe,20', 'fred,6', 'ann,5',
'joe,30', 'ann,10', 'mary,1']
- @OptionsContext(pipeline_type_check=True)
- def test_basics_with_type_check_n(self):
- # Run the workflow with pipeline_type_check option. This will make sure
+ def create_temp_file(self, records):
+ with tempfile.NamedTemporaryFile(delete=False) as f:
+ for record in records:
+ f.write('%s\n' % record)
+ return f.name
+
+ def test_basics_with_type_check(self):
+ # Run the workflow with --pipeline_type_check option. This will make sure
# the typehints associated with all transforms will have non-default values
# and therefore any custom coders will be used. In our case we want to make
# sure the coder for the Player class will be used.
- p = beam.Pipeline('DirectPipelineRunner')
- data = p | beam.Create('create', self.SAMPLE_RECORDS)
- result = (data
- | beam.Map('get players', group_with_coder.get_players)
- | beam.CombinePerKey(sum)
- | beam.Map(lambda (k, v): '%s,%d' % (k.name, v)))
- assert_that(result, equal_to(
- ['x:ann,15', 'x:fred,9', 'x:joe,60', 'x:mary,8']))
- p.run()
+ temp_path = self.create_temp_file(self.SAMPLE_RECORDS)
+ group_with_coder.run([
+ '--pipeline_type_check',
+ '--input=%s*' % temp_path,
+ '--output=%s.result' % temp_path])
+ # Parse result file and compare.
+ results = []
+ with open(temp_path + '.result-00000-of-00001') as result_file:
+ for line in result_file:
+ name, points = line.split(',')
+ results.append((name, int(points)))
+ logging.info('result: %s', results)
+ self.assertEqual(
+ sorted(results),
+ sorted([('x:ann', 15), ('x:fred', 9), ('x:joe', 60), ('x:mary', 8)]))
- @OptionsContext(pipeline_type_check=False)
- def test_basics_without_type_check_n(self):
- # Run the workflow without pipeline_type_check option. This will make sure
+ def test_basics_without_type_check(self):
+ # Run the workflow without --pipeline_type_check option. This will make sure
# the typehints associated with all transforms will have default values and
# therefore any custom coders will not be used. The default coder (pickler)
# will be used instead.
- p = beam.Pipeline('DirectPipelineRunner')
- data = p | beam.Create('create', self.SAMPLE_RECORDS)
- result = (data
- | beam.Map('get players', group_with_coder.get_players)
- | beam.CombinePerKey(sum)
- | beam.Map(lambda (k, v): '%s,%d' % (k.name, v)))
- assert_that(result, equal_to(
- ['ann,15', 'fred,9', 'joe,60', 'mary,8']))
- p.run()
+ temp_path = self.create_temp_file(self.SAMPLE_RECORDS)
+ group_with_coder.run([
+ '--no_pipeline_type_check',
+ '--input=%s*' % temp_path,
+ '--output=%s.result' % temp_path])
+ # Parse result file and compare.
+ results = []
+ with open(temp_path + '.result-00000-of-00001') as result_file:
+ for line in result_file:
+ name, points = line.split(',')
+ results.append((name, int(points)))
+ logging.info('result: %s', results)
+ self.assertEqual(
+ sorted(results),
+ sorted([('ann', 15), ('fred', 9), ('joe', 60), ('mary', 8)]))
+
if __name__ == '__main__':
logging.getLogger().setLevel(logging.INFO)
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3bfd6813/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 59f4338..b7a121d 100644
--- a/sdks/python/apache_beam/transforms/util.py
+++ b/sdks/python/apache_beam/transforms/util.py
@@ -20,8 +20,6 @@
from __future__ import absolute_import
-import collections
-
from apache_beam.pvalue import AsIter as AllOf
from apache_beam.transforms.core import CombinePerKey, Create, Flatten, GroupByKey, Map
from apache_beam.transforms.ptransform import PTransform
@@ -37,7 +35,6 @@ __all__ = [
'assert_that',
'equal_to',
'is_empty',
- 'contains_in_any_order',
]
@@ -199,16 +196,6 @@ def is_empty():
return _empty
-def contains_in_any_order(expected):
- def _contains_in_any_order(actual):
- vs = collections.Counter(actual)
- es = collections.Counter(expected)
- if vs != es:
- raise DataflowAssertException(
- 'Failed assert: extra: %s, missing: %s' % (vs - es, es - vs))
- return _contains_in_any_order
-
-
def assert_that(actual, matcher, label='assert_that'):
"""A PTransform that checks a PCollection has an expected value.