Repository: incubator-beam
Updated Branches:
  refs/heads/python-sdk f19f767b0 -> d5e8c79a3


Display data keys in Python should be snake_case

This fixes existing mis-cased display data keys.


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

Branch: refs/heads/python-sdk
Commit: 0dcc9bc03772d8bd5ab1e0d138c445e9418ebf66
Parents: f19f767
Author: Pablo <pabl...@google.com>
Authored: Mon Nov 28 13:05:47 2016 -0800
Committer: bchambers <bchamb...@google.com>
Committed: Tue Dec 6 17:17:16 2016 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/io/avroio.py            |  8 +++-----
 sdks/python/apache_beam/io/avroio_test.py       |  8 ++++----
 sdks/python/apache_beam/io/filebasedsource.py   |  3 ++-
 .../apache_beam/io/filebasedsource_test.py      |  4 ++--
 sdks/python/apache_beam/io/fileio.py            |  8 ++++----
 sdks/python/apache_beam/io/fileio_test.py       | 12 ++++++------
 sdks/python/apache_beam/io/pubsub.py            |  2 +-
 sdks/python/apache_beam/io/pubsub_test.py       |  2 +-
 sdks/python/apache_beam/io/textio.py            | 20 ++++++++++----------
 sdks/python/apache_beam/io/textio_test.py       |  8 +++++---
 .../apache_beam/transforms/combiners_test.py    |  8 ++++----
 sdks/python/apache_beam/transforms/core.py      |  8 ++++----
 12 files changed, 46 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0dcc9bc0/sdks/python/apache_beam/io/avroio.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/io/avroio.py 
b/sdks/python/apache_beam/io/avroio.py
index 6cba12d..35d0e94 100644
--- a/sdks/python/apache_beam/io/avroio.py
+++ b/sdks/python/apache_beam/io/avroio.py
@@ -75,8 +75,7 @@ class ReadFromAvro(PTransform):
       **kwargs: Additional keyword arguments to be passed to the base class.
     """
     super(ReadFromAvro, self).__init__()
-    self._args = (file_pattern, min_bundle_size)
-    self._source = _AvroSource(*self._args, validate=validate)
+    self._source = _AvroSource(file_pattern, min_bundle_size, 
validate=validate)
 
   def apply(self, pvalue):
     return pvalue.pipeline | Read(self._source)
@@ -292,9 +291,8 @@ class WriteToAvro(beam.transforms.PTransform):
     Returns:
       A WriteToAvro transform usable for writing.
     """
-    self._args = (file_path_prefix, schema, codec, file_name_suffix, 
num_shards,
-                  shard_name_template, mime_type)
-    self._sink = _AvroSink(*self._args)
+    self._sink = _AvroSink(file_path_prefix, schema, codec, file_name_suffix,
+                           num_shards, shard_name_template, mime_type)
 
   def apply(self, pcoll):
     return pcoll | beam.io.iobase.Write(self._sink)

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0dcc9bc0/sdks/python/apache_beam/io/avroio_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/io/avroio_test.py 
b/sdks/python/apache_beam/io/avroio_test.py
index e8fb12b..e5db196 100644
--- a/sdks/python/apache_beam/io/avroio_test.py
+++ b/sdks/python/apache_beam/io/avroio_test.py
@@ -157,7 +157,7 @@ class TestAvro(unittest.TestCase):
     # No extra avro parameters for AvroSource.
     expected_items = [
         DisplayDataItemMatcher('compression', 'auto'),
-        DisplayDataItemMatcher('filePattern', file_name)]
+        DisplayDataItemMatcher('file_pattern', file_name)]
     hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))
 
   def test_read_display_data(self):
@@ -168,7 +168,7 @@ class TestAvro(unittest.TestCase):
     # No extra avro parameters for AvroSource.
     expected_items = [
         DisplayDataItemMatcher('compression', 'auto'),
-        DisplayDataItemMatcher('filePattern', file_name)]
+        DisplayDataItemMatcher('file_pattern', file_name)]
     hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))
 
   def test_sink_display_data(self):
@@ -186,7 +186,7 @@ class TestAvro(unittest.TestCase):
             'schema',
             str(self.SCHEMA)),
         DisplayDataItemMatcher(
-            'filePattern',
+            'file_pattern',
             'some_avro_sink-%(shard_num)05d-of-%(num_shards)05d.end'),
         DisplayDataItemMatcher(
             'shards',
@@ -209,7 +209,7 @@ class TestAvro(unittest.TestCase):
             'schema',
             str(self.SCHEMA)),
         DisplayDataItemMatcher(
-            'filePattern',
+            'file_pattern',
             'some_avro_sink-%(shard_num)05d-of-%(num_shards)05d'),
         DisplayDataItemMatcher(
             'shards',

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0dcc9bc0/sdks/python/apache_beam/io/filebasedsource.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/io/filebasedsource.py 
b/sdks/python/apache_beam/io/filebasedsource.py
index 8921801..1bfde25 100644
--- a/sdks/python/apache_beam/io/filebasedsource.py
+++ b/sdks/python/apache_beam/io/filebasedsource.py
@@ -99,7 +99,8 @@ class FileBasedSource(iobase.BoundedSource):
       self._validate()
 
   def display_data(self):
-    return {'filePattern': DisplayDataItem(self._pattern, label="File 
Pattern"),
+    return {'file_pattern': DisplayDataItem(self._pattern,
+                                            label="File Pattern"),
             'compression': DisplayDataItem(str(self._compression_type),
                                            label='Compression Type')}
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0dcc9bc0/sdks/python/apache_beam/io/filebasedsource_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/io/filebasedsource_test.py 
b/sdks/python/apache_beam/io/filebasedsource_test.py
index a455cd3..ab7c69f 100644
--- a/sdks/python/apache_beam/io/filebasedsource_test.py
+++ b/sdks/python/apache_beam/io/filebasedsource_test.py
@@ -253,7 +253,7 @@ class TestFileBasedSource(unittest.TestCase):
     fbs = LineSource(file_name)
     dd = DisplayData.create_from(fbs)
     expected_items = [
-        DisplayDataItemMatcher('filePattern', file_name),
+        DisplayDataItemMatcher('file_pattern', file_name),
         DisplayDataItemMatcher('compression', 'auto')]
     hc.assert_that(dd.items,
                    hc.contains_inanyorder(*expected_items))
@@ -586,7 +586,7 @@ class TestSingleFileSource(unittest.TestCase):
     dd = DisplayData.create_from(fbs)
     expected_items = [
         DisplayDataItemMatcher('compression', 'auto'),
-        DisplayDataItemMatcher('filePattern', file_name)]
+        DisplayDataItemMatcher('file_pattern', file_name)]
     hc.assert_that(dd.items,
                    hc.contains_inanyorder(*expected_items))
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0dcc9bc0/sdks/python/apache_beam/io/fileio.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/io/fileio.py 
b/sdks/python/apache_beam/io/fileio.py
index 82e7813..d7ff35b 100644
--- a/sdks/python/apache_beam/io/fileio.py
+++ b/sdks/python/apache_beam/io/fileio.py
@@ -168,8 +168,8 @@ class NativeFileSource(dataflow_io.NativeSource):
     self.mime_type = mime_type
 
   def display_data(self):
-    return {'filePattern': DisplayDataItem(self.file_path,
-                                           label="File Pattern"),
+    return {'file_pattern': DisplayDataItem(self.file_path,
+                                            label="File Pattern"),
             'compression': DisplayDataItem(str(self.compression_type),
                                            label='Compression')}
 
@@ -845,7 +845,7 @@ class FileSink(iobase.Sink):
             DisplayDataItem(self.num_shards, label='Number of Shards'),
             'compression':
             DisplayDataItem(str(self.compression_type)),
-            'filePattern':
+            'file_pattern':
             DisplayDataItem('{}{}{}'.format(self.file_path_prefix,
                                             self.shard_name_format,
                                             self.file_name_suffix),
@@ -1136,7 +1136,7 @@ class NativeFileSink(dataflow_io.NativeSink):
     return {'shards':
             DisplayDataItem(self.num_shards,
                             label='Number of Shards'),
-            'filePattern':
+            'file_pattern':
             DisplayDataItem(file_name_pattern,
                             label='File Name Pattern'),
             'compression':

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0dcc9bc0/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 a68d484..dbf0e69 100644
--- a/sdks/python/apache_beam/io/fileio_test.py
+++ b/sdks/python/apache_beam/io/fileio_test.py
@@ -66,7 +66,7 @@ class TestTextFileSource(unittest.TestCase):
     self.assertEqual(read_lines, output_lines)
     dd = DisplayData.create_from(source)
     expected_items = [
-        DisplayDataItemMatcher('filePattern', file_name),
+        DisplayDataItemMatcher('file_pattern', file_name),
         DisplayDataItemMatcher('compression', 'auto')]
     hc.assert_that(dd.items,
                    hc.contains_inanyorder(*expected_items))
@@ -647,7 +647,7 @@ class TestNativeTextFileSink(unittest.TestCase):
     dd = DisplayData.create_from(sink)
     expected_items = [
         DisplayDataItemMatcher(
-            'filePattern',
+            'file_pattern',
             '{}{}'.format(self.path, '-SSSSS-of-NNNNN')),
         DisplayDataItemMatcher(
             'compression',
@@ -662,7 +662,7 @@ class TestNativeTextFileSink(unittest.TestCase):
     dd = DisplayData.create_from(sink)
     expected_items = [
         DisplayDataItemMatcher(
-            'filePattern',
+            'file_pattern',
             '{}{}{}'.format(self.path, '-SSSSS-of-NNNNN', '.pdf')),
         DisplayDataItemMatcher(
             'compression',
@@ -693,7 +693,7 @@ class TestNativeTextFileSink(unittest.TestCase):
     dd = DisplayData.create_from(sink)
     expected_items = [
         DisplayDataItemMatcher(
-            'filePattern',
+            'file_pattern',
             '{}{}'.format(self.path, '-SSSSS-of-NNNNN')),
         DisplayDataItemMatcher(
             'compression',
@@ -733,7 +733,7 @@ class TestNativeTextFileSink(unittest.TestCase):
     dd = DisplayData.create_from(sink)
     expected_items = [
         DisplayDataItemMatcher(
-            'filePattern',
+            'file_pattern',
             '{}{}'.format(self.path, '-SSSSS-of-NNNNN')),
         DisplayDataItemMatcher(
             'compression',
@@ -870,7 +870,7 @@ class TestFileSink(unittest.TestCase):
         DisplayDataItemMatcher(
             'compression', 'auto'),
         DisplayDataItemMatcher(
-            'filePattern',
+            'file_pattern',
             '{}{}'.format(temp_path,
                           '-%(shard_num)05d-of-%(num_shards)05d.foo'))]
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0dcc9bc0/sdks/python/apache_beam/io/pubsub.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/io/pubsub.py 
b/sdks/python/apache_beam/io/pubsub.py
index 40bd368..efc628d 100644
--- a/sdks/python/apache_beam/io/pubsub.py
+++ b/sdks/python/apache_beam/io/pubsub.py
@@ -56,7 +56,7 @@ class PubSubSource(dataflow_io.NativeSource):
     return 'pubsub'
 
   def display_data(self):
-    return {'idLabel':
+    return {'id_label':
             DisplayDataItem(self.id_label,
                             label='ID Label Attribute').drop_if_none(),
             'topic':

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0dcc9bc0/sdks/python/apache_beam/io/pubsub_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/io/pubsub_test.py 
b/sdks/python/apache_beam/io/pubsub_test.py
index 828d233..11b5d50 100644
--- a/sdks/python/apache_beam/io/pubsub_test.py
+++ b/sdks/python/apache_beam/io/pubsub_test.py
@@ -35,7 +35,7 @@ class TestPubSubSource(unittest.TestCase):
     expected_items = [
         DisplayDataItemMatcher('topic', 'a_topic'),
         DisplayDataItemMatcher('subscription', 'a_subscription'),
-        DisplayDataItemMatcher('idLabel', 'a_label')]
+        DisplayDataItemMatcher('id_label', 'a_label')]
 
     hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0dcc9bc0/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 bb664d1..ebadf69 100644
--- a/sdks/python/apache_beam/io/textio.py
+++ b/sdks/python/apache_beam/io/textio.py
@@ -234,11 +234,7 @@ class ReadFromText(PTransform):
     """
 
     super(ReadFromText, self).__init__(**kwargs)
-    self._file_pattern = file_pattern
-    self._min_bundle_size = min_bundle_size
-    self._compression_type = compression_type
     self._strip_trailing_newlines = strip_trailing_newlines
-    self._coder = coder
     self._source = _TextSource(file_pattern, min_bundle_size, compression_type,
                                strip_trailing_newlines, coder,
                                validate=validate)
@@ -248,8 +244,8 @@ class ReadFromText(PTransform):
 
   def display_data(self):
     return {'source_dd': self._source,
-            'strip_nwln': DisplayDataItem(self._strip_trailing_newlines,
-                                          label='Strip Trailing New Lines')}
+            'strip_newline': DisplayDataItem(self._strip_trailing_newlines,
+                                             label='Strip Trailing New Lines')}
 
 
 class WriteToText(PTransform):
@@ -296,12 +292,16 @@ class WriteToText(PTransform):
           compression.
     """
 
-    self._args = (file_path_prefix, file_name_suffix, append_trailing_newlines,
-                  num_shards, shard_name_template, coder, compression_type)
-    self._sink = _TextSink(*self._args)
+    self._append_trailing_newlines = append_trailing_newlines
+    self._sink = _TextSink(file_path_prefix, file_name_suffix,
+                           append_trailing_newlines, num_shards,
+                           shard_name_template, coder, compression_type)
 
   def apply(self, pcoll):
     return pcoll | Write(self._sink)
 
   def display_data(self):
-    return {'sink_dd': self._sink}
+    return {'sink_dd': self._sink,
+            'append_newline': DisplayDataItem(
+                self._append_trailing_newlines,
+                label='Append Trailing New Lines')}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0dcc9bc0/sdks/python/apache_beam/io/textio_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/io/textio_test.py 
b/sdks/python/apache_beam/io/textio_test.py
index b1d3fb0..5784fef 100644
--- a/sdks/python/apache_beam/io/textio_test.py
+++ b/sdks/python/apache_beam/io/textio_test.py
@@ -257,8 +257,8 @@ class TextSourceTest(unittest.TestCase):
     dd = DisplayData.create_from(read)
     expected_items = [
         DisplayDataItemMatcher('compression', 'auto'),
-        DisplayDataItemMatcher('filePattern', 'prefix'),
-        DisplayDataItemMatcher('strip_nwln', True)]
+        DisplayDataItemMatcher('file_pattern', 'prefix'),
+        DisplayDataItemMatcher('strip_newline', True)]
     hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))
 
   def test_dataflow_single_file(self):
@@ -469,11 +469,13 @@ class TextSinkTest(unittest.TestCase):
     dd = DisplayData.create_from(write)
     expected_items = [
         DisplayDataItemMatcher(
+            'append_newline', True),
+        DisplayDataItemMatcher(
             'compression', 'auto'),
         DisplayDataItemMatcher(
             'shards', 0),
         DisplayDataItemMatcher(
-            'filePattern',
+            'file_pattern',
             '{}{}'.format('prefix',
                           '-%(shard_num)05d-of-%(num_shards)05d'))]
     hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0dcc9bc0/sdks/python/apache_beam/transforms/combiners_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/transforms/combiners_test.py 
b/sdks/python/apache_beam/transforms/combiners_test.py
index d28c63f..8dc274e 100644
--- a/sdks/python/apache_beam/transforms/combiners_test.py
+++ b/sdks/python/apache_beam/transforms/combiners_test.py
@@ -146,7 +146,7 @@ class CombineTest(unittest.TestCase):
       transform = beam.CombinePerKey(combineFn)
       dd = DisplayData.create_from(transform)
       expected_items = [
-          DisplayDataItemMatcher('combineFn', combineFn.__class__),
+          DisplayDataItemMatcher('combine_fn', combineFn.__class__),
           DisplayDataItemMatcher('n', combineFn._n),
           DisplayDataItemMatcher('compare', combineFn._compare.__name__)]
       hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))
@@ -164,7 +164,7 @@ class CombineTest(unittest.TestCase):
         dd = DisplayData.create_from(transform)
         expected_items = [
             DisplayDataItemMatcher('fn', sampleFn.fn.__name__),
-            DisplayDataItemMatcher('combineFn',
+            DisplayDataItemMatcher('combine_fn',
                                    transform.fn.__class__)]
         if len(args) > 0:
           expected_items.append(
@@ -185,7 +185,7 @@ class CombineTest(unittest.TestCase):
     transform = beam.CombineGlobally(combine.Smallest(5))
     dd = DisplayData.create_from(transform)
     expected_items = [
-        DisplayDataItemMatcher('combineFn', combine.Smallest),
+        DisplayDataItemMatcher('combine_fn', combine.Smallest),
         DisplayDataItemMatcher('n', 5),
         DisplayDataItemMatcher('compare', 'gt')]
     hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))
@@ -195,7 +195,7 @@ class CombineTest(unittest.TestCase):
         combine.TupleCombineFn(max, combine.MeanCombineFn(), sum))
     dd = DisplayData.create_from(transform)
     expected_items = [
-        DisplayDataItemMatcher('combineFn', combine.TupleCombineFn),
+        DisplayDataItemMatcher('combine_fn', combine.TupleCombineFn),
         DisplayDataItemMatcher('combiners',
                                "['max', 'MeanCombineFn', 'sum']")]
     hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0dcc9bc0/sdks/python/apache_beam/transforms/core.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/transforms/core.py 
b/sdks/python/apache_beam/transforms/core.py
index ffcdd10..523c5a6 100644
--- a/sdks/python/apache_beam/transforms/core.py
+++ b/sdks/python/apache_beam/transforms/core.py
@@ -832,9 +832,9 @@ class CombineGlobally(PTransform):
     self.kwargs = kwargs
 
   def display_data(self):
-    return {'combineFn':
+    return {'combine_fn':
             DisplayDataItem(self.fn.__class__, label='Combine Function'),
-            'combineFn_dd':
+            'combine_fn_dd':
             self.fn}
 
   def default_label(self):
@@ -924,9 +924,9 @@ class CombinePerKey(PTransformWithSideInputs):
     A PObject holding the result of the combine operation.
   """
   def display_data(self):
-    return {'combineFn':
+    return {'combine_fn':
             DisplayDataItem(self.fn.__class__, label='Combine Function'),
-            'combineFn_dd':
+            'combine_fn_dd':
             self.fn}
 
   def make_fn(self, fn):

Reply via email to