Repository: beam
Updated Branches:
  refs/heads/master b67bd111e -> 8e0573ba5


Add cross-sdk tests for IterableCoder


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

Branch: refs/heads/master
Commit: cab5e6347f3fcece2cac4819e268488d7ce66830
Parents: b67bd11
Author: Vikas Kedigehalli <vika...@google.com>
Authored: Mon Feb 13 10:23:28 2017 -0800
Committer: Vikas Kedigehalli <vika...@google.com>
Committed: Mon Feb 13 15:35:19 2017 -0800

----------------------------------------------------------------------
 .../org/apache/beam/fn/v1/standard_coders.yaml  | 20 ++++++++++++++++++++
 .../apache/beam/sdk/coders/CommonCoderTest.java | 12 ++++++++++++
 .../apache_beam/coders/standard_coders_test.py  |  2 ++
 .../apache_beam/tests/data/standard_coders.yaml | 20 ++++++++++++++++++++
 4 files changed, 54 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/cab5e634/sdks/common/fn-api/src/test/resources/org/apache/beam/fn/v1/standard_coders.yaml
----------------------------------------------------------------------
diff --git 
a/sdks/common/fn-api/src/test/resources/org/apache/beam/fn/v1/standard_coders.yaml
 
b/sdks/common/fn-api/src/test/resources/org/apache/beam/fn/v1/standard_coders.yaml
index 948ac6b..58a2a90 100644
--- 
a/sdks/common/fn-api/src/test/resources/org/apache/beam/fn/v1/standard_coders.yaml
+++ 
b/sdks/common/fn-api/src/test/resources/org/apache/beam/fn/v1/standard_coders.yaml
@@ -106,3 +106,23 @@ examples:
   
"\u0080\u0000\u0001\u0053\u0034\u00ec\u0074\u00e8\u0080\u0090\u00fb\u00d3\u0009"
 : {end: 1456881825000, span: 2592000000}
   "\u007f\u00df\u003b\u0064\u005a\u001c\u00ad\u0076\u00ed\u0002" : {end: 
-9223372036854410, span: 365}
   "\u0080\u0020\u00c4\u009b\u00a5\u00e3\u0053\u00f7\u0000" : {end: 
9223372036854775, span: 0}
+
+---
+
+coder:
+  urn: "urn:beam:coders:stream:0.1"
+  components: [{urn: "urn:beam:coders:varint:0.1"}]
+examples:
+  "\0\0\0\u0001\0": [0]
+  "\0\0\0\u0004\u0001\n\u00c8\u0001\u00e8\u0007": [1, 10, 200, 1000]
+  "\0\0\0\0": []
+
+---
+
+coder:
+  urn: "urn:beam:coders:stream:0.1"
+  components: [{urn: "urn:beam:coders:bytes:0.1"}]
+examples:
+  "\0\0\0\u0001\u0003abc": ["abc"]
+  "\0\0\0\u0002\u0004ab\0c\u0004de\0f": ["ab\0c", "de\0f"]
+  "\0\0\0\0": []
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/beam/blob/cab5e634/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CommonCoderTest.java
----------------------------------------------------------------------
diff --git 
a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CommonCoderTest.java 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CommonCoderTest.java
index ad5d9c3..7eafbe2 100644
--- 
a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CommonCoderTest.java
+++ 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CommonCoderTest.java
@@ -70,6 +70,7 @@ public class CommonCoderTest {
       .put("urn:beam:coders:kv:0.1", KvCoder.class)
       .put("urn:beam:coders:varint:0.1", VarLongCoder.class)
       .put("urn:beam:coders:intervalwindow:0.1", IntervalWindowCoder.class)
+      .put("urn:beam:coders:stream:0.1", IterableCoder.class)
       .build();
 
   @AutoValue
@@ -198,6 +199,15 @@ public class CommonCoderTest {
         Duration span = Duration.millis(((Number) 
kvMap.get("span")).longValue());
         return new IntervalWindow(end.minus(span), span);
       }
+      case "urn:beam:coders:stream:0.1":
+        Coder elementCoder = ((IterableCoder) coder).getElemCoder();
+        List<Object> elements = (List<Object>) value;
+        List<Object> convertedElements = new LinkedList<>();
+        for (Object element : elements) {
+          convertedElements.add(
+              convertValue(element, coderSpec.getComponents().get(0), 
elementCoder));
+        }
+        return convertedElements;
       default:
         throw new IllegalStateException("Unknown coder URN: " + 
coderSpec.getUrn());
     }
@@ -217,6 +227,8 @@ public class CommonCoderTest {
         return VarLongCoder.of();
       case "urn:beam:coders:intervalwindow:0.1":
         return IntervalWindowCoder.of();
+      case "urn:beam:coders:stream:0.1":
+        return IterableCoder.of(components.get(0));
       default:
         throw new IllegalStateException("Unknown coder URN: " + 
coder.getUrn());
     }

http://git-wip-us.apache.org/repos/asf/beam/blob/cab5e634/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 e66ec7b..d4179eb 100644
--- a/sdks/python/apache_beam/coders/standard_coders_test.py
+++ b/sdks/python/apache_beam/coders/standard_coders_test.py
@@ -38,6 +38,7 @@ class StandardCodersTest(unittest.TestCase):
       'urn:beam:coders:varint:0.1': coders.VarIntCoder,
       'urn:beam:coders:kv:0.1': lambda k, v: coders.TupleCoder((k, v)),
       'urn:beam:coders:intervalwindow:0.1': coders.IntervalWindowCoder,
+      'urn:beam:coders:stream:0.1': lambda t: coders.IterableCoder(t),
   }
 
   _urn_to_json_value_parser = {
@@ -50,6 +51,7 @@ class StandardCodersTest(unittest.TestCase):
           lambda x: IntervalWindow(
               start=Timestamp(micros=(x['end'] - x['span']) * 1000),
               end=Timestamp(micros=x['end'] * 1000)),
+      'urn:beam:coders:stream:0.1': lambda x, parser: map(parser, x)
   }
 
   # We must prepend an underscore to this name so that the open-source unittest

http://git-wip-us.apache.org/repos/asf/beam/blob/cab5e634/sdks/python/apache_beam/tests/data/standard_coders.yaml
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/tests/data/standard_coders.yaml 
b/sdks/python/apache_beam/tests/data/standard_coders.yaml
index 948ac6b..ea0b11b 100644
--- a/sdks/python/apache_beam/tests/data/standard_coders.yaml
+++ b/sdks/python/apache_beam/tests/data/standard_coders.yaml
@@ -106,3 +106,23 @@ examples:
   
"\u0080\u0000\u0001\u0053\u0034\u00ec\u0074\u00e8\u0080\u0090\u00fb\u00d3\u0009"
 : {end: 1456881825000, span: 2592000000}
   "\u007f\u00df\u003b\u0064\u005a\u001c\u00ad\u0076\u00ed\u0002" : {end: 
-9223372036854410, span: 365}
   "\u0080\u0020\u00c4\u009b\u00a5\u00e3\u0053\u00f7\u0000" : {end: 
9223372036854775, span: 0}
+
+---
+
+coder:
+  urn: "urn:beam:coders:stream:0.1"
+  components: [{urn: "urn:beam:coders:varint:0.1"}]
+examples:
+  "\0\0\0\u0001\0": [0]
+  "\0\0\0\u0004\u0001\n\u00c8\u0001\u00e8\u0007": [1, 10, 200, 1000]
+  "\0\0\0\0": []
+
+---
+
+coder:
+  urn: "urn:beam:coders:stream:0.1"
+  components: [{urn: "urn:beam:coders:bytes:0.1"}]
+examples:
+  "\0\0\0\u0001\u0003abc": ["abc"]
+  "\0\0\0\u0002\u0004ab\0c\u0004de\0f": ["ab\0c", "de\0f"]
+  "\0\0\0\0": []

Reply via email to