TheNeuralBit commented on a change in pull request #11980: URL: https://github.com/apache/beam/pull/11980#discussion_r472308346
########## File path: sdks/python/apache_beam/dataframe/schemas.py ########## @@ -0,0 +1,82 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +"""Utilities for relating schema-aware PCollections and dataframe transforms. +""" + +# pytype: skip-file + +from __future__ import absolute_import + +from typing import NamedTuple +from typing import TypeVar + +import pandas as pd + +from apache_beam import typehints +from apache_beam.transforms.core import DoFn +from apache_beam.transforms.core import ParDo +from apache_beam.transforms.util import BatchElements +from apache_beam.typehints.schemas import named_fields_from_element_type + +__all__ = ('BatchRowsAsDataFrame', 'generate_proxy') + +T = TypeVar('T', bound=NamedTuple) + + [email protected]_input_types(T) [email protected]_output_types(pd.DataFrame) +class BatchRowsAsDataFrame(BatchElements): + """A transform that batches schema-aware PCollection elements into DataFrames + + Batching parameters are inherited from + :class:`~apache_beam.transforms.util.BatchElements`. + """ + def __init__(self, *args, **kwargs): + super(BatchRowsAsDataFrame, self).__init__(*args, **kwargs) + self._batch_elements_transform = BatchElements(*args, **kwargs) + + def expand(self, pcoll): + return super(BatchRowsAsDataFrame, self).expand(pcoll) | ParDo( Review comment: Done! Looks like I actually started to do it that way with the unused self._batch_elements_transform but then changed my mind ########## File path: sdks/python/apache_beam/dataframe/transforms_test.py ########## @@ -112,6 +133,64 @@ def test_scalar(self): self.run_scenario( df, lambda df: df.groupby('key').sum().val / df.val.agg(sum)) + 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='A.*'))) Review comment: You and me both :) I just reused the operation from `test_filter` above. Changed it to Anim.* in both places ########## File path: sdks/python/apache_beam/pvalue.py ########## @@ -88,7 +88,7 @@ class PValue(object): def __init__(self, pipeline, # type: Pipeline tag=None, # type: Optional[str] - element_type=None, # type: Optional[object] + element_type=None, # type: Optional[type] Review comment: Done ########## File path: sdks/python/apache_beam/dataframe/schemas.py ########## @@ -0,0 +1,82 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +"""Utilities for relating schema-aware PCollections and dataframe transforms. +""" + +# pytype: skip-file + +from __future__ import absolute_import + +from typing import NamedTuple +from typing import TypeVar + +import pandas as pd + +from apache_beam import typehints +from apache_beam.transforms.core import DoFn +from apache_beam.transforms.core import ParDo +from apache_beam.transforms.util import BatchElements +from apache_beam.typehints.schemas import named_fields_from_element_type + +__all__ = ('BatchRowsAsDataFrame', 'generate_proxy') + +T = TypeVar('T', bound=NamedTuple) + + [email protected]_input_types(T) [email protected]_output_types(pd.DataFrame) +class BatchRowsAsDataFrame(BatchElements): + """A transform that batches schema-aware PCollection elements into DataFrames + + Batching parameters are inherited from + :class:`~apache_beam.transforms.util.BatchElements`. + """ + def __init__(self, *args, **kwargs): + super(BatchRowsAsDataFrame, self).__init__(*args, **kwargs) + self._batch_elements_transform = BatchElements(*args, **kwargs) + + def expand(self, pcoll): + return super(BatchRowsAsDataFrame, self).expand(pcoll) | ParDo( + _RowBatchToDataFrameDoFn(pcoll.element_type)) + + +class _RowBatchToDataFrameDoFn(DoFn): Review comment: Done ########## File path: sdks/python/apache_beam/dataframe/transforms_test.py ########## @@ -17,17 +17,61 @@ from __future__ import absolute_import from __future__ import division +import typing import unittest import pandas as pd +from past.builtins import unicode import apache_beam as beam +from apache_beam import coders from apache_beam.dataframe import expressions from apache_beam.dataframe import frame_base from apache_beam.dataframe import transforms from apache_beam.testing.util import assert_that +def sort_by_value_and_drop_index(df): + if isinstance(df, pd.DataFrame): + sorted_df = df.sort_values(by=list(df.columns)) + else: + sorted_df = df.sort_values() + return sorted_df.reset_index(drop=True) Review comment: Note there's actually a diff here from the original `check_correct`. It sorts by value and resets the index rather than sorting by index. I had to do this because the concatenated indices of the batches (e.g. `[0,1,2,0,1,0,]`) wouldn't match the index in my expected df (e.g. `[0,1,2,3,4,5]`). ########## File path: sdks/python/apache_beam/dataframe/schemas_test.py ########## @@ -0,0 +1,110 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +"""Tests for schemas.""" + +# pytype: skip-file + +from __future__ import absolute_import + +import unittest +from typing import NamedTuple + +import future.tests.base # pylint: disable=unused-import +# patches unittest.testcase to be python3 compatible +import pandas as pd +from past.builtins import unicode + +import apache_beam as beam +from apache_beam.coders import RowCoder +from apache_beam.coders.typecoders import registry as coders_registry +from apache_beam.dataframe import schemas +from apache_beam.dataframe import transforms +from apache_beam.testing.test_pipeline import TestPipeline +from apache_beam.testing.util import assert_that + +Simple = NamedTuple( + 'Simple', [('name', unicode), ('id', int), ('height', float)]) +coders_registry.register_coder(Simple, RowCoder) +Animal = NamedTuple('Animal', [('animal', unicode), ('max_speed', float)]) +coders_registry.register_coder(Animal, RowCoder) + + +def matches_df(expected): + def check_df_pcoll_equal(actual): + actual = pd.concat(actual) + sorted_actual = actual.sort_values(by=list(actual.columns)).reset_index( + drop=True) + sorted_expected = expected.sort_values( + by=list(expected.columns)).reset_index(drop=True) + if not sorted_actual.equals(sorted_expected): + raise AssertionError( + 'Dataframes not equal: \n\nActual:\n%s\n\nExpected:\n%s' % + (sorted_actual, sorted_expected)) + + return check_df_pcoll_equal + + +class SchemasTest(unittest.TestCase): + def test_simple_df(self): + expected = pd.DataFrame({ + 'name': list(unicode(i) for i in range(5)), + 'id': list(range(5)), + 'height': list(float(i) for i in range(5)) + }, + columns=['name', 'id', 'height']) + + with TestPipeline() as p: + res = ( + p + | beam.Create([ + Simple(name=unicode(i), id=i, height=float(i)) for i in range(5) + ]) + | schemas.BatchRowsAsDataFrame(min_batch_size=10, max_batch_size=10)) + assert_that(res, matches_df(expected)) + + def test_generate_proxy(self): + expected = pd.DataFrame({ + 'animal': pd.Series(dtype=unicode), 'max_speed': pd.Series(dtype=float) + }) + + self.assertTrue(schemas.generate_proxy(Animal).equals(expected)) + + def test_batch_with_df_transform(self): Review comment: I added a test using to_dataframe in transforms_test: test_batching_beam_row_to_dataframe. I intended for these tests to just test schemas.py, while transforms_test verifies the integration with DataframeTransform. The one below was a stepping stone to integrating, we could even remove it now. ########## File path: sdks/python/apache_beam/dataframe/schemas.py ########## @@ -0,0 +1,82 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +"""Utilities for relating schema-aware PCollections and dataframe transforms. +""" + +# pytype: skip-file + +from __future__ import absolute_import + +from typing import NamedTuple +from typing import TypeVar + +import pandas as pd + +from apache_beam import typehints +from apache_beam.transforms.core import DoFn Review comment: Done ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected]
