def expand(self, dataset): """Transform the dataset by applying the preprocessing_fn. Args: dataset: A dataset. Returns: A (Dataset, TransformFn) pair containing the preprocessed dataset and the graph that maps the input to the output data. """ # Expand is currently implemented by composing AnalyzeDataset and # TransformDataset. Future versions however could do somthing more optimal, # e.g. caching the values of expensive computations done in AnalyzeDataset. transform_fn = ( dataset | 'AnalyzeDataset' >> AnalyzeDataset(self._preprocessing_fn)) if Context.get_use_deep_copy_optimization(): data, metadata = dataset # obviates unnecessary data materialization when the input data source is # safe to read more than once. tf.logging.info( 'Deep copying the dataset before applying transformation') dataset = (deep_copy.deep_copy(data), metadata) transformed_dataset = ((dataset, transform_fn) | 'TransformDataset' >> TransformDataset()) return transformed_dataset, transform_fn
def testSideInputNotCopied(self): with beam.Pipeline() as p: side = (p | 'CreateSide' >> beam.Create(['s1', 's2', 's3']) | beam.Map(lambda x: DeepCopyTest._CountingIdentityFn( 'SideInput', x))) main = ( p | 'CreateMain' >> beam.Create([1, 2, 3]) | beam.Map(lambda x: DeepCopyTest._CountingIdentityFn('Main', x)) | beam.Map(lambda e, s: (e, list(s)), pvalue.AsList(side))) copied = deep_copy.deep_copy(main) # Check that deep copy was performed. self.assertIsNot(main, copied) self.assertIsNot(main.producer, copied.producer) # Check that deep copy stops at the side input materialization boundary. self.assertIs(main.producer.side_inputs[0], copied.producer.side_inputs[0]) self.assertIs(main.producer.side_inputs[0].pvalue, side) # Check counts of processed items. self.assertEqual(DeepCopyTest._counts['SideInput'], 3) self.assertEqual(DeepCopyTest._counts['Main'], 6)
def expand(self, inputs): # We don't deep_copy pcollections used for the first phase, or when # the user defined `Context` disables it. if self._phase > 0 and Context.get_use_deep_copy_optimization(): # Obviates unnecessary data materialization when the input data source is # safe to read more than once. tf.compat.v1.logging.info('Deep copying inputs for phase: %d', self._phase) input_values = deep_copy.deep_copy(self._input_values_pcoll) else: input_values = self._input_values_pcoll if not self._use_tfxio: input_values |= 'BatchInputs' >> _BatchElements() return (input_values | 'ApplySavedModel' >> beam.ParDo( _RunMetaGraphDoFn( self._tf_config, use_tfxio=self._use_tfxio, input_schema=self._input_schema, input_tensor_adapter_config=self._input_tensor_adapter_config, shared_graph_state_handle=shared.Shared(), passthrough_keys=Context.get_passthrough_keys()), saved_model_dir=beam.pvalue.AsSingleton(inputs[0])))
def testFlatten(self): with beam.Pipeline() as p: create_1 = p | 'Create1' >> beam.Create([(1, 'a'), (2, 'b')]) create_2 = p | 'Create2' >> beam.Create([(3, 'c')]) created = (create_1, create_2) | 'Flatten1' >> beam.Flatten() grouped1 = ( created | 'PreGroup1' >> beam.Map( lambda x: DeepCopyTest._CountingIdentityFn('PreGroup1', x)) | 'GBK1' >> beam.GroupByKey()) grouped2 = ( p | beam.Create([(1, 'a'), (2, 'b'), (3, 'c')]) | 'PreGroup2' >> beam.Map( lambda x: DeepCopyTest._CountingIdentityFn('PreGroup2', x)) | 'GBK2' >> beam.GroupByKey()) modified1 = ( grouped1 | 'Add1' >> beam.Map(lambda (x, y): DeepCopyTest. _CountingIdentityFn('Add1', (x + 1, y)))) modified2 = ( grouped2 | 'Add2' >> beam.Map(lambda (x, y): DeepCopyTest. _CountingIdentityFn('Add2', (x + 1, y)))) flattened = (modified1, modified2) | 'Flatten2' >> beam.Flatten() modified3 = ( flattened | 'Add3' >> beam.Map(lambda (x, y): DeepCopyTest. _CountingIdentityFn('Add3', (x + 1, y)))) copied = deep_copy.deep_copy(modified3) # Check that deep copy was performed. self.assertNotEqual(copied.producer.inputs[0], modified3.producer.inputs[0]) self.assertNotEqual( copied.producer.inputs[0].producer.inputs[0], modified3.producer.inputs[0].producer.inputs[0]) self.assertNotEqual( copied.producer.inputs[0].producer.inputs[1], modified3.producer.inputs[0].producer.inputs[1]) # Check that copy stops at materialization boundary. self.assertEqual( copied.producer.inputs[0].producer.inputs[0].producer. inputs[0], modified3.producer.inputs[0].producer.inputs[0]. producer.inputs[0]) self.assertEqual( copied.producer.inputs[0].producer.inputs[1].producer. inputs[0], modified3.producer.inputs[0].producer.inputs[1]. producer.inputs[0]) # Check counts of processed items. self.assertEqual(DeepCopyTest._counts['PreGroup1'], 3) self.assertEqual(DeepCopyTest._counts['PreGroup2'], 3) self.assertEqual(DeepCopyTest._counts['Add1'], 6) self.assertEqual(DeepCopyTest._counts['Add2'], 6) self.assertEqual(DeepCopyTest._counts['Add3'], 12)
def _maybe_deep_copy_pcollection_inputs(self, inputs): input_values, tensor_pcoll_mapping = inputs # We don't deep_copy pcollections used for the first phase, or when # the user defined `Context` disables it. if self._phase_index > 0 and Context.get_use_deep_copy_optimization(): # obviates unnecessary data materialization when the input data source is # safe to read more than once. tf.logging.info('Deep copying inputs for: %s', [a.name for a in self._analyzer_infos]) input_values = deep_copy.deep_copy(input_values) return input_values, tensor_pcoll_mapping
def testCombineGlobally(self): with beam.Pipeline() as p: combined = ( p | beam.Create([1, 2, 3]) | beam.Map(lambda x: DeepCopyTest._CountingIdentityFn( 'PreCombine', x)) | beam.WindowInto(beam.window.FixedWindows(5, 0)) | beam.CombineGlobally(beam.transforms.combiners.MeanCombineFn( )).without_defaults() | beam.Map(lambda x: DeepCopyTest._CountingIdentityFn( 'PostCombine', x))) copied = deep_copy.deep_copy(combined) # Check that deep copy was performed. self.assertIsNot(combined, copied) self.assertIsNot(combined.producer.inputs[0], copied.producer.inputs[0]) self.assertEqual(combined.producer.inputs[0].producer.full_label, 'CombineGlobally(MeanCombineFn)/UnKey') self.assertEqual(copied.producer.inputs[0].producer.full_label, 'CombineGlobally(MeanCombineFn)/UnKey.Copy') # Check that deep copy stops at materialization boundary. self.assertIs(combined.producer.inputs[0].producer.inputs[0], copied.producer.inputs[0].producer.inputs[0]) self.assertEqual( str(combined.producer.inputs[0].producer.inputs[0]), ('PCollection[CombineGlobally(MeanCombineFn)/CombinePerKey/Combine/' 'ParDo(CombineValuesDoFn).None]')) self.assertIs( combined.producer.inputs[0].producer.inputs[0].producer, copied.producer.inputs[0].producer.inputs[0].producer) self.assertEqual( copied.producer.inputs[0].producer.inputs[0].producer. full_label, ('CombineGlobally(MeanCombineFn)/CombinePerKey/Combine/' 'ParDo(CombineValuesDoFn)')) # Check counts of processed items. self.assertEqual(DeepCopyTest._counts['PreCombine'], 3) self.assertEqual(DeepCopyTest._counts['PostCombine'], 2)
def testBasicDeepCopy(self): with beam.Pipeline() as p: grouped = (p | beam.Create([(1, 'a'), (2, 'b'), (3, 'c')]) | beam.Map( lambda x: DeepCopyTest._CountingIdentityFn( 'PreGroup', x)) | beam.GroupByKey()) modified = ( grouped | 'Add1' >> beam.Map(DeepCopyTest._MakeAdd1CountingIdentityFn('Add1')) | 'Add2' >> beam.Map(DeepCopyTest._MakeAdd1CountingIdentityFn('Add2'))) copied = deep_copy.deep_copy(modified) # pylint: disable=expression-not-assigned modified | 'Add3' >> beam.Map( DeepCopyTest._MakeAdd1CountingIdentityFn('Add3')) # pylint: enable=expression-not-assigned # Check labels. self.assertEqual(copied.producer.full_label, 'Add2.Copy') self.assertEqual(copied.producer.inputs[0].producer.full_label, 'Add1.Copy') # Check that deep copy was performed. self.assertIsNot(copied.producer.inputs[0], modified.producer.inputs[0]) # Check that copy stops at materialization boundary. self.assertIs(copied.producer.inputs[0].producer.inputs[0], modified.producer.inputs[0].producer.inputs[0]) # Check counts of processed items. self.assertEqual(DeepCopyTest._counts['PreGroup'], 3) self.assertEqual(DeepCopyTest._counts['Add1'], 6) self.assertEqual(DeepCopyTest._counts['Add2'], 6) self.assertEqual(DeepCopyTest._counts['Add3'], 3)