Fixing error with PipelineOptions DisplayData of lists
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/c127f7a9 Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/c127f7a9 Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/c127f7a9 Branch: refs/heads/python-sdk Commit: c127f7a9e6f9885bc595e1462b27b6961c0f465d Parents: 50afc87 Author: Pablo <[email protected]> Authored: Mon Nov 21 11:15:37 2016 -0800 Committer: bchambers <[email protected]> Committed: Mon Nov 21 16:21:26 2016 -0800 ---------------------------------------------------------------------- sdks/python/apache_beam/internal/apiclient.py | 2 +- .../apache_beam/internal/apiclient_test.py | 1 + sdks/python/apache_beam/transforms/display.py | 27 ++++++++++++++++++++ .../apache_beam/transforms/display_test.py | 17 ++++++++++++ 4 files changed, 46 insertions(+), 1 deletion(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c127f7a9/sdks/python/apache_beam/internal/apiclient.py ---------------------------------------------------------------------- diff --git a/sdks/python/apache_beam/internal/apiclient.py b/sdks/python/apache_beam/internal/apiclient.py index 8992ec3..5612631 100644 --- a/sdks/python/apache_beam/internal/apiclient.py +++ b/sdks/python/apache_beam/internal/apiclient.py @@ -242,7 +242,7 @@ class Environment(object): dataflow.Environment.SdkPipelineOptionsValue.AdditionalProperty( key='options', value=to_json_value(options_dict))) - dd = DisplayData.create_from(options) + dd = DisplayData.create_from_options(options) items = [item.get_dict() for item in dd.items] self.proto.sdkPipelineOptions.additionalProperties.append( dataflow.Environment.SdkPipelineOptionsValue.AdditionalProperty( http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c127f7a9/sdks/python/apache_beam/internal/apiclient_test.py ---------------------------------------------------------------------- diff --git a/sdks/python/apache_beam/internal/apiclient_test.py b/sdks/python/apache_beam/internal/apiclient_test.py index 8fddae7..66cc8db 100644 --- a/sdks/python/apache_beam/internal/apiclient_test.py +++ b/sdks/python/apache_beam/internal/apiclient_test.py @@ -31,5 +31,6 @@ class UtilTest(unittest.TestCase): pipeline_options, DataflowPipelineRunner.BATCH_ENVIRONMENT_MAJOR_VERSION) + if __name__ == '__main__': unittest.main() http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c127f7a9/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 365abaf..c38fd9b 100644 --- a/sdks/python/apache_beam/transforms/display.py +++ b/sdks/python/apache_beam/transforms/display.py @@ -108,6 +108,33 @@ class DisplayData(object): key=key)) @classmethod + def create_from_options(cls, pipeline_options): + """ Creates DisplayData from a PipelineOptions instance. + + When creating DisplayData, this method will convert the value of any + item of a non-supported type to its string representation. + The normal DisplayData.create_from method rejects those items. + + Returns: + A DisplayData instance with populated items. + + Raises: + ValueError: If the has_display_data argument is not an instance of + HasDisplayData. + """ + from apache_beam.utils.options import PipelineOptions + if not isinstance(pipeline_options, PipelineOptions): + raise ValueError( + 'Element of class {}.{} does not subclass PipelineOptions' + .format(pipeline_options.__module__, + pipeline_options.__class__.__name__)) + + items = {k: (v if DisplayDataItem._get_value_type(v) is not None + else str(v)) + for k, v in pipeline_options.display_data().items()} + return cls(pipeline_options._namespace(), items) + + @classmethod def create_from(cls, has_display_data): """ Creates DisplayData from a HasDisplayData instance. http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c127f7a9/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 d514065..fc50abe 100644 --- a/sdks/python/apache_beam/transforms/display_test.py +++ b/sdks/python/apache_beam/transforms/display_test.py @@ -29,6 +29,7 @@ import apache_beam as beam from apache_beam.transforms.display import HasDisplayData from apache_beam.transforms.display import DisplayData from apache_beam.transforms.display import DisplayDataItem +from apache_beam.utils.options import PipelineOptions class DisplayDataItemMatcher(BaseMatcher): @@ -105,6 +106,22 @@ class DisplayDataTest(unittest.TestCase): self.assertTrue(isinstance(display_dofn, HasDisplayData)) self.assertEqual(display_dofn.display_data(), {}) + def test_unsupported_type_display_data(self): + class MyDisplayComponent(HasDisplayData): + def display_data(self): + return {'item_key': 'item_value'} + + with self.assertRaises(ValueError): + DisplayData.create_from_options(MyDisplayComponent()) + + def test_create_list_display_data(self): + flags = ['--extra_package', 'package1', '--extra_package', 'package2'] + pipeline_options = PipelineOptions(flags=flags) + items = DisplayData.create_from_options(pipeline_options).items + hc.assert_that(items, hc.contains_inanyorder( + DisplayDataItemMatcher('extra_packages', + str(['package1', 'package2'])))) + def test_base_cases(self): """ Tests basic display data cases (key:value, key:dict) It does not test subcomponent inclusion
