- Notifications
You must be signed in to change notification settings - Fork 4.3k
/
Copy pathtransforms.py
624 lines (532 loc) · 22.3 KB
/
transforms.py
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
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
#
# 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.
importcollections
importlogging
fromcollections.abcimportMapping
fromtypingimportAny
fromtypingimportTypeVar
fromtypingimportUnion
importpandasaspd
importapache_beamasbeam
fromapache_beamimporttransforms
fromapache_beam.dataframeimportexpressions
fromapache_beam.dataframeimportframe_base
fromapache_beam.dataframeimportframes# pylint: disable=unused-import
fromapache_beam.dataframeimportpartitionings
fromapache_beam.pvalueimportPCollection
fromapache_beam.utilsimportwindowed_value
__all__= [
'DataframeTransform',
]
T=TypeVar('T')
TARGET_PARTITION_SIZE=1<<23# 8M
MIN_PARTITION_SIZE=1<<19# 0.5M
MAX_PARTITIONS=1000
DEFAULT_PARTITIONS=100
MIN_PARTITIONS=10
PER_COL_OVERHEAD=1000
classDataframeTransform(transforms.PTransform):
"""A PTransform for applying function that takes and returns dataframes
to one or more PCollections.
:class:`DataframeTransform` will accept a PCollection with a `schema`_ and
batch it into :class:`~pandas.DataFrame` instances if necessary::
(pcoll | beam.Select(key=..., foo=..., bar=...)
| DataframeTransform(lambda df: df.group_by('key').sum()))
It is also possible to process a PCollection of :class:`~pandas.DataFrame`
instances directly, in this case a "proxy" must be provided. For example, if
``pcoll`` is a PCollection of DataFrames, one could write::
pcoll | DataframeTransform(lambda df: df.group_by('key').sum(), proxy=...)
To pass multiple PCollections, pass a tuple of PCollections wich will be
passed to the callable as positional arguments, or a dictionary of
PCollections, in which case they will be passed as keyword arguments.
Args:
yield_elements: (optional, default: "schemas") If set to ``"pandas"``,
return PCollection(s) containing the raw Pandas objects
(:class:`~pandas.DataFrame` or :class:`~pandas.Series` as appropriate).
If set to ``"schemas"``, return an element-wise PCollection, where
DataFrame and Series instances are expanded to one element per row.
DataFrames are converted to `schema-aware`_ PCollections, where column
values can be accessed by attribute.
include_indexes: (optional, default: False) When
``yield_elements="schemas"``, if ``include_indexes=True``, attempt to
include index columns in the output schema for expanded DataFrames.
Raises an error if any of the index levels are unnamed (name=None), or if
any of the names are not unique among all column and index names.
proxy: (optional) An empty :class:`~pandas.DataFrame` or
:class:`~pandas.Series` instance with the same ``dtype`` and ``name``
as the elements of the input PCollection. Required when input
PCollection :class:`~pandas.DataFrame` or :class:`~pandas.Series`
elements. Ignored when input PCollection has a `schema`_.
.. _schema:
https://beam.apache.org/documentation/programming-guide/#what-is-a-schema
.. _schema-aware:
https://beam.apache.org/documentation/programming-guide/#what-is-a-schema
"""
def__init__(
self, func, proxy=None, yield_elements="schemas", include_indexes=False):
self._func=func
self._proxy=proxy
self._yield_elements=yield_elements
self._include_indexes=include_indexes
defexpand(self, input_pcolls):
# Avoid circular import.
fromapache_beam.dataframeimportconvert
# Convert inputs to a flat dict.
input_dict: dict[Any, PCollection] =_flatten(input_pcolls)
proxies=_flatten(self._proxy) ifself._proxyisnotNoneelse {
tag: None
fortagininput_dict
}
input_frames: dict[Any, frame_base.DeferredFrame] = {
k: convert.to_dataframe(pc, proxies[k])
fork, pcininput_dict.items()
} # noqa: F821
# Apply the function.
frames_input=_substitute(input_pcolls, input_frames)
ifisinstance(frames_input, dict):
result_frames=self._func(**frames_input)
elifisinstance(frames_input, tuple):
result_frames=self._func(*frames_input)
else:
result_frames=self._func(frames_input)
# Compute results as a tuple.
result_frames_dict=_flatten(result_frames)
keys=list(result_frames_dict.keys())
result_frames_tuple=tuple(result_frames_dict[key] forkeyinkeys)
result_pcolls_tuple=convert.to_pcollection(
*result_frames_tuple,
label='Eval',
always_return_tuple=True,
yield_elements=self._yield_elements,
include_indexes=self._include_indexes)
# Convert back to the structure returned by self._func.
result_pcolls_dict=dict(zip(keys, result_pcolls_tuple))
return_substitute(result_frames, result_pcolls_dict)
class_DataframeExpressionsTransform(transforms.PTransform):
def__init__(self, outputs):
self._outputs=outputs
defexpand(self, inputs):
returnself._apply_deferred_ops(inputs, self._outputs)
def_apply_deferred_ops(
self,
inputs: dict[expressions.Expression, PCollection],
outputs: dict[Any, expressions.Expression],
) ->dict[Any, PCollection]:
"""Construct a Beam graph that evaluates a set of expressions on a set of
input PCollections.
:param inputs: A mapping of placeholder expressions to PCollections.
:param outputs: A mapping of keys to expressions defined in terms of the
placeholders of inputs.
Returns a dictionary whose keys are those of outputs, and whose values are
PCollections corresponding to the values of outputs evaluated at the
values of inputs.
Logically, `_apply_deferred_ops({x: a, y: b}, {f: F(x, y), g: G(x, y)})`
returns `{f: F(a, b), g: G(a, b)}`.
"""
classComputeStage(beam.PTransform):
"""A helper transform that computes a single stage of operations.
"""
def__init__(self, stage):
self.stage=stage
defdefault_label(self):
return'%s:%s'% (self.stage.ops, id(self))
defexpand(self, pcolls):
logging.info('Computing dataframe stage %s for %s', self, self.stage)
scalar_inputs= [exprforexprinself.stage.inputsifis_scalar(expr)]
tabular_inputs= [
exprforexprinself.stage.inputsifnotis_scalar(expr)
]
iflen(tabular_inputs) ==0:
partitioned_pcoll=next(iter(
pcolls.values())).pipeline|beam.Create([{}])
elifself.stage.partitioning!=partitionings.Arbitrary():
# Partitioning required for these operations.
# Compute the number of partitions to use for the inputs based on
# the estimated size of the inputs.
ifself.stage.partitioning==partitionings.Singleton():
# Always a single partition, don't waste time computing sizes.
num_partitions=1
else:
# Estimate the sizes from the outputs of a *previous* stage such
# that using these estimates will not cause a fusion break.
input_sizes= [
estimate_size(input, same_stage_ok=False)
forinputintabular_inputs
]
ifNoneininput_sizes:
# We were unable to (cheaply) compute the size of one or more
# inputs.
num_partitions=DEFAULT_PARTITIONS
else:
num_partitions=beam.pvalue.AsSingleton(
input_sizes
|'FlattenSizes'>>beam.Flatten()
|'SumSizes'>>beam.CombineGlobally(sum)
|'NumPartitions'>>beam.Map(
lambdasize: max(
MIN_PARTITIONS,
min(MAX_PARTITIONS, size//TARGET_PARTITION_SIZE))))
partition_fn=self.stage.partitioning.partition_fn
classPartition(beam.PTransform):
defexpand(self, pcoll):
return (
pcoll
# Attempt to create batches of reasonable size.
|beam.ParDo(_PreBatch())
# Actually partition.
|beam.FlatMap(partition_fn, num_partitions)
# Don't bother shuffling empty partitions.
|beam.Filter(lambdak_df: len(k_df[1])))
# Arrange such that partitioned_pcoll is properly partitioned.
main_pcolls= {
expr._id: pcolls[expr._id] |'Partition_%s_%s'%
(self.stage.partitioning, expr._id) >>Partition()
forexprintabular_inputs
} |beam.CoGroupByKey()
partitioned_pcoll=main_pcolls|beam.ParDo(_ReBatch())
else:
# Already partitioned, or no partitioning needed.
assertlen(tabular_inputs) ==1
tag=tabular_inputs[0]._id
partitioned_pcoll=pcolls[tag] |beam.Map(lambdadf: {tag: df})
side_pcolls= {
expr._id: beam.pvalue.AsSingleton(pcolls[expr._id])
forexprinscalar_inputs
}
# Actually evaluate the expressions.
defevaluate(partition, stage=self.stage, **side_inputs):
deflookup(expr):
# Use proxy if there's no data in this partition
returnexpr.proxy(
).iloc[:0] ifpartition[expr._id] isNoneelsepartition[expr._id]
session=expressions.Session(
dict([(expr, lookup(expr)) forexprintabular_inputs] +
[(expr, side_inputs[expr._id]) forexprinscalar_inputs]))
forexprinstage.outputs:
yieldbeam.pvalue.TaggedOutput(expr._id, expr.evaluate_at(session))
returnpartitioned_pcoll|beam.FlatMap(evaluate, **
side_pcolls).with_outputs()
classStage(object):
"""Used to build up a set of operations that can be fused together.
Note that these Dataframe "stages" contain a CoGBK and hence are often
split across multiple "executable" stages.
"""
def__init__(self, inputs, partitioning):
self.inputs=set(inputs)
if (len(self.inputs) >1and
partitioning.is_subpartitioning_of(partitionings.Index())):
# We have to shuffle to co-locate, might as well partition.
self.partitioning=partitionings.Index()
elifisinstance(partitioning, partitionings.JoinIndex):
# Not an actionable partitioning, use index.
self.partitioning=partitionings.Index()
else:
self.partitioning=partitioning
self.ops= []
self.outputs=set()
def__repr__(self, indent=0):
ifindent:
sep='\n'+' '*indent
else:
sep=''
return (
"Stage[%sinputs=%s, %spartitioning=%s, %sops=%s, %soutputs=%s]"% (
sep,
self.inputs,
sep,
self.partitioning,
sep,
self.ops,
sep,
self.outputs))
# First define some helper functions.
@_memoize
defoutput_partitioning_in_stage(expr, stage):
"""Return the output partitioning of expr when computed in stage,
or returns None if the expression cannot be computed in this stage.
"""
defmaybe_upgrade_to_join_index(partitioning):
ifpartitioning.is_subpartitioning_of(partitionings.JoinIndex()):
returnpartitionings.JoinIndex(expr)
else:
returnpartitioning
ifexprinstage.inputsorexprininputs:
# Inputs are all partitioned by stage.partitioning.
returnmaybe_upgrade_to_join_index(stage.partitioning)
# Anything that's not an input must have arguments
assertlen(expr.args())
arg_partitionings=set(
output_partitioning_in_stage(arg, stage) forarginexpr.args()
ifnotis_scalar(arg))
iflen(arg_partitionings) ==0:
# All inputs are scalars, output partitioning isn't dependent on the
# input.
returnmaybe_upgrade_to_join_index(expr.preserves_partition_by())
iflen(arg_partitionings) >1:
# Arguments must be identically partitioned, can't compute this
# expression here.
returnNone
arg_partitioning=arg_partitionings.pop()
ifnotexpr.requires_partition_by().is_subpartitioning_of(
arg_partitioning):
# Arguments aren't partitioned sufficiently for this expression
returnNone
returnmaybe_upgrade_to_join_index(
expressions.output_partitioning(expr, arg_partitioning))
defis_computable_in_stage(expr, stage):
returnoutput_partitioning_in_stage(expr, stage) isnotNone
defcommon_stages(stage_lists):
# Set intersection, with a preference for earlier items in the list.
ifstage_lists:
forstageinstage_lists[0]:
ifall(stageinotherforotherinstage_lists[1:]):
yieldstage
@_memoize
defis_scalar(expr):
returnnotisinstance(expr.proxy(), pd.core.generic.NDFrame)
@_memoize
defexpr_to_stages(expr):
ifexprininputs:
# Don't create a stage for each input, but it is still useful to record
# what which stages inputs are available from.
return []
# First attempt to compute this expression as part of an existing stage,
# if possible.
ifall(argininputsforarginexpr.args()):
# All input arguments; try to pick a stage that already has as many
# of the inputs, correctly partitioned, as possible.
inputs_by_stage=collections.defaultdict(int)
forarginexpr.args():
forstageinexpr_to_stages(arg):
ifis_computable_in_stage(expr, stage):
inputs_by_stage[stage] +=1+100* (
expr.requires_partition_by() ==stage.partitioning)
ifinputs_by_stage:
# Take the stage with the largest count.
stage=max(inputs_by_stage.items(), key=lambdakv: kv[1])[0]
else:
stage=None
else:
# Try to pick a stage that has all the available non-input expressions.
# TODO(robertwb): Baring any that have all of them, we could try and
# pick one that has the most, but we need to ensure it is not a
# predecessor of any of the missing argument's stages.
forstageincommon_stages([expr_to_stages(arg) forarginexpr.args()
ifargnotininputs]):
ifis_computable_in_stage(expr, stage):
break
else:
stage=None
ifstageisNone:
# No stage available, compute this expression as part of a new stage.
stage=Stage([
argforarginexpr.args()
ifnotisinstance(arg, expressions.ConstantExpression)
],
expr.requires_partition_by())
forarginexpr.args():
# For each argument, declare that it is also available in
# this new stage.
expr_to_stages(arg).append(stage)
# It also must be declared as an output of the producing stage.
expr_to_stage(arg).outputs.add(arg)
stage.ops.append(expr)
# Ensure that any inputs for the overall transform are added
# in downstream stages.
forarginexpr.args():
ifargininputs:
stage.inputs.add(arg)
# This is a list as given expression may be available in many stages.
return [stage]
defexpr_to_stage(expr):
# Any will do; the first requires the fewest intermediate stages.
returnexpr_to_stages(expr)[0]
# Ensure each output is computed.
forexprinoutputs.values():
ifexprnotininputs:
expr_to_stage(expr).outputs.add(expr)
@_memoize
defstage_to_result(stage):
return {expr._id: expr_to_pcoll(expr)
forexprinstage.inputs} |ComputeStage(stage)
@_memoize
defexpr_to_pcoll(expr):
ifexprininputs:
returninputs[expr]
else:
returnstage_to_result(expr_to_stage(expr))[expr._id]
@_memoize
defestimate_size(expr, same_stage_ok):
# Returns a pcollection of ints whose sum is the estimated size of the
# given expression.
pipeline=next(iter(inputs.values())).pipeline
label='Size[%s, %s]'% (expr._id, same_stage_ok)
ifis_scalar(expr):
returnpipeline|label>>beam.Create([0])
elifsame_stage_ok:
returnexpr_to_pcoll(expr) |label>>beam.Map(_total_memory_usage)
elifexprininputs:
returnNone
else:
# This is the stage to avoid.
expr_stage=expr_to_stage(expr)
# If the stage doesn't start with a shuffle, it's not safe to fuse
# the computation into its parent either.
has_shuffle=expr_stage.partitioning!=partitionings.Arbitrary()
# We assume the size of an expression is the sum of the size of its
# inputs, which may be off by quite a bit, but the goal is to get
# within an order of magnitude or two.
arg_sizes= []
forarginexpr.args():
ifis_scalar(arg):
continue
elifargininputs:
returnNone
arg_size=estimate_size(
arg,
same_stage_ok=has_shuffleandexpr_to_stage(arg) !=expr_stage)
ifarg_sizeisNone:
returnNone
arg_sizes.append(arg_size)
returnarg_sizes|label>>beam.Flatten(pipeline=pipeline)
# Now we can compute and return the result.
return {k: expr_to_pcoll(expr) fork, exprinoutputs.items()}
def_total_memory_usage(frame):
assertisinstance(frame, (pd.core.generic.NDFrame, pd.Index))
try:
size=frame.memory_usage()
ifnotisinstance(size, int):
size=size.sum() +PER_COL_OVERHEAD*len(size)
else:
size+=PER_COL_OVERHEAD
returnsize
exceptAttributeError:
# Don't know, assume it's really big.
float('inf')
class_PreBatch(beam.DoFn):
def__init__(
self, target_size=TARGET_PARTITION_SIZE, min_size=MIN_PARTITION_SIZE):
self._target_size=target_size
self._min_size=min_size
defstart_bundle(self):
self._parts=collections.defaultdict(list)
self._running_size=0
defprocess(
self,
part,
window=beam.DoFn.WindowParam,
timestamp=beam.DoFn.TimestampParam):
part_size=_total_memory_usage(part)
ifpart_size>=self._min_size:
yieldpart
else:
self._running_size+=part_size
self._parts[window, timestamp].append(part)
ifself._running_size>=self._target_size:
yieldfromself.finish_bundle()
deffinish_bundle(self):
for (window, timestamp), partsinself._parts.items():
yieldwindowed_value.WindowedValue(_concat(parts), timestamp, (window, ))
self.start_bundle()
class_ReBatch(beam.DoFn):
"""Groups all the parts from various workers into the same dataframe.
Also groups across partitions, up to a given data size, to recover some
efficiency in the face of over-partitioning.
"""
def__init__(
self, target_size=TARGET_PARTITION_SIZE, min_size=MIN_PARTITION_SIZE):
self._target_size=target_size
self._min_size=min_size
defstart_bundle(self):
self._parts=collections.defaultdict(lambda: collections.defaultdict(list))
self._running_size=0
defprocess(
self,
element,
window=beam.DoFn.WindowParam,
timestamp=beam.DoFn.TimestampParam):
_, tagged_parts=element
fortag, partsintagged_parts.items():
forpartinparts:
self._running_size+=_total_memory_usage(part)
self._parts[window, timestamp][tag].extend(parts)
ifself._running_size>=self._target_size:
yieldfromself.finish_bundle()
deffinish_bundle(self):
for (window, timestamp), tagged_partsinself._parts.items():
yieldwindowed_value.WindowedValue( # yapf break
{
tag: _concat(parts) ifpartselseNone
for (tag, parts) intagged_parts.items()
},
timestamp, (window, ))
self.start_bundle()
def_memoize(f):
cache= {}
defwrapper(*args, **kwargs):
key=args, tuple(sorted(kwargs.items()))
ifkeynotincache:
cache[key] =f(*args, **kwargs)
returncache[key]
returnwrapper
def_dict_union(dicts):
result= {}
fordindicts:
result.update(d)
returnresult
def_concat(parts):
iflen(parts) ==1:
returnparts[0]
else:
returnpd.concat(parts)
def_flatten(
valueish: Union[T, list[T], tuple[T], dict[Any, T]],
root: tuple[Any, ...] = (),
) ->Mapping[tuple[Any, ...], T]:
"""Given a nested structure of dicts, tuples, and lists, return a flat
dictionary where the values are the leafs and the keys are the "paths" to
these leaves.
For example `{a: x, b: (y, z)}` becomes `{(a,): x, (b, 0): y, (b, 1): c}`.
"""
ifisinstance(valueish, dict):
return_dict_union(_flatten(v, root+ (k, )) fork, vinvalueish.items())
elifisinstance(valueish, (tuple, list)):
return_dict_union(
_flatten(v, root+ (ix, )) forix, vinenumerate(valueish))
else:
return {root: valueish}
def_substitute(valueish, replacements, root=()):
"""Substitutes the values in valueish with those in replacements where the
keys are as in _flatten.
For example,
```
_substitute(
{a: x, b: (y, z)},
{(a,): X, (b, 0): Y, (b, 1): Z})
```
returns `{a: X, b: (Y, Z)}`.
"""
ifisinstance(valueish, dict):
returntype(valueish)({
k: _substitute(v, replacements, root+ (k, ))
for (k, v) invalueish.items()
})
elifisinstance(valueish, (tuple, list)):
returntype(valueish)((
_substitute(v, replacements, root+ (ix, ))
for (ix, v) inenumerate(valueish)))
else:
returnreplacements[root]