def test_ptransform_override_replacement_inputs(self): class MyParDoOverride(PTransformOverride): def matches(self, applied_ptransform): return (isinstance(applied_ptransform.transform, ParDo) and isinstance(applied_ptransform.transform.fn, AddWithProductDoFn)) def get_replacement_transform(self, transform): return AddThenMultiply() def get_replacement_inputs(self, applied_ptransform): assert len(applied_ptransform.inputs) == 1 assert len(applied_ptransform.side_inputs) == 2 # Swap the order of the two side inputs return (applied_ptransform.inputs[0], applied_ptransform.side_inputs[1].pvalue, applied_ptransform.side_inputs[0].pvalue) p = Pipeline() pcoll1 = p | 'pc1' >> beam.Create([2]) pcoll2 = p | 'pc2' >> beam.Create([3]) pcoll3 = p | 'pc3' >> beam.Create([4, 5, 6]) result = pcoll3 | 'Operate' >> beam.ParDo( AddWithProductDoFn(), AsSingleton(pcoll1), AsSingleton(pcoll2)) assert_that(result, equal_to([14, 16, 18])) p.replace_all([MyParDoOverride()]) p.run()
def test_ptransform_override_multiple_inputs(self): class MyParDoOverride(PTransformOverride): def matches(self, applied_ptransform): return isinstance(applied_ptransform.transform, FlattenAndDouble) def get_replacement_transform(self, applied_ptransform): return FlattenAndTriple() p = Pipeline() pcoll1 = p | 'pc1' >> beam.Create([1, 2, 3]) pcoll2 = p | 'pc2' >> beam.Create([4, 5, 6]) pcoll3 = (pcoll1, pcoll2) | 'FlattenAndMultiply' >> FlattenAndDouble() assert_that(pcoll3, equal_to([3, 6, 9, 12, 15, 18])) p.replace_all([MyParDoOverride()]) p.run()
def test_streaming_create_translation(self): remote_runner = DataflowRunner() self.default_properties.append("--streaming") p = Pipeline(remote_runner, PipelineOptions(self.default_properties)) p | ptransform.Create([1]) # pylint: disable=expression-not-assigned remote_runner.job = apiclient.Job(p._options) # Performing configured PTransform overrides here. p.replace_all(DataflowRunner._PTRANSFORM_OVERRIDES) super(DataflowRunner, remote_runner).run(p) job_dict = json.loads(str(remote_runner.job)) self.assertEqual(len(job_dict[u'steps']), 2) self.assertEqual(job_dict[u'steps'][0][u'kind'], u'ParallelRead') self.assertEqual( job_dict[u'steps'][0][u'properties'][u'pubsub_subscription'], '_starting_signal/') self.assertEqual(job_dict[u'steps'][1][u'kind'], u'ParallelDo')
def test_ptransform_overrides(self): class MyParDoOverride(PTransformOverride): def matches(self, applied_ptransform): return isinstance(applied_ptransform.transform, DoubleParDo) def get_replacement_transform(self, ptransform): if isinstance(ptransform, DoubleParDo): return TripleParDo() raise ValueError('Unsupported type of transform: %r' % ptransform) p = Pipeline() pcoll = p | beam.Create([1, 2, 3]) | 'Multiply' >> DoubleParDo() assert_that(pcoll, equal_to([3, 6, 9])) p.replace_all([MyParDoOverride()]) p.run()
def test_ptransform_override_side_inputs(self): class MyParDoOverride(PTransformOverride): def matches(self, applied_ptransform): return (isinstance(applied_ptransform.transform, ParDo) and isinstance(applied_ptransform.transform.fn, AddWithProductDoFn)) def get_replacement_transform(self, transform): return AddThenMultiply() p = Pipeline() pcoll1 = p | 'pc1' >> beam.Create([2]) pcoll2 = p | 'pc2' >> beam.Create([3]) pcoll3 = p | 'pc3' >> beam.Create([4, 5, 6]) result = pcoll3 | 'Operate' >> beam.ParDo( AddWithProductDoFn(), AsSingleton(pcoll1), AsSingleton(pcoll2)) assert_that(result, equal_to([18, 21, 24])) p.replace_all([MyParDoOverride()]) p.run()