You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by al...@apache.org on 2017/05/12 00:12:00 UTC

[06/19] beam git commit: [BEAM-1345] Clearly delineate public api in apache_beam/coders.

[BEAM-1345] Clearly delineate public api in apache_beam/coders.


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

Branch: refs/heads/release-2.0.0
Commit: 7fd012b63f7ff8b10d80bce53a860ad7102673d6
Parents: 0ce2543
Author: Robert Bradshaw <ro...@google.com>
Authored: Wed May 10 17:13:06 2017 -0700
Committer: Ahmet Altay <al...@google.com>
Committed: Thu May 11 16:20:36 2017 -0700

----------------------------------------------------------------------
 sdks/python/apache_beam/coders/coder_impl.py    |  2 ++
 sdks/python/apache_beam/coders/coders.py        | 32 +++++++++++++++++---
 sdks/python/apache_beam/coders/coders_test.py   | 11 ++++---
 .../apache_beam/coders/coders_test_common.py    |  2 +-
 sdks/python/apache_beam/coders/observable.py    |  5 ++-
 sdks/python/apache_beam/coders/slow_stream.py   |  5 ++-
 .../apache_beam/coders/standard_coders_test.py  |  2 +-
 sdks/python/apache_beam/coders/stream.pyx       |  5 +++
 sdks/python/apache_beam/coders/typecoders.py    |  3 ++
 .../examples/snippets/snippets_test.py          |  3 +-
 sdks/python/apache_beam/io/fileio_test.py       |  2 +-
 sdks/python/apache_beam/io/textio.py            |  2 +-
 sdks/python/apache_beam/transforms/window.py    |  2 +-
 13 files changed, 59 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/7fd012b6/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 a0496a2..10298bf 100644
--- a/sdks/python/apache_beam/coders/coder_impl.py
+++ b/sdks/python/apache_beam/coders/coder_impl.py
@@ -23,6 +23,8 @@ encode many elements with minimal overhead.
 
 This module may be optionally compiled with Cython, using the corresponding
 coder_impl.pxd file for type hints.
+
+For internal use only; no backwards-compatibility guarantees.
 """
 from types import NoneType
 

http://git-wip-us.apache.org/repos/asf/beam/blob/7fd012b6/sdks/python/apache_beam/coders/coders.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/coders/coders.py b/sdks/python/apache_beam/coders/coders.py
index 4f75182..ce914dd 100644
--- a/sdks/python/apache_beam/coders/coders.py
+++ b/sdks/python/apache_beam/coders/coders.py
@@ -15,7 +15,10 @@
 # limitations under the License.
 #
 
-"""Collection of useful coders."""
+"""Collection of useful coders.
+
+Only those coders listed in __all__ are part of the public API of this module.
+"""
 
 import base64
 import cPickle as pickle
@@ -45,6 +48,13 @@ except ImportError:
   import dill
 
 
+__all__ = ['Coder',
+           'BytesCoder', 'DillCoder', 'FastPrimitivesCoder', 'FloatCoder',
+           'IterableCoder', 'PickleCoder', 'ProtoCoder', 'SingletonCoder',
+           'StrUtf8Coder', 'TimestampCoder', 'TupleCoder',
+           'TupleSequenceCoder', 'VarIntCoder', 'WindowedValueCoder']
+
+
 def serialize_coder(coder):
   from apache_beam.internal import pickler
   return '%s$%s' % (coder.__class__.__name__, pickler.dumps(coder))
@@ -116,6 +126,10 @@ class Coder(object):
                                         self.estimate_size)
 
   def get_impl(self):
+    """For internal use only; no backwards-compatibility guarantees.
+
+    Returns the CoderImpl backing this Coder.
+    """
     if not hasattr(self, '_impl'):
       self._impl = self._create_impl()
       assert isinstance(self._impl, coder_impl.CoderImpl)
@@ -152,13 +166,17 @@ class Coder(object):
       raise ValueError('Not a KV coder: %s.' % self)
 
   def _get_component_coders(self):
-    """Returns the internal component coders of this coder."""
+    """For internal use only; no backwards-compatibility guarantees.
+
+    Returns the internal component coders of this coder."""
     # This is an internal detail of the Coder API and does not need to be
     # refined in user-defined Coders.
     return []
 
   def as_cloud_object(self):
-    """Returns Google Cloud Dataflow API description of this coder."""
+    """For internal use only; no backwards-compatibility guarantees.
+
+    Returns Google Cloud Dataflow API description of this coder."""
     # This is an internal detail of the Coder API and does not need to be
     # refined in user-defined Coders.
 
@@ -184,6 +202,8 @@ class Coder(object):
     # pylint: enable=protected-access
 
   def to_runner_api(self, context):
+    """For internal use only; no backwards-compatibility guarantees.
+    """
     # TODO(BEAM-115): Use specialized URNs and components.
     from apache_beam.runners.api import beam_runner_api_pb2
     return beam_runner_api_pb2.Coder(
@@ -196,6 +216,8 @@ class Coder(object):
 
   @staticmethod
   def from_runner_api(proto, context):
+    """For internal use only; no backwards-compatibility guarantees.
+    """
     any_proto = proto.spec.spec.parameter
     bytes_proto = google.protobuf.wrappers_pb2.BytesValue()
     any_proto.Unpack(bytes_proto)
@@ -779,7 +801,9 @@ class WindowedValueCoder(FastCoder):
 
 
 class LengthPrefixCoder(FastCoder):
-  """Coder which prefixes the length of the encoded object in the stream."""
+  """For internal use only; no backwards-compatibility guarantees.
+
+  Coder which prefixes the length of the encoded object in the stream."""
 
   def __init__(self, value_coder):
     self._value_coder = value_coder

http://git-wip-us.apache.org/repos/asf/beam/blob/7fd012b6/sdks/python/apache_beam/coders/coders_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/coders/coders_test.py b/sdks/python/apache_beam/coders/coders_test.py
index 575503b..c89e810 100644
--- a/sdks/python/apache_beam/coders/coders_test.py
+++ b/sdks/python/apache_beam/coders/coders_test.py
@@ -20,8 +20,9 @@ import base64
 import logging
 import unittest
 
-from apache_beam import coders
+from apache_beam.coders import coders
 from apache_beam.coders import proto2_coder_test_messages_pb2 as test_message
+from apache_beam.coders.typecoders import registry as coders_registry
 
 
 class PickleCoderTest(unittest.TestCase):
@@ -46,13 +47,13 @@ class PickleCoderTest(unittest.TestCase):
 class CodersTest(unittest.TestCase):
 
   def test_str_utf8_coder(self):
-    real_coder = coders.registry.get_coder(str)
+    real_coder = coders_registry.get_coder(str)
     expected_coder = coders.BytesCoder()
     self.assertEqual(
         real_coder.encode('abc'), expected_coder.encode('abc'))
     self.assertEqual('abc', real_coder.decode(real_coder.encode('abc')))
 
-    real_coder = coders.registry.get_coder(bytes)
+    real_coder = coders_registry.get_coder(bytes)
     expected_coder = coders.BytesCoder()
     self.assertEqual(
         real_coder.encode('abc'), expected_coder.encode('abc'))
@@ -82,7 +83,7 @@ class ProtoCoderTest(unittest.TestCase):
     mb.field1 = True
     ma.field1 = u'hello world'
     expected_coder = coders.ProtoCoder(ma.__class__)
-    real_coder = coders.registry.get_coder(ma.__class__)
+    real_coder = coders_registry.get_coder(ma.__class__)
     self.assertEqual(expected_coder, real_coder)
     self.assertEqual(real_coder.encode(ma), expected_coder.encode(ma))
     self.assertEqual(ma, real_coder.decode(real_coder.encode(ma)))
@@ -104,7 +105,7 @@ class FallbackCoderTest(unittest.TestCase):
   def test_default_fallback_path(self):
     """Test fallback path picks a matching coder if no coder is registered."""
 
-    coder = coders.registry.get_coder(DummyClass)
+    coder = coders_registry.get_coder(DummyClass)
     # No matching coder, so picks the last fallback coder which is a
     # FastPrimitivesCoder.
     self.assertEqual(coder, coders.FastPrimitivesCoder())

http://git-wip-us.apache.org/repos/asf/beam/blob/7fd012b6/sdks/python/apache_beam/coders/coders_test_common.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/coders/coders_test_common.py b/sdks/python/apache_beam/coders/coders_test_common.py
index e5bfe35..c9b67b3 100644
--- a/sdks/python/apache_beam/coders/coders_test_common.py
+++ b/sdks/python/apache_beam/coders/coders_test_common.py
@@ -23,12 +23,12 @@ import unittest
 
 import dill
 
-import coders
 import observable
 from apache_beam.transforms import window
 from apache_beam.utils import timestamp
 from apache_beam.utils import windowed_value
 
+from apache_beam.coders import coders
 from apache_beam.coders import proto2_coder_test_messages_pb2 as test_message
 
 

http://git-wip-us.apache.org/repos/asf/beam/blob/7fd012b6/sdks/python/apache_beam/coders/observable.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/coders/observable.py b/sdks/python/apache_beam/coders/observable.py
index 5a808d8..fc952cf 100644
--- a/sdks/python/apache_beam/coders/observable.py
+++ b/sdks/python/apache_beam/coders/observable.py
@@ -16,7 +16,10 @@
 #
 
 
-"""Observable base class for iterables."""
+"""Observable base class for iterables.
+
+For internal use only; no backwards-compatibility guarantees.
+"""
 
 
 class ObservableMixin(object):

http://git-wip-us.apache.org/repos/asf/beam/blob/7fd012b6/sdks/python/apache_beam/coders/slow_stream.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/coders/slow_stream.py b/sdks/python/apache_beam/coders/slow_stream.py
index 85837bc..1ab55d9 100644
--- a/sdks/python/apache_beam/coders/slow_stream.py
+++ b/sdks/python/apache_beam/coders/slow_stream.py
@@ -15,7 +15,10 @@
 # limitations under the License.
 #
 
-"""A pure Python implementation of stream.pyx."""
+"""A pure Python implementation of stream.pyx.
+
+For internal use only; no backwards-compatibility guarantees.
+"""
 
 import struct
 

http://git-wip-us.apache.org/repos/asf/beam/blob/7fd012b6/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 885e88f..5f98455 100644
--- a/sdks/python/apache_beam/coders/standard_coders_test.py
+++ b/sdks/python/apache_beam/coders/standard_coders_test.py
@@ -26,7 +26,7 @@ import unittest
 
 import yaml
 
-from apache_beam import coders
+from apache_beam.coders import coders
 from apache_beam.coders import coder_impl
 from apache_beam.utils import windowed_value
 from apache_beam.utils.timestamp import Timestamp

http://git-wip-us.apache.org/repos/asf/beam/blob/7fd012b6/sdks/python/apache_beam/coders/stream.pyx
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/coders/stream.pyx b/sdks/python/apache_beam/coders/stream.pyx
index ae24418..8d97681 100644
--- a/sdks/python/apache_beam/coders/stream.pyx
+++ b/sdks/python/apache_beam/coders/stream.pyx
@@ -15,6 +15,11 @@
 # limitations under the License.
 #
 
+"""Compiled version of the Stream objects used by CoderImpl.
+
+For internal use only; no backwards-compatibility guarantees.
+"""
+
 cimport libc.stdlib
 cimport libc.string
 

http://git-wip-us.apache.org/repos/asf/beam/blob/7fd012b6/sdks/python/apache_beam/coders/typecoders.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/coders/typecoders.py b/sdks/python/apache_beam/coders/typecoders.py
index 60832c9..3894bb5 100644
--- a/sdks/python/apache_beam/coders/typecoders.py
+++ b/sdks/python/apache_beam/coders/typecoders.py
@@ -71,6 +71,9 @@ from apache_beam.coders import coders
 from apache_beam.typehints import typehints
 
 
+__all__ = ['registry']
+
+
 class CoderRegistry(object):
   """A coder registry for typehint/coder associations."""
 

http://git-wip-us.apache.org/repos/asf/beam/blob/7fd012b6/sdks/python/apache_beam/examples/snippets/snippets_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/examples/snippets/snippets_test.py b/sdks/python/apache_beam/examples/snippets/snippets_test.py
index 85d8bde..211da24 100644
--- a/sdks/python/apache_beam/examples/snippets/snippets_test.py
+++ b/sdks/python/apache_beam/examples/snippets/snippets_test.py
@@ -29,6 +29,7 @@ import apache_beam as beam
 from apache_beam import coders
 from apache_beam import pvalue
 from apache_beam import typehints
+from apache_beam.coders.coders import ToStringCoder
 from apache_beam.transforms.util import assert_that
 from apache_beam.transforms.util import equal_to
 from apache_beam.options.pipeline_options import PipelineOptions
@@ -422,7 +423,7 @@ class SnippetsTest(unittest.TestCase):
       def __init__(self, file_to_write):
         self.file_to_write = file_to_write
         self.file_obj = None
-        self.coder = coders.ToStringCoder()
+        self.coder = ToStringCoder()
 
       def start_bundle(self):
         assert self.file_to_write

http://git-wip-us.apache.org/repos/asf/beam/blob/7fd012b6/sdks/python/apache_beam/io/fileio_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/io/fileio_test.py b/sdks/python/apache_beam/io/fileio_test.py
index 4c25505..b92b8be 100644
--- a/sdks/python/apache_beam/io/fileio_test.py
+++ b/sdks/python/apache_beam/io/fileio_test.py
@@ -29,7 +29,7 @@ import hamcrest as hc
 import mock
 
 import apache_beam as beam
-from apache_beam import coders
+from apache_beam.coders import coders
 from apache_beam.io import fileio
 from apache_beam.testing.test_pipeline import TestPipeline
 from apache_beam.transforms.display import DisplayData

http://git-wip-us.apache.org/repos/asf/beam/blob/7fd012b6/sdks/python/apache_beam/io/textio.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/io/textio.py b/sdks/python/apache_beam/io/textio.py
index 750ec45..d43f4fc 100644
--- a/sdks/python/apache_beam/io/textio.py
+++ b/sdks/python/apache_beam/io/textio.py
@@ -21,7 +21,7 @@
 from __future__ import absolute_import
 import logging
 
-from apache_beam import coders
+from apache_beam.coders import coders
 from apache_beam.io import filebasedsource
 from apache_beam.io import fileio
 from apache_beam.io import iobase

http://git-wip-us.apache.org/repos/asf/beam/blob/7fd012b6/sdks/python/apache_beam/transforms/window.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/transforms/window.py b/sdks/python/apache_beam/transforms/window.py
index 29994c0..6d0db3a 100644
--- a/sdks/python/apache_beam/transforms/window.py
+++ b/sdks/python/apache_beam/transforms/window.py
@@ -53,7 +53,7 @@ import abc
 
 from google.protobuf import struct_pb2
 
-from apache_beam import coders
+from apache_beam.coders import coders
 from apache_beam.runners.api import beam_runner_api_pb2
 from apache_beam.transforms import timeutil
 from apache_beam.utils import proto_utils