Repository: incubator-beam Updated Branches: refs/heads/python-sdk 811858bca -> 00b17e542
Compare display data items, not dicts. Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/9d1cfa1a Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/9d1cfa1a Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/9d1cfa1a Branch: refs/heads/python-sdk Commit: 9d1cfa1a565137dc2ca968700e8b9f3d5d1f392e Parents: 811858b Author: Robert Bradshaw <rober...@google.com> Authored: Thu Oct 27 11:04:20 2016 -0700 Committer: Robert Bradshaw <rober...@gmail.com> Committed: Fri Oct 28 16:10:01 2016 -0700 ---------------------------------------------------------------------- sdks/python/apache_beam/transforms/display.py | 13 ++++-- .../apache_beam/transforms/display_test.py | 44 +++++++++++--------- 2 files changed, 35 insertions(+), 22 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9d1cfa1a/sdks/python/apache_beam/transforms/display.py ---------------------------------------------------------------------- diff --git a/sdks/python/apache_beam/transforms/display.py b/sdks/python/apache_beam/transforms/display.py index 71fd34e..e93d560 100644 --- a/sdks/python/apache_beam/transforms/display.py +++ b/sdks/python/apache_beam/transforms/display.py @@ -199,6 +199,12 @@ class DisplayDataItem(object): return self.get_dict() == other.get_dict() return False + def __ne__(self, other): + return not self == other + + def __hash__(self): + return hash(tuple(sorted(self.get_dict().items()))) + @classmethod def _format_value(cls, value, type_): """ Returns the API representation of a value given its type. @@ -213,9 +219,9 @@ class DisplayDataItem(object): res = value if type_ == 'CLASS': res = '{}.{}'.format(value.__module__, value.__name__) - if type_ == 'DURATION': + elif type_ == 'DURATION': res = value.total_seconds()*1000 - if type_ == 'TIMESTAMP': + elif type_ == 'TIMESTAMP': res = calendar.timegm(value.timetuple())*1000 + value.microsecond//1000 return res @@ -232,7 +238,8 @@ class DisplayDataItem(object): """ if type_ == 'CLASS': return value.__name__ - return None + else: + return None @classmethod def _get_value_type(cls, value): http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9d1cfa1a/sdks/python/apache_beam/transforms/display_test.py ---------------------------------------------------------------------- diff --git a/sdks/python/apache_beam/transforms/display_test.py b/sdks/python/apache_beam/transforms/display_test.py index 013172f..87d0920 100644 --- a/sdks/python/apache_beam/transforms/display_test.py +++ b/sdks/python/apache_beam/transforms/display_test.py @@ -23,7 +23,9 @@ from datetime import datetime import unittest import apache_beam as beam -from apache_beam.transforms.display import HasDisplayData, DisplayData, DisplayDataItem +from apache_beam.transforms.display import HasDisplayData +from apache_beam.transforms.display import DisplayData +from apache_beam.transforms.display import DisplayDataItem class DisplayDataTest(unittest.TestCase): @@ -68,26 +70,30 @@ class DisplayDataTest(unittest.TestCase): now = datetime.now() fn = MyDoFn(my_display_data=now) dd = DisplayData.create_from(fn) - dd_dicts = sorted([item.get_dict() for item in dd.items], - key=lambda x: x['namespace']+x['key']) nspace = '{}.{}'.format(fn.__module__, fn.__class__.__name__) - expected_items = sorted([ - {'url': 'http://github.com', 'namespace': nspace, - 'value': 'github.com', 'label': 'The URL', - 'key': 'complex_url', 'type': 'STRING'}, - {'type': 'TIMESTAMP', 'namespace': nspace, 'key': 'my_dd', - 'value': DisplayDataItem._format_value(now, 'TIMESTAMP')}, - {'type': 'STRING', 'namespace': nspace, - 'shortValue': 'HasDisplayData', 'key': 'python_class', - 'value': 'apache_beam.transforms.display.HasDisplayData'}, - {'type': 'INTEGER', 'namespace': nspace, - 'value': 120, 'key': 'static_integer'}, - {'type': 'STRING', 'namespace': nspace, - 'value': 'static me!', 'key': 'static_string'}], - key=lambda x: x['namespace']+x['key']) - - self.assertEqual(dd_dicts, expected_items) + expected_items = set([ + DisplayDataItem(namespace=nspace, + key='complex_url', + value='github.com', + label='The URL', + url='http://github.com'), + DisplayDataItem(namespace=nspace, + key='my_dd', + value=now), + DisplayDataItem(namespace=nspace, + key='python_class', + shortValue='HasDisplayData', + value='apache_beam.transforms.display.HasDisplayData'), + DisplayDataItem(namespace=nspace, + key='static_integer', + value=120), + DisplayDataItem(namespace=nspace, + key='static_string', + value='static me!'), + ]) + + self.assertEqual(set(dd.items), expected_items) def test_subcomponent(self): class SpecialParDo(beam.PTransform):