-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Expand file tree
/
Copy pathpandas_type_compatibility.py
More file actions
301 lines (233 loc) · 10.1 KB
/
pandas_type_compatibility.py
File metadata and controls
301 lines (233 loc) · 10.1 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
#
# 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.
#
r"""Utilities for converting between Beam schemas and pandas DataFrames.
Imposes a mapping between native Python typings (specifically those compatible
with :mod:`apache_beam.typehints.schemas`), and common pandas dtypes::
pandas dtype Python typing
np.int{8,16,32,64} <-----> np.int{8,16,32,64}*
pd.Int{8,16,32,64}Dtype <-----> Optional[np.int{8,16,32,64}]*
np.float{32,64} <-----> Optional[np.float{32,64}]
\--- np.float{32,64}
Not supported <------ Optional[bytes]
np.bool <-----> np.bool
np.dtype('S') <-----> bytes
pd.BooleanDType() <-----> Optional[bool]
pd.StringDType() <-----> Optional[str]
\--- str
np.object <-----> Any
* int, float, bool are treated the same as np.int64, np.float64, np.bool
Note that when converting to pandas dtypes, any types not specified here are
shunted to ``np.object``.
Similarly when converting from pandas to Python types, types that aren't
otherwise specified here are shunted to ``Any``. Notably, this includes
``np.datetime64``.
Pandas does not support hierarchical data natively. Currently, all structured
types (``Sequence``, ``Mapping``, nested ``NamedTuple`` types), are
shunted to ``np.object`` like all other unknown types. In the future these
types may be given special consideration.
Note utilities in this package are for internal use only, we make no backward
compatibility guarantees, except for the type mapping itself.
"""
from typing import Any
from typing import List
from typing import Optional
import numpy as np
import pandas as pd
from apache_beam.typehints.batch import BatchConverter
from apache_beam.typehints.row_type import RowTypeConstraint
from apache_beam.typehints.typehints import is_nullable
from apache_beam.typehints.typehints import normalize
# No public API currently, this just exists to register BatchConverter
# implementations.
__all__ = []
# Name for a valueless field-level option which, when present, indicates that
# a field should map to an index in the Beam DataFrame API.
INDEX_OPTION_NAME = 'beam:dataframe:index'
# Generate type map (presented visually in the docstring)
_BIDIRECTIONAL = [
(bool, bool),
(np.int8, np.int8),
(np.int16, np.int16),
(np.int32, np.int32),
(np.int64, np.int64),
(pd.Int8Dtype(), Optional[np.int8]),
(pd.Int16Dtype(), Optional[np.int16]),
(pd.Int32Dtype(), Optional[np.int32]),
(pd.Int64Dtype(), Optional[np.int64]),
(np.float32, Optional[np.float32]),
(np.float64, Optional[np.float64]),
(object, Any),
(pd.StringDtype(), Optional[str]),
(pd.BooleanDtype(), Optional[bool]),
]
PANDAS_TO_BEAM = {
pd.Series([], dtype=dtype).dtype: fieldtype
for dtype, fieldtype in _BIDIRECTIONAL
}
BEAM_TO_PANDAS = {fieldtype: dtype for dtype, fieldtype in _BIDIRECTIONAL}
# Shunt non-nullable Beam types to the same pandas types as their non-nullable
# equivalents for FLOATs, DOUBLEs, and STRINGs. pandas has no non-nullable dtype
# for these.
OPTIONAL_SHUNTS = [np.float32, np.float64, str]
for typehint in OPTIONAL_SHUNTS:
BEAM_TO_PANDAS[typehint] = BEAM_TO_PANDAS[Optional[typehint]]
# int, float -> int64, np.float64
BEAM_TO_PANDAS[int] = BEAM_TO_PANDAS[np.int64]
BEAM_TO_PANDAS[Optional[int]] = BEAM_TO_PANDAS[Optional[np.int64]]
BEAM_TO_PANDAS[float] = BEAM_TO_PANDAS[np.float64]
BEAM_TO_PANDAS[Optional[float]] = BEAM_TO_PANDAS[Optional[np.float64]]
BEAM_TO_PANDAS[bytes] = 'bytes'
# Add shunts for normalized (Beam) typehints as well
BEAM_TO_PANDAS.update({
normalize(typehint): pandas_dtype
for (typehint, pandas_dtype) in BEAM_TO_PANDAS.items()
})
def dtype_from_typehint(typehint):
# Default to np.object. This is lossy, we won't be able to recover
# the type at the output.
return BEAM_TO_PANDAS.get(typehint, object)
def dtype_to_fieldtype(dtype):
fieldtype = PANDAS_TO_BEAM.get(dtype)
if fieldtype is not None:
return fieldtype
elif dtype.kind == 'S':
return bytes
else:
return Any
@BatchConverter.register(name="pandas")
def create_pandas_batch_converter(
element_type: type, batch_type: type) -> BatchConverter:
if batch_type == pd.DataFrame:
return DataFrameBatchConverter.from_typehints(
element_type=element_type, batch_type=batch_type)
elif batch_type == pd.Series:
return SeriesBatchConverter.from_typehints(
element_type=element_type, batch_type=batch_type)
raise TypeError("batch type must be pd.Series or pd.DataFrame")
class DataFrameBatchConverter(BatchConverter):
def __init__(
self,
element_type: RowTypeConstraint,
):
super().__init__(pd.DataFrame, element_type)
self._columns = [name for name, _ in element_type._fields]
@staticmethod
def from_typehints(element_type,
batch_type) -> Optional['DataFrameBatchConverter']:
assert batch_type == pd.DataFrame
if not isinstance(element_type, RowTypeConstraint):
element_type = RowTypeConstraint.from_user_type(element_type)
if element_type is None:
raise TypeError(
"Element type must be compatible with Beam Schemas ("
"https://beam.apache.org/documentation/programming-guide/#schemas) "
"for batch type pd.DataFrame")
index_columns = [
field_name
for (field_name, field_options) in element_type._field_options.items()
if any(key == INDEX_OPTION_NAME for key, value in field_options)
]
if index_columns:
return DataFrameBatchConverterKeepIndex(element_type, index_columns)
else:
return DataFrameBatchConverterDropIndex(element_type)
def _get_series(self, batch: pd.DataFrame):
raise NotImplementedError
def explode_batch(self, batch: pd.DataFrame):
# TODO(https://github.com/apache/beam/issues/22948): Only do null checks for
# nullable types
def make_null_checking_generator(series):
nulls = pd.isnull(series)
return (None if isnull else value for isnull, value in zip(nulls, series))
all_series = self._get_series(batch)
iterators = [make_null_checking_generator(series) for series in all_series]
for values in zip(*iterators):
yield self._element_type.user_type(
**{
column: value
for column, value in zip(self._columns, values)
})
def combine_batches(self, batches: List[pd.DataFrame]):
return pd.concat(batches)
def estimate_byte_size(self, batch: pd.DataFrame):
return batch.memory_usage().sum()
def get_length(self, batch: pd.DataFrame):
return len(batch)
class DataFrameBatchConverterDropIndex(DataFrameBatchConverter):
"""A DataFrameBatchConverter that assumes the DataFrame index has no meaning.
When producing a DataFrame from Rows, a meaningless index will be generated.
When exploding a DataFrame into Rows, the index will be dropped.
"""
def _get_series(self, batch: pd.DataFrame):
return [batch[column] for column in batch.columns]
def produce_batch(self, elements):
batch = pd.DataFrame.from_records(elements, columns=self._columns)
for column, typehint in self._element_type._fields:
batch[column] = batch[column].astype(dtype_from_typehint(typehint))
return batch
class DataFrameBatchConverterKeepIndex(DataFrameBatchConverter):
"""A DataFrameBatchConverter that preserves the DataFrame index.
This is tracked via options on the Beam schema. Each field in the schema that
should map to the index is tagged in an option with name 'dataframe:index'.
"""
def __init__(self, element_type: RowTypeConstraint, index_columns: List[Any]):
super().__init__(element_type)
self._index_columns = index_columns
def _get_series(self, batch: pd.DataFrame):
assert list(batch.index.names) == self._index_columns
return [
batch.index.get_level_values(i) for i in range(len(batch.index.names))
] + [batch[column] for column in batch.columns]
def produce_batch(self, elements):
# Note from_records has an index= parameter
batch = pd.DataFrame.from_records(elements, columns=self._columns)
for column, typehint in self._element_type._fields:
batch[column] = batch[column].astype(dtype_from_typehint(typehint))
return batch.set_index(self._index_columns)
class SeriesBatchConverter(BatchConverter):
def __init__(
self,
element_type: type,
dtype,
):
super().__init__(pd.DataFrame, element_type)
self._dtype = dtype
if is_nullable(element_type):
def unbatch(series):
for isnull, value in zip(pd.isnull(series), series):
yield None if isnull else value
else:
def unbatch(series):
yield from series
self.explode_batch = unbatch
@staticmethod
def from_typehints(element_type,
batch_type) -> Optional['SeriesBatchConverter']:
assert batch_type == pd.Series
dtype = dtype_from_typehint(element_type)
return SeriesBatchConverter(element_type, dtype)
def produce_batch(self, elements: List[Any]) -> pd.Series:
return pd.Series(elements, dtype=self._dtype)
def explode_batch(self, batch: pd.Series):
raise NotImplementedError(
"explode_batch should be generated in SeriesBatchConverter.__init__")
def combine_batches(self, batches: List[pd.Series]):
return pd.concat(batches)
def estimate_byte_size(self, batch: pd.Series):
return batch.memory_usage()
def get_length(self, batch: pd.Series):
return len(batch)