github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/dataframe/convert.py (about) 1 # 2 # Licensed to the Apache Software Foundation (ASF) under one or more 3 # contributor license agreements. See the NOTICE file distributed with 4 # this work for additional information regarding copyright ownership. 5 # The ASF licenses this file to You under the Apache License, Version 2.0 6 # (the "License"); you may not use this file except in compliance with 7 # the License. You may obtain a copy of the License at 8 # 9 # http://www.apache.org/licenses/LICENSE-2.0 10 # 11 # Unless required by applicable law or agreed to in writing, software 12 # distributed under the License is distributed on an "AS IS" BASIS, 13 # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 # See the License for the specific language governing permissions and 15 # limitations under the License. 16 17 import inspect 18 import warnings 19 import weakref 20 from typing import TYPE_CHECKING 21 from typing import Any 22 from typing import Dict 23 from typing import Iterable 24 from typing import Tuple 25 from typing import Union 26 27 import pandas as pd 28 29 import apache_beam as beam 30 from apache_beam import pvalue 31 from apache_beam.dataframe import expressions 32 from apache_beam.dataframe import frame_base 33 from apache_beam.dataframe import transforms 34 from apache_beam.dataframe.schemas import element_typehint_from_dataframe_proxy 35 from apache_beam.dataframe.schemas import generate_proxy 36 from apache_beam.typehints.pandas_type_compatibility import dtype_to_fieldtype 37 38 if TYPE_CHECKING: 39 # pylint: disable=ungrouped-imports 40 from typing import Optional 41 42 43 # TODO: Or should this be called as_dataframe? 44 def to_dataframe( 45 pcoll, # type: pvalue.PCollection 46 proxy=None, # type: Optional[pd.core.generic.NDFrame] 47 label=None, # type: Optional[str] 48 ): 49 # type: (...) -> frame_base.DeferredFrame 50 51 """Converts a PCollection to a deferred dataframe-like object, which can 52 manipulated with pandas methods like `filter` and `groupby`. 53 54 For example, one might write:: 55 56 pcoll = ... 57 df = to_dataframe(pcoll, proxy=...) 58 result = df.groupby('col').sum() 59 pcoll_result = to_pcollection(result) 60 61 A proxy object must be given if the schema for the PCollection is not known. 62 """ 63 if proxy is None: 64 if pcoll.element_type is None: 65 raise ValueError( 66 "Cannot infer a proxy because the input PCollection does not have a " 67 "schema defined. Please make sure a schema type is specified for " 68 "the input PCollection, or provide a proxy.") 69 # If no proxy is given, assume this is an element-wise schema-aware 70 # PCollection that needs to be batched. 71 if label is None: 72 # Attempt to come up with a reasonable, stable label by retrieving 73 # the name of these variables in the calling context. 74 label = 'BatchElements(%s)' % _var_name(pcoll, 2) 75 proxy = generate_proxy(pcoll.element_type) 76 77 shim_dofn: beam.DoFn 78 if isinstance(proxy, pd.DataFrame): 79 shim_dofn = RowsToDataFrameFn() 80 elif isinstance(proxy, pd.Series): 81 shim_dofn = ElementsToSeriesFn() 82 else: 83 raise AssertionError("Unknown proxy type: %s" % proxy) 84 85 pcoll = pcoll | label >> beam.ParDo(shim_dofn) 86 return frame_base.DeferredFrame.wrap( 87 expressions.PlaceholderExpression(proxy, pcoll)) 88 89 90 # PCollections generated by to_pcollection are memoized, keyed by expression id. 91 # WeakValueDictionary is used so the caches are cleaned up with the parent 92 # pipelines 93 # Note that the pipeline (indirectly) holds references to the transforms which 94 # keeps both the PCollections and expressions alive. This ensures the 95 # expression's ids are never accidentally re-used. 96 TO_PCOLLECTION_CACHE = weakref.WeakValueDictionary( 97 ) # type: weakref.WeakValueDictionary[str, pvalue.PCollection] 98 UNBATCHED_CACHE = weakref.WeakValueDictionary( 99 ) # type: weakref.WeakValueDictionary[str, pvalue.PCollection] 100 101 102 class RowsToDataFrameFn(beam.DoFn): 103 @beam.DoFn.yields_elements 104 def process_batch(self, batch: pd.DataFrame) -> Iterable[pd.DataFrame]: 105 yield batch 106 107 108 class ElementsToSeriesFn(beam.DoFn): 109 @beam.DoFn.yields_elements 110 def process_batch(self, batch: pd.Series) -> Iterable[pd.Series]: 111 yield batch 112 113 114 def _make_unbatched_pcoll( 115 pc: pvalue.PCollection, expr: expressions.Expression, 116 include_indexes: bool): 117 label = f"Unbatch '{expr._id}'" 118 if include_indexes: 119 label += " with indexes" 120 121 if label not in UNBATCHED_CACHE: 122 proxy = expr.proxy() 123 shim_dofn: beam.DoFn 124 if isinstance(proxy, pd.DataFrame): 125 shim_dofn = DataFrameToRowsFn(proxy, include_indexes) 126 elif isinstance(proxy, pd.Series): 127 if include_indexes: 128 warnings.warn( 129 "Pipeline is converting a DeferredSeries to PCollection " 130 "with include_indexes=True. Note that this parameter is " 131 "_not_ respected for DeferredSeries conversion. To " 132 "include the index with your data, produce a" 133 "DeferredDataFrame instead.") 134 135 shim_dofn = SeriesToElementsFn(proxy) 136 else: 137 raise TypeError(f"Proxy '{proxy}' has unsupported type '{type(proxy)}'") 138 139 UNBATCHED_CACHE[label] = pc | label >> beam.ParDo(shim_dofn) 140 141 # Note unbatched cache is keyed by the expression id as well as parameters 142 # for the unbatching (i.e. include_indexes) 143 return UNBATCHED_CACHE[label] 144 145 146 class DataFrameToRowsFn(beam.DoFn): 147 def __init__(self, proxy, include_indexes): 148 self._proxy = proxy 149 self._include_indexes = include_indexes 150 151 @beam.DoFn.yields_batches 152 def process(self, element: pd.DataFrame) -> Iterable[pd.DataFrame]: 153 yield element 154 155 def infer_output_type(self, input_element_type): 156 return element_typehint_from_dataframe_proxy( 157 self._proxy, self._include_indexes) 158 159 160 class SeriesToElementsFn(beam.DoFn): 161 def __init__(self, proxy): 162 self._proxy = proxy 163 164 @beam.DoFn.yields_batches 165 def process(self, element: pd.Series) -> Iterable[pd.Series]: 166 yield element 167 168 def infer_output_type(self, input_element_type): 169 return dtype_to_fieldtype(self._proxy.dtype) 170 171 172 # TODO: Or should this be called from_dataframe? 173 174 175 def to_pcollection( 176 *dataframes, # type: Union[frame_base.DeferredFrame, pd.DataFrame, pd.Series] 177 label=None, 178 always_return_tuple=False, 179 yield_elements='schemas', 180 include_indexes=False, 181 pipeline=None) -> Union[pvalue.PCollection, Tuple[pvalue.PCollection, ...]]: 182 """Converts one or more deferred dataframe-like objects back to a PCollection. 183 184 This method creates and applies the actual Beam operations that compute 185 the given deferred dataframes, returning a PCollection of their results. By 186 default the resulting PCollections are schema-aware PCollections where each 187 element is one row from the output dataframes, excluding indexes. This 188 behavior can be modified with the `yield_elements` and `include_indexes` 189 arguments. 190 191 Also accepts non-deferred pandas dataframes, which are converted to deferred, 192 schema'd PCollections. In this case the contents of the entire dataframe are 193 serialized into the graph, so for large amounts of data it is preferable to 194 write them to disk and read them with one of the read methods. 195 196 If more than one (related) result is desired, it can be more efficient to 197 pass them all at the same time to this method. 198 199 Args: 200 label: (optional, default "ToPCollection(...)"") the label to use for the 201 conversion transform. 202 always_return_tuple: (optional, default: False) If true, always return 203 a tuple of PCollections, even if there's only one output. 204 yield_elements: (optional, default: "schemas") If set to "pandas", return 205 PCollections containing the raw Pandas objects (DataFrames or Series), 206 if set to "schemas", return an element-wise PCollection, where DataFrame 207 and Series instances are expanded to one element per row. DataFrames are 208 converted to schema-aware PCollections, where column values can be 209 accessed by attribute. 210 include_indexes: (optional, default: False) When yield_elements="schemas", 211 if include_indexes=True, attempt to include index columns in the output 212 schema for expanded DataFrames. Raises an error if any of the index 213 levels are unnamed (name=None), or if any of the names are not unique 214 among all column and index names. 215 pipeline: (optional, unless non-deferred dataframes are passed) Used when 216 creating a PCollection from a non-deferred dataframe. 217 """ 218 if not yield_elements in ("pandas", "schemas"): 219 raise ValueError( 220 "Invalid value for yield_elements argument, '%s'. " 221 "Allowed values are 'pandas' and 'schemas'" % yield_elements) 222 if label is None: 223 # Attempt to come up with a reasonable, stable label by retrieving the name 224 # of these variables in the calling context. 225 label = 'ToPCollection(%s)' % ', '.join(_var_name(e, 3) for e in dataframes) 226 227 # Support for non-deferred dataframes. 228 deferred_dataframes = [] 229 for ix, df in enumerate(dataframes): 230 if isinstance(df, frame_base.DeferredBase): 231 # TODO(robertwb): Maybe extract pipeline object? 232 deferred_dataframes.append(df) 233 elif isinstance(df, (pd.Series, pd.DataFrame)): 234 if pipeline is None: 235 raise ValueError( 236 'Pipeline keyword required for non-deferred dataframe conversion.') 237 deferred = pipeline | '%s_Defer%s' % (label, ix) >> beam.Create([df]) 238 deferred_dataframes.append( 239 frame_base.DeferredFrame.wrap( 240 expressions.PlaceholderExpression(df.iloc[:0], deferred))) 241 else: 242 raise TypeError( 243 'Unable to convert objects of type %s to a PCollection' % type(df)) 244 dataframes = tuple(deferred_dataframes) 245 246 def extract_input(placeholder): 247 if not isinstance(placeholder._reference, pvalue.PCollection): 248 raise TypeError( 249 'Expression roots must have been created with to_dataframe.') 250 return placeholder._reference 251 252 placeholders = frozenset.union( 253 frozenset(), *[df._expr.placeholders() for df in dataframes]) 254 255 # Exclude any dataframes that have already been converted to PCollections. 256 # We only want to convert each DF expression once, then re-use. 257 new_dataframes = [ 258 df for df in dataframes if df._expr._id not in TO_PCOLLECTION_CACHE 259 ] 260 if len(new_dataframes): 261 new_results = {p: extract_input(p) 262 for p in placeholders 263 } | label >> transforms._DataframeExpressionsTransform({ 264 ix: df._expr 265 for (ix, df) in enumerate(new_dataframes) 266 }) # type: Dict[Any, pvalue.PCollection] 267 268 TO_PCOLLECTION_CACHE.update( 269 {new_dataframes[ix]._expr._id: pc 270 for ix, pc in new_results.items()}) 271 272 raw_results = { 273 ix: TO_PCOLLECTION_CACHE[df._expr._id] 274 for ix, 275 df in enumerate(dataframes) 276 } 277 278 if yield_elements == "schemas": 279 280 def maybe_unbatch(pc, value): 281 if isinstance(value, frame_base._DeferredScalar): 282 return pc 283 else: 284 return _make_unbatched_pcoll(pc, value._expr, include_indexes) 285 286 results = { 287 ix: maybe_unbatch(pc, dataframes[ix]) 288 for (ix, pc) in raw_results.items() 289 } 290 else: 291 results = raw_results 292 293 if len(results) == 1 and not always_return_tuple: 294 return results[0] 295 else: 296 return tuple(value for key, value in sorted(results.items())) 297 298 299 def _var_name(obj, level): 300 frame = inspect.currentframe() 301 for _ in range(level): 302 if frame is None: 303 return '...' 304 frame = frame.f_back 305 for key, value in frame.f_locals.items(): 306 if obj is value: 307 return key 308 for key, value in frame.f_globals.items(): 309 if obj is value: 310 return key 311 return '...'