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 2017/01/20 22:48:52 UTC

[1/4] beam git commit: Add tests for standard beam coder types.

Repository: beam
Updated Branches:
  refs/heads/python-sdk 5588db8d1 -> c57c66ed4


Add tests for standard beam coder types.


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/1503682c
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/1503682c
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/1503682c

Branch: refs/heads/python-sdk
Commit: 1503682c26db74bd6e84b6042a4227a4c2af5846
Parents: 5588db8
Author: Robert Bradshaw <ro...@gmail.com>
Authored: Tue Jan 10 15:47:58 2017 -0800
Committer: Robert Bradshaw <ro...@gmail.com>
Committed: Fri Jan 20 14:48:05 2017 -0800

----------------------------------------------------------------------
 .../apache_beam/coders/standard_coders.yaml     |  25 +++++
 .../apache_beam/coders/standard_coders_test.py  | 108 +++++++++++++++++++
 2 files changed, 133 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/1503682c/sdks/python/apache_beam/coders/standard_coders.yaml
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/coders/standard_coders.yaml b/sdks/python/apache_beam/coders/standard_coders.yaml
new file mode 100644
index 0000000..77db379
--- /dev/null
+++ b/sdks/python/apache_beam/coders/standard_coders.yaml
@@ -0,0 +1,25 @@
+coder:
+  urn: "beam:coders:bytes:0.1"
+nested: false
+examples:
+  abc: abc
+  "ab\0c": "ab\0c"
+
+---
+
+coder:
+  urn: "beam:coders:varint:0.1"
+examples:
+  "\0": 0
+  "\u0001": 1
+  "\u000A": 10
+
+---
+
+coder:
+  urn: "beam:coders:kv:0.1"
+  components: [{urn: "beam:coders:bytes:0.1"},
+               {urn: "beam:coders:varint:0.1"}]
+examples:
+  "\u0003abc\0": {key: abc, value: 0}
+  "\u0004ab\0c\u000A": {key: "ab\0c", value: 10}

http://git-wip-us.apache.org/repos/asf/beam/blob/1503682c/sdks/python/apache_beam/coders/standard_coders_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/coders/standard_coders_test.py b/sdks/python/apache_beam/coders/standard_coders_test.py
new file mode 100644
index 0000000..18dc0a7
--- /dev/null
+++ b/sdks/python/apache_beam/coders/standard_coders_test.py
@@ -0,0 +1,108 @@
+#
+# 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.
+#
+
+"""Unit tests for coders that must be consistent across all beam SDKs.
+"""
+
+import collections
+import os.path
+import pickle
+import unittest
+
+import yaml
+
+import apache_beam as beam
+from apache_beam import coders
+from apache_beam.coders import coder_impl
+
+
+class StandardCodersTest(unittest.TestCase):
+
+  _urn_to_coder_class = {
+    'beam:coders:bytes:0.1': coders.BytesCoder,
+    'beam:coders:varint:0.1': coders.VarIntCoder,
+    'beam:coders:kv:0.1': lambda k, v: coders.TupleCoder((k, v))
+  }
+
+  _urn_to_json_value_parser = {
+    'beam:coders:bytes:0.1': lambda x: x,
+    'beam:coders:varint:0.1': lambda x: x,
+    'beam:coders:kv:0.1':
+        lambda x, key_parser, value_parser: (key_parser(x['key']),
+                                             value_parser(x['value']))
+  }
+
+  # We must prepend an underscore to this name so that the open-source unittest
+  # runner does not execute this method directly as a test.
+  @classmethod
+  def _create_test(cls, spec):
+    counter = 0
+    name = spec.get('name', spec['coder']['urn'].split(':')[-2])
+    unique_name = 'test_' + name
+    while hasattr(cls, unique_name):
+      counter += 1
+      unique_name = 'test_%s_%d' % (name, counter)
+    setattr(cls, unique_name, lambda self: self._run_coder_test(spec))
+
+  # We must prepend an underscore to this name so that the open-source unittest
+  # runner does not execute this method directly as a test.
+  @classmethod
+  def _create_tests(cls, coder_test_specs):
+    for spec in yaml.load_all(open(coder_test_specs)):
+      cls._create_test(spec)
+
+  def _run_coder_test(self, spec):
+    coder = self.parse_coder(spec['coder'])
+    parse_value = self.json_value_parser(spec['coder'])
+    for encoded, json_value in spec['examples'].items():
+      value = parse_value(json_value)
+      if spec.get('nested', True):
+        self.assertEqual(decode_nested(coder, encoded), value)
+        self.assertEqual(encoded, encode_nested(coder, value))
+      if not spec.get('nested', False):
+        self.assertEqual(coder.decode(encoded), value)
+        self.assertEqual(encoded, coder.encode(value))
+
+  def parse_coder(self, spec):
+    return self._urn_to_coder_class[spec['urn']](
+        *[self.parse_coder(c) for c in spec.get('components', ())])
+
+  def json_value_parser(self, coder_spec):
+    component_parsers = [
+        self.json_value_parser(c) for c in coder_spec.get('components', ())]
+    return lambda x: self._urn_to_json_value_parser[coder_spec['urn']](
+        x, *component_parsers)
+
+
+STANDARD_CODERS_YAML = os.path.join(
+    os.path.dirname(__file__), 'standard_coders.yaml')
+if os.path.exists(STANDARD_CODERS_YAML):
+  StandardCodersTest._create_tests(STANDARD_CODERS_YAML)
+
+
+def encode_nested(coder, value):
+  out = coder_impl.create_OutputStream()
+  coder.get_impl().encode_to_stream(value, out, True)
+  return out.get()
+
+def decode_nested(coder, encoded):
+  return coder.get_impl().decode_from_stream(
+      coder_impl.create_InputStream(encoded), True)
+
+
+if __name__ == '__main__':
+  unittest.main()


[3/4] beam git commit: A couple more examples.

Posted by ro...@apache.org.
A couple more examples.


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/7e5dc400
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/7e5dc400
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/7e5dc400

Branch: refs/heads/python-sdk
Commit: 7e5dc40018520fd727c29df81396d7029e0a3358
Parents: 5c054ae
Author: Robert Bradshaw <ro...@gmail.com>
Authored: Tue Jan 10 16:49:33 2017 -0800
Committer: Robert Bradshaw <ro...@gmail.com>
Committed: Fri Jan 20 14:48:07 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/coders/standard_coders.yaml | 16 +++++++++++++++-
 1 file changed, 15 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/7e5dc400/sdks/python/apache_beam/coders/standard_coders.yaml
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/coders/standard_coders.yaml b/sdks/python/apache_beam/coders/standard_coders.yaml
index 3193883..34e4336 100644
--- a/sdks/python/apache_beam/coders/standard_coders.yaml
+++ b/sdks/python/apache_beam/coders/standard_coders.yaml
@@ -20,17 +20,31 @@ coder:
   urn: "beam:coders:bytes:0.1"
 nested: false
 examples:
-  abc: abc
+  "abc": abc
   "ab\0c": "ab\0c"
 
 ---
 
 coder:
+  urn: "beam:coders:bytes:0.1"
+nested: true
+examples:
+  "\u0003abc": abc
+  "\u0004ab\0c": "ab\0c"
+  "\u00c8\u0001       10|       20|       30|       40|       50|       60|       70|       80|       90|      100|      110|      120|      130|      140|      150|      160|      170|      180|      190|      200|":
+              "       10|       20|       30|       40|       50|       60|       70|       80|       90|      100|      110|      120|      130|      140|      150|      160|      170|      180|      190|      200|"
+
+---
+
+coder:
   urn: "beam:coders:varint:0.1"
 examples:
   "\0": 0
   "\u0001": 1
   "\u000A": 10
+  "\u00c8\u0001": 200
+  "\u00e8\u0007": 1000
+  "\u00ff\u00ff\u00ff\u00ff\u00ff\u00ff\u00ff\u00ff\u00ff\u0001": -1
 
 ---
 


[2/4] beam git commit: Add a --fix option to the standard coder test that populates encodings.

Posted by ro...@apache.org.
Add a --fix option to the standard coder test that populates encodings.


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/5c054aef
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/5c054aef
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/5c054aef

Branch: refs/heads/python-sdk
Commit: 5c054aeffd6b5eba8cb62065dd626d33cae0312a
Parents: 1503682
Author: Robert Bradshaw <ro...@gmail.com>
Authored: Tue Jan 10 16:33:23 2017 -0800
Committer: Robert Bradshaw <ro...@gmail.com>
Committed: Fri Jan 20 14:48:07 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/coders/coder_impl.py    |  3 +
 .../apache_beam/coders/standard_coders.yaml     | 28 +++++++
 .../apache_beam/coders/standard_coders_test.py  | 78 +++++++++++++-------
 3 files changed, 84 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/5c054aef/sdks/python/apache_beam/coders/coder_impl.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/coders/coder_impl.py b/sdks/python/apache_beam/coders/coder_impl.py
index fcdc441..96d0a3e 100644
--- a/sdks/python/apache_beam/coders/coder_impl.py
+++ b/sdks/python/apache_beam/coders/coder_impl.py
@@ -37,6 +37,9 @@ try:
   from stream import OutputStream as create_OutputStream
   from stream import ByteCountingOutputStream
   from stream import get_varint_size
+  globals()['create_InputStream'] = create_InputStream
+  globals()['create_OutputStream'] = create_OutputStream
+  globals()['ByteCountingOutputStream'] = ByteCountingOutputStream
 except ImportError:
   from slow_stream import InputStream as create_InputStream
   from slow_stream import OutputStream as create_OutputStream

http://git-wip-us.apache.org/repos/asf/beam/blob/5c054aef/sdks/python/apache_beam/coders/standard_coders.yaml
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/coders/standard_coders.yaml b/sdks/python/apache_beam/coders/standard_coders.yaml
index 77db379..3193883 100644
--- a/sdks/python/apache_beam/coders/standard_coders.yaml
+++ b/sdks/python/apache_beam/coders/standard_coders.yaml
@@ -1,3 +1,21 @@
+#
+# 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.
+#
+
+
 coder:
   urn: "beam:coders:bytes:0.1"
 nested: false
@@ -23,3 +41,13 @@ coder:
 examples:
   "\u0003abc\0": {key: abc, value: 0}
   "\u0004ab\0c\u000A": {key: "ab\0c", value: 10}
+
+---
+
+coder:
+  urn: "beam:coders:kv:0.1"
+  components: [{urn: "beam:coders:bytes:0.1"},
+               {urn: "beam:coders:bytes:0.1"}]
+examples:
+  "\u0003abc\u0003def": {key: abc, value: def}
+  "\u0004ab\0c\u0004de\0f": {key: "ab\0c", value: "de\0f"}

http://git-wip-us.apache.org/repos/asf/beam/blob/5c054aef/sdks/python/apache_beam/coders/standard_coders_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/coders/standard_coders_test.py b/sdks/python/apache_beam/coders/standard_coders_test.py
index 18dc0a7..d451132 100644
--- a/sdks/python/apache_beam/coders/standard_coders_test.py
+++ b/sdks/python/apache_beam/coders/standard_coders_test.py
@@ -15,17 +15,16 @@
 # limitations under the License.
 #
 
-"""Unit tests for coders that must be consistent across all beam SDKs.
+"""Unit tests for coders that must be consistent across all Beam SDKs.
 """
 
-import collections
+import json
 import os.path
-import pickle
+import sys
 import unittest
 
 import yaml
 
-import apache_beam as beam
 from apache_beam import coders
 from apache_beam.coders import coder_impl
 
@@ -33,17 +32,17 @@ from apache_beam.coders import coder_impl
 class StandardCodersTest(unittest.TestCase):
 
   _urn_to_coder_class = {
-    'beam:coders:bytes:0.1': coders.BytesCoder,
-    'beam:coders:varint:0.1': coders.VarIntCoder,
-    'beam:coders:kv:0.1': lambda k, v: coders.TupleCoder((k, v))
+      'beam:coders:bytes:0.1': coders.BytesCoder,
+      'beam:coders:varint:0.1': coders.VarIntCoder,
+      'beam:coders:kv:0.1': lambda k, v: coders.TupleCoder((k, v))
   }
 
   _urn_to_json_value_parser = {
-    'beam:coders:bytes:0.1': lambda x: x,
-    'beam:coders:varint:0.1': lambda x: x,
-    'beam:coders:kv:0.1':
-        lambda x, key_parser, value_parser: (key_parser(x['key']),
-                                             value_parser(x['value']))
+      'beam:coders:bytes:0.1': lambda x: x,
+      'beam:coders:varint:0.1': lambda x: x,
+      'beam:coders:kv:0.1':
+          lambda x, key_parser, value_parser: (key_parser(x['key']),
+                                               value_parser(x['value']))
   }
 
   # We must prepend an underscore to this name so that the open-source unittest
@@ -62,20 +61,25 @@ class StandardCodersTest(unittest.TestCase):
   # runner does not execute this method directly as a test.
   @classmethod
   def _create_tests(cls, coder_test_specs):
-    for spec in yaml.load_all(open(coder_test_specs)):
+    for ix, spec in enumerate(yaml.load_all(open(coder_test_specs))):
+      spec['index'] = ix
       cls._create_test(spec)
 
   def _run_coder_test(self, spec):
     coder = self.parse_coder(spec['coder'])
     parse_value = self.json_value_parser(spec['coder'])
-    for encoded, json_value in spec['examples'].items():
-      value = parse_value(json_value)
-      if spec.get('nested', True):
-        self.assertEqual(decode_nested(coder, encoded), value)
-        self.assertEqual(encoded, encode_nested(coder, value))
-      if not spec.get('nested', False):
-        self.assertEqual(coder.decode(encoded), value)
-        self.assertEqual(encoded, coder.encode(value))
+    nested_list = [spec['nested']] if 'nested' in spec else [True, False]
+    for nested in nested_list:
+      for expected_encoded, json_value in spec['examples'].items():
+        value = parse_value(json_value)
+        expected_encoded = expected_encoded.encode('latin1')
+        actual_encoded = encode_nested(coder, value, nested)
+        if self.fix and actual_encoded != expected_encoded:
+          self.to_fix[spec['index'], expected_encoded] = actual_encoded
+        else:
+          self.assertEqual(decode_nested(coder, expected_encoded, nested),
+                           value)
+          self.assertEqual(expected_encoded, actual_encoded)
 
   def parse_coder(self, spec):
     return self._urn_to_coder_class[spec['urn']](
@@ -87,6 +91,26 @@ class StandardCodersTest(unittest.TestCase):
     return lambda x: self._urn_to_json_value_parser[coder_spec['urn']](
         x, *component_parsers)
 
+  # Used when --fix is passed.
+
+  fix = False
+  to_fix = {}
+
+  @classmethod
+  def tearDownClass(cls):
+    if cls.fix and cls.to_fix:
+      print "FIXING", len(cls.to_fix), "TESTS"
+      doc_sep = '\n---\n'
+      docs = open(STANDARD_CODERS_YAML).read().split(doc_sep)
+
+      def quote(s):
+        return json.dumps(s.decode('latin1')).replace(r'\u0000', r'\0')
+      for (doc_ix, expected_encoded), actual_encoded in cls.to_fix.items():
+        print quote(expected_encoded), "->", quote(actual_encoded)
+        docs[doc_ix] = docs[doc_ix].replace(
+            quote(expected_encoded) + ':', quote(actual_encoded) + ':')
+      open(STANDARD_CODERS_YAML, 'w').write(doc_sep.join(docs))
+
 
 STANDARD_CODERS_YAML = os.path.join(
     os.path.dirname(__file__), 'standard_coders.yaml')
@@ -94,15 +118,19 @@ if os.path.exists(STANDARD_CODERS_YAML):
   StandardCodersTest._create_tests(STANDARD_CODERS_YAML)
 
 
-def encode_nested(coder, value):
+def encode_nested(coder, value, nested=True):
   out = coder_impl.create_OutputStream()
-  coder.get_impl().encode_to_stream(value, out, True)
+  coder.get_impl().encode_to_stream(value, out, nested)
   return out.get()
 
-def decode_nested(coder, encoded):
+
+def decode_nested(coder, encoded, nested=True):
   return coder.get_impl().decode_from_stream(
-      coder_impl.create_InputStream(encoded), True)
+      coder_impl.create_InputStream(encoded), nested)
 
 
 if __name__ == '__main__':
+  if '--fix' in sys.argv:
+    StandardCodersTest.fix = True
+    sys.argv.remove('--fix')
   unittest.main()


[4/4] beam git commit: Closes #1764

Posted by ro...@apache.org.
Closes #1764


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/c57c66ed
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/c57c66ed
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/c57c66ed

Branch: refs/heads/python-sdk
Commit: c57c66ed4f1252388c13abb0d8a6cd02a2c803df
Parents: 5588db8 7e5dc40
Author: Robert Bradshaw <ro...@gmail.com>
Authored: Fri Jan 20 14:48:20 2017 -0800
Committer: Robert Bradshaw <ro...@gmail.com>
Committed: Fri Jan 20 14:48:20 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/coders/coder_impl.py    |   3 +
 .../apache_beam/coders/standard_coders.yaml     |  67 +++++++++
 .../apache_beam/coders/standard_coders_test.py  | 136 +++++++++++++++++++
 3 files changed, 206 insertions(+)
----------------------------------------------------------------------