-
Notifications
You must be signed in to change notification settings - Fork 4.2k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Add BatchRowsAsDataframe and generate_proxy, integrated into DataFram…
…eTransform
- Loading branch information
1 parent
1671ce5
commit 55c4920
Showing
7 changed files
with
317 additions
and
37 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,78 @@ | ||
# | ||
# 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. | ||
""" | ||
|
||
from typing import NamedTuple | ||
from typing import TypeVar | ||
|
||
import pandas as pd | ||
|
||
from apache_beam import typehints | ||
from apache_beam.typehints.schemas import named_fields_from_element_type | ||
from apache_beam.transforms.core import DoFn | ||
from apache_beam.transforms.core import ParDo | ||
from apache_beam.transforms.util import BatchElements | ||
|
||
__all__ = ('BatchRowsAsDataFrame', 'generate_proxy') | ||
|
||
T = TypeVar('T', bound=NamedTuple) | ||
|
||
|
||
@typehints.with_input_types(T) | ||
@typehints.with_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): | ||
def __init__(self, element_type): | ||
self._columns = [ | ||
name for name, _ in named_fields_from_element_type(element_type) | ||
] | ||
|
||
def process(self, element): | ||
result = pd.DataFrame.from_records(element, columns=self._columns) | ||
yield result | ||
|
||
|
||
def _make_empty_series(name, typ): | ||
try: | ||
return pd.Series(name=name, dtype=typ) | ||
except TypeError: | ||
raise TypeError("Unable to convert type '%s' for field '%s'" % (name, typ)) | ||
|
||
|
||
def generate_proxy(element_type): | ||
# type: (type) -> pd.DataFrame | ||
return pd.DataFrame({ | ||
name: _make_empty_series(name, typ) | ||
for name, | ||
typ in named_fields_from_element_type(element_type) | ||
}) |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,82 @@ | ||
"""Tests for schemas.""" | ||
|
||
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 | ||
|
||
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', str), ('id', int), ('height', float)]) | ||
coders_registry.register_coder(Simple, RowCoder) | ||
Animal = NamedTuple('Animal', [('animal', str), ('max_speed', float)]) | ||
coders_registry.register_coder(Animal, RowCoder) | ||
|
||
|
||
def matches_df(expected): | ||
def check_df_pcoll_equal(actual): | ||
sorted_actual = pd.concat(actual).sort_index() | ||
sorted_expected = expected.sort_index() | ||
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(map(str, range(5))), | ||
'id': list(range(5)), | ||
'height': list(map(float, range(5))) | ||
}) | ||
|
||
with TestPipeline() as p: | ||
res = ( | ||
p | ||
| beam.Create( | ||
[Simple(name=str(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=str), 'max_speed': pd.Series(dtype=float) | ||
}) | ||
|
||
self.assertTrue(schemas.generate_proxy(Animal).equals(expected)) | ||
|
||
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(), | ||
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')))) | ||
|
||
|
||
if __name__ == '__main__': | ||
unittest.main() |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.