def test_batch_with_df_transform(self): with TestPipeline() as p: res = ( p | beam.Create([ Animal('Falcon', 380.0), Animal('Falcon', 370.0), Animal('Parrot', 24.0), Animal('Parrot', 26.0) ]) | schemas.BatchRowsAsDataFrame() | transforms.DataframeTransform( lambda df: df.groupby('animal').mean(), # TODO: Generate proxy in this case as well proxy=schemas.generate_proxy(Animal), include_indexes=True)) assert_that(res, equal_to([('Falcon', 375.), ('Parrot', 25.)])) # Do the same thing, but use reset_index() to make sure 'animal' is included with TestPipeline() as p: with beam.dataframe.allow_non_parallel_operations(): res = ( p | beam.Create([ Animal('Falcon', 380.0), Animal('Falcon', 370.0), Animal('Parrot', 24.0), Animal('Parrot', 26.0) ]) | schemas.BatchRowsAsDataFrame() | transforms.DataframeTransform( lambda df: df.groupby('animal').mean().reset_index(), # TODO: Generate proxy in this case as well proxy=schemas.generate_proxy(Animal))) assert_that(res, equal_to([('Falcon', 375.), ('Parrot', 25.)]))
def test_input_output_polymorphism(self): one_series = pd.Series([1]) two_series = pd.Series([2]) three_series = pd.Series([3]) proxy = one_series[:0] def equal_to_series(expected): def check(actual): actual = pd.concat(actual) if not expected.equals(actual): raise AssertionError('Series not equal: \n%s\n%s\n' % (expected, actual)) return check with beam.Pipeline() as p: one = p | 'One' >> beam.Create([one_series]) two = p | 'Two' >> beam.Create([two_series]) assert_that( one | 'PcollInPcollOut' >> transforms.DataframeTransform( lambda x: 3 * x, proxy=proxy, yield_elements='pandas'), equal_to_series(three_series), label='CheckPcollInPcollOut') assert_that((one, two) | 'TupleIn' >> transforms.DataframeTransform( lambda x, y: (x + y), (proxy, proxy), yield_elements='pandas'), equal_to_series(three_series), label='CheckTupleIn') assert_that(dict(x=one, y=two) | 'DictIn' >> transforms.DataframeTransform( lambda x, y: (x + y), proxy=dict(x=proxy, y=proxy), yield_elements='pandas'), equal_to_series(three_series), label='CheckDictIn') double, triple = one | 'TupleOut' >> transforms.DataframeTransform( lambda x: (2 * x, 3 * x), proxy, yield_elements='pandas') assert_that(double, equal_to_series(two_series), 'CheckTupleOut0') assert_that(triple, equal_to_series(three_series), 'CheckTupleOut1') res = one | 'DictOut' >> transforms.DataframeTransform( lambda x: {'res': 3 * x}, proxy, yield_elements='pandas') assert_that(res['res'], equal_to_series(three_series), 'CheckDictOut')
def run_scenario(self, input, func): expected = func(input) empty = input[0:0] input_placeholder = expressions.PlaceholderExpression(empty) input_deferred = frame_base.DeferredFrame.wrap(input_placeholder) actual_deferred = func(input_deferred)._expr.evaluate_at( expressions.Session({input_placeholder: input})) def check_correct(actual): if actual is None: raise AssertionError('Empty frame but expected: \n\n%s' % (expected)) sorted_actual = actual.sort_index() sorted_expected = expected.sort_index() if not sorted_actual.equals(sorted_expected): raise AssertionError( 'Dataframes not equal: \n\n%s\n\n%s' % (sorted_actual, sorted_expected)) check_correct(actual_deferred) with beam.Pipeline() as p: input_pcoll = p | beam.Create([input[::2], input[1::2]]) output_pcoll = input_pcoll | transforms.DataframeTransform( func, proxy=empty) assert_that( output_pcoll, lambda actual: check_correct(pd.concat(actual) if actual else None))
def test_batching_passthrough_nested_schema(self): with beam.Pipeline() as p: nested_schema_pc = (p | beam.Create([ Nested(1, AnimalSpeed('Aardvark', 5)) ]).with_output_types(Nested)) result = nested_schema_pc | transforms.DataframeTransform( # pylint: disable=expression-not-assigned lambda df: df.filter(items=['animal_speed'])) assert_that(result, equal_to([(('Aardvark', 5), )]))
def test_column_manipulation(self): def set_column(df, name, s): df[name] = s return df with beam.Pipeline() as p: _ = (self.create_animal_speed_input(p) | transforms.DataframeTransform(lambda df: set_column( df, 'x', df.Speed + df.Animal.str.len()))) self.assertEqual(self.fused_stages(p), 1)
def test_unbatching_series(self): with beam.Pipeline() as p: result = ( p | beam.Create([(u'Falcon', 380.), (u'Falcon', 370.), (u'Parrot', 24.), (u'Parrot', 26.)]) | beam.Map(lambda tpl: beam.Row(Animal=tpl[0], Speed=tpl[1])) | transforms.DataframeTransform(lambda df: df.Animal)) assert_that(result, equal_to(['Falcon', 'Falcon', 'Parrot', 'Parrot']))
def test_batching_unsupported_array_schema_raises(self): Array = typing.NamedTuple( 'Array', [('id', int), ('business_numbers', typing.Sequence[int])]) coders.registry.register_coder(Array, coders.RowCoder) with beam.Pipeline() as p: array_schema_pc = (p | beam.Create([Array(1, [7, 8, 9])])) with self.assertRaisesRegex(TypeError, 'business_numbers'): array_schema_pc | transforms.DataframeTransform( # pylint: disable=expression-not-assigned lambda df: df.filter(items=['id']))
def test_batching_passthrough_nested_array(self): Array = typing.NamedTuple( 'Array', [('id', int), ('business_numbers', typing.Sequence[int])]) coders.registry.register_coder(Array, coders.RowCoder) with beam.Pipeline() as p: array_schema_pc = (p | beam.Create([Array(1, [7, 8, 9])])) result = array_schema_pc | transforms.DataframeTransform( # pylint: disable=expression-not-assigned lambda df: df.filter(items=['business_numbers'])) assert_that(result, equal_to([([7, 8, 9], )]))
def test_batching_beam_row_input(self): with beam.Pipeline() as p: result = ( p | beam.Create([(u'Falcon', 380.), (u'Falcon', 370.), (u'Parrot', 24.), (u'Parrot', 26.)]) | beam.Map(lambda tpl: beam.Row(Animal=tpl[0], Speed=tpl[1])) | transforms.DataframeTransform( lambda df: df.groupby('Animal').mean(), include_indexes=True)) assert_that(result, equal_to([('Falcon', 375.), ('Parrot', 25.)]))
def test_batching_unsupported_nested_schema_raises(self): Nested = typing.NamedTuple( 'Nested', [('id', int), ('animal_speed', AnimalSpeed)]) coders.registry.register_coder(Nested, coders.RowCoder) with beam.Pipeline() as p: nested_schema_pc = ( p | beam.Create([Nested(1, AnimalSpeed('Aardvark', 5)) ]).with_output_types(Nested)) with self.assertRaisesRegex(TypeError, 'animal_speed'): nested_schema_pc | transforms.DataframeTransform( # pylint: disable=expression-not-assigned lambda df: df.filter(items=['id']))
def test_batching_named_tuple_input(self): with beam.Pipeline() as p: result = ( p | beam.Create([ AnimalSpeed('Aardvark', 5), AnimalSpeed('Ant', 2), AnimalSpeed('Elephant', 35), AnimalSpeed('Zebra', 40) ]).with_output_types(AnimalSpeed) | transforms.DataframeTransform(lambda df: df.filter(regex='Anim.*'))) assert_that( result, equal_to([('Aardvark', ), ('Ant', ), ('Elephant', ), ('Zebra', )]))
def test_batching_beam_row_input(self): with beam.Pipeline() as p: result = ( p | beam.Create([(u'Falcon', 380.), (u'Falcon', 370.), (u'Parrot', 24.), (u'Parrot', 26.)]) | beam.Map(lambda tpl: beam.Row(Animal=tpl[0], Speed=tpl[1])) | transforms.DataframeTransform(lambda df: df.groupby('Animal').mean())) assert_that( result, df_equal_to( pd.DataFrame({ 'Animal': ['Falcon', 'Parrot'], 'Speed': [375., 25.] }).set_index('Animal')))
def run_scenario(self, input, func): expected = func(input) empty = input[0:0] input_placeholder = expressions.PlaceholderExpression(empty) input_deferred = frame_base.DeferredFrame.wrap(input_placeholder) actual_deferred = func(input_deferred)._expr.evaluate_at( expressions.Session({input_placeholder: input})) check_correct(expected, actual_deferred) with beam.Pipeline() as p: input_pcoll = p | beam.Create([input[::2], input[1::2]]) output_pcoll = input_pcoll | transforms.DataframeTransform( func, proxy=empty, yield_elements='pandas') assert_that(output_pcoll, lambda actual: check_correct(expected, concat(actual)))
def test_batch_with_df_transform(self): with TestPipeline() as p: res = ( p | beam.Create([ Animal('Falcon', 380.0), Animal('Falcon', 370.0), Animal('Parrot', 24.0), Animal('Parrot', 26.0) ]) | schemas.BatchRowsAsDataFrame() | transforms.DataframeTransform( lambda df: df.groupby('animal').mean(), # TODO: Generate proxy in this case as well proxy=schemas.generate_proxy(Animal))) assert_that( res, matches_df( pd.DataFrame({'max_speed': [375.0, 25.0]}, index=pd.Index(data=['Falcon', 'Parrot'], name='animal'))))
def run_scenario(self, input, func): expected = func(input) empty = input[0:0] input_placeholder = expressions.PlaceholderExpression(empty) input_deferred = frame_base.DeferredFrame.wrap(input_placeholder) actual_deferred = func(input_deferred)._expr.evaluate_at( expressions.Session({input_placeholder: input})) def concat(parts): if len(parts) > 1: return pd.concat(parts) elif len(parts) == 1: return parts[0] else: return None def check_correct(actual): if actual is None: raise AssertionError('Empty frame but expected: \n\n%s' % (expected)) if isinstance(expected, pd.core.generic.NDFrame): sorted_actual = actual.sort_index() sorted_expected = expected.sort_index() if not sorted_actual.equals(sorted_expected): raise AssertionError('Dataframes not equal: \n\n%s\n\n%s' % (sorted_actual, sorted_expected)) else: if actual != expected: raise AssertionError('Scalars not equal: %s != %s' % (actual, expected)) check_correct(actual_deferred) with beam.Pipeline() as p: input_pcoll = p | beam.Create([input[::2], input[1::2]]) output_pcoll = input_pcoll | transforms.DataframeTransform( func, proxy=empty) assert_that(output_pcoll, lambda actual: check_correct(concat(actual)))
def test_loc_filter(self): with beam.Pipeline() as p: _ = (self.create_animal_speed_input(p) | transforms.DataframeTransform(lambda df: df[df.Speed > 10])) self.assertEqual(self.fused_stages(p), 1)