def expand(self, pcolls): scalar_inputs = [ expr for expr in self.stage.inputs if is_scalar(expr) ] tabular_inputs = [ expr for expr in self.stage.inputs if not is_scalar(expr) ] if len(tabular_inputs) == 0: partitioned_pcoll = next( pcolls.values()).pipeline | beam.Create([{}]) elif self.stage.partitioning != partitionings.Nothing(): # Partitioning required for these operations. # Compute the number of partitions to use for the inputs based on # the estimated size of the inputs. if self.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) for input in tabular_inputs ] if None in input_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(lambda size: max( MIN_PARTITIONS, min(MAX_PARTITIONS, size // TARGET_PARTITION_SIZE)))) # Arrange such that partitioned_pcoll is properly partitioned. main_pcolls = { expr._id: pcolls[expr._id] | 'Partition_%s_%s' % (self.stage.partitioning, expr._id) >> beam.FlatMap( self.stage.partitioning.partition_fn, num_partitions) for expr in tabular_inputs } | beam.CoGroupByKey() partitioned_pcoll = main_pcolls | beam.ParDo(_ReBatch()) else: # Already partitioned, or no partitioning needed. assert len(tabular_inputs) == 1 tag = tabular_inputs[0]._id partitioned_pcoll = pcolls[tag] | beam.Map( lambda df: {tag: df}) side_pcolls = { expr._id: beam.pvalue.AsSingleton(pcolls[expr._id]) for expr in scalar_inputs } # Actually evaluate the expressions. def evaluate(partition, stage=self.stage, **side_inputs): session = expressions.Session( dict([(expr, partition[expr._id]) for expr in tabular_inputs] + [(expr, side_inputs[expr._id]) for expr in scalar_inputs])) for expr in stage.outputs: yield beam.pvalue.TaggedOutput( expr._id, expr.evaluate_at(session)) return partitioned_pcoll | beam.FlatMap( evaluate, **side_pcolls).with_outputs()
def wrapper(*args, **kwargs): for key, values in restrictions.items(): if key in kwargs: value = kwargs[key] else: try: ix = getfullargspec(func).args.index(key) except ValueError: # TODO: fix for delegation? continue if len(args) <= ix: continue value = args[ix] if callable(values): check = values elif isinstance(values, list): check = lambda x, values=values: x in values else: check = lambda x, value=value: x == value if not check(value): raise NotImplementedError( '%s=%s not supported for %s' % (key, value, name)) deferred_arg_indices = [] deferred_arg_exprs = [] constant_args = [None] * len(args) from apache_beam.dataframe.frames import _DeferredIndex for ix, arg in enumerate(args): if isinstance(arg, DeferredBase): deferred_arg_indices.append(ix) deferred_arg_exprs.append(arg._expr) elif isinstance(arg, _DeferredIndex): # TODO(robertwb): Consider letting indices pass through as indices. # This would require updating the partitioning code, as indices don't # have indices. deferred_arg_indices.append(ix) deferred_arg_exprs.append( expressions.ComputedExpression( 'index_as_series', lambda ix: ix.index.to_series(), # yapf break [arg._frame._expr], preserves_partition_by=partitionings.Singleton(), requires_partition_by=partitionings.Arbitrary())) elif isinstance(arg, pd.core.generic.NDFrame): deferred_arg_indices.append(ix) deferred_arg_exprs.append(expressions.ConstantExpression(arg, arg[0:0])) else: constant_args[ix] = arg deferred_kwarg_keys = [] deferred_kwarg_exprs = [] constant_kwargs = {key: None for key in kwargs} for key, arg in kwargs.items(): if isinstance(arg, DeferredBase): deferred_kwarg_keys.append(key) deferred_kwarg_exprs.append(arg._expr) elif isinstance(arg, pd.core.generic.NDFrame): deferred_kwarg_keys.append(key) deferred_kwarg_exprs.append( expressions.ConstantExpression(arg, arg[0:0])) else: constant_kwargs[key] = arg deferred_exprs = deferred_arg_exprs + deferred_kwarg_exprs if inplace: actual_func = _copy_and_mutate(func) else: actual_func = func def apply(*actual_args): actual_args, actual_kwargs = (actual_args[:len(deferred_arg_exprs)], actual_args[len(deferred_arg_exprs):]) full_args = list(constant_args) for ix, arg in zip(deferred_arg_indices, actual_args): full_args[ix] = arg full_kwargs = dict(constant_kwargs) for key, arg in zip(deferred_kwarg_keys, actual_kwargs): full_kwargs[key] = arg return actual_func(*full_args, **full_kwargs) if (requires_partition_by.is_subpartitioning_of(partitionings.Index()) and sum(isinstance(arg.proxy(), pd.core.generic.NDFrame) for arg in deferred_exprs) > 1): # Implicit join on index if there is more than one indexed input. actual_requires_partition_by = partitionings.Index() else: actual_requires_partition_by = requires_partition_by result_expr = expressions.ComputedExpression( name, apply, deferred_exprs, requires_partition_by=actual_requires_partition_by, preserves_partition_by=preserves_partition_by) if inplace: args[0]._expr = result_expr else: return DeferredFrame.wrap(result_expr)
def _proxy_function( func, # type: Union[Callable, str] name=None, # type: Optional[str] restrictions=None, # type: Optional[Dict[str, Union[Any, List[Any]]]] inplace=False, # type: bool requires_partition_by=partitionings.Singleton(), # type: partitionings.Partitioning preserves_partition_by=partitionings.Nothing(), # type: partitionings.Partitioning ): if name is None: if isinstance(func, str): name = func else: name = func.__name__ if restrictions is None: restrictions = {} def wrapper(*args, **kwargs): for key, values in (): #restrictions.items(): if key in kwargs: value = kwargs[key] else: try: ix = _getargspec(func).args.index(key) except ValueError: # TODO: fix for delegation? continue if len(args) <= ix: continue value = args[ix] if not isinstance(values, list): values = [values] if value not in values: raise NotImplementedError( '%s=%s not supported for %s' % (key, value, name)) deferred_arg_indices = [] deferred_arg_exprs = [] constant_args = [None] * len(args) for ix, arg in enumerate(args): if isinstance(arg, DeferredBase): deferred_arg_indices.append(ix) deferred_arg_exprs.append(arg._expr) elif isinstance(arg, pd.core.generic.NDFrame): deferred_arg_indices.append(ix) deferred_arg_exprs.append(expressions.ConstantExpression(arg, arg[0:0])) else: constant_args[ix] = arg if inplace: actual_func = copy_and_mutate(func) else: actual_func = func def apply(*actual_args): full_args = list(constant_args) for ix, arg in zip(deferred_arg_indices, actual_args): full_args[ix] = arg return actual_func(*full_args, **kwargs) result_expr = expressions.ComputedExpression( name, apply, deferred_arg_exprs, requires_partition_by=requires_partition_by, preserves_partition_by=preserves_partition_by) if inplace: args[0]._expr = result_expr else: return DeferredFrame.wrap(result_expr) return wrapper
def evaluate(self, expr): import pandas as pd import collections def is_scalar(expr): return not isinstance(expr.proxy(), pd.core.generic.NDFrame) if expr not in self._bindings: if is_scalar(expr) or not expr.args(): result = super(PartitioningSession, self).evaluate(expr) else: scaler_args = [arg for arg in expr.args() if is_scalar(arg)] def evaluate_with(input_partitioning): parts = collections.defaultdict(lambda: Session( {arg: self.evaluate(arg) for arg in scaler_args})) for arg in expr.args(): if not is_scalar(arg): input = self.evaluate(arg) for key, part in input_partitioning.test_partition_fn( input): parts[key]._bindings[arg] = part if not parts: parts[None] # Create at least one entry. results = [] for session in parts.values(): if any( len(session.lookup(arg)) for arg in expr.args() if not is_scalar(arg)): results.append(session.evaluate(expr)) expected_output_partitioning = output_partitioning( expr, input_partitioning) if not expected_output_partitioning.check(results): raise AssertionError( f"""Expression does not preserve partitioning! Expression: {expr} Requires: {expr.requires_partition_by()} Preserves: {expr.preserves_partition_by()} Input partitioning: {input_partitioning} Expected output partitioning: {expected_output_partitioning} """) if results: return pd.concat(results) else: # Choose any single session. return next(iter(parts.values())).evaluate(expr) # Store random state so it can be re-used for each execution, in case # the expression is part of a test that relies on the random seed. random_state = random.getstate() result = None # Run with all supported partitionings s.t. the smallest subpartitioning # is used last. This way the final result is computed with the most # challenging partitioning. Avoids heisenbugs where sometimes the result # is computed trivially with Singleton partitioning and passes. for input_partitioning in sorted( set([ expr.requires_partition_by(), partitionings.Arbitrary(), partitionings.Index(), partitionings.Singleton() ])): if not expr.requires_partition_by().is_subpartitioning_of( input_partitioning): continue random.setstate(random_state) result = evaluate_with(input_partitioning) assert result is not None self._bindings[expr] = result return self._bindings[expr]
def evaluate(self, expr): import pandas as pd import collections def is_scalar(expr): return not isinstance(expr.proxy(), pd.core.generic.NDFrame) def difficulty(partitioning): """Imposes an ordering on partitionings where the largest schemes are the most likely to reveal an error. This order is different from the one defined by is_subpartitioning_of: Nothing() > Index() > ... > Index([i,j]) > Index([j]) > Singleton() """ if isinstance(partitioning, partitionings.Singleton): return -float('inf') elif isinstance(partitioning, partitionings.Index): if partitioning._levels is None: return 1_000_000 else: return len(partitioning._levels) elif isinstance(partitioning, partitionings.Nothing): return float('inf') if expr not in self._bindings: if is_scalar(expr) or not expr.args(): result = super(PartitioningSession, self).evaluate(expr) else: scaler_args = [arg for arg in expr.args() if is_scalar(arg)] def evaluate_with(input_partitioning): parts = collections.defaultdict( lambda: Session({arg: self.evaluate(arg) for arg in scaler_args})) for arg in expr.args(): if not is_scalar(arg): input = self.evaluate(arg) for key, part in input_partitioning.test_partition_fn(input): parts[key]._bindings[arg] = part if not parts: parts[None] # Create at least one entry. results = [] for session in parts.values(): if any(len(session.lookup(arg)) for arg in expr.args() if not is_scalar(arg)): results.append(session.evaluate(expr)) expected_output_partitioning = expr.preserves_partition_by( ) if input_partitioning.is_subpartitioning_of( expr.preserves_partition_by()) else input_partitioning if not expected_output_partitioning.check(results): raise AssertionError( f"""Expression does not preserve partitioning! Expression: {expr} Requires: {expr.requires_partition_by()} Preserves: {expr.preserves_partition_by()} Input partitioning: {input_partitioning} Expected output partitioning: {expected_output_partitioning} """) if results: return pd.concat(results) else: # Choose any single session. return next(iter(parts.values())).evaluate(expr) # Store random state so it can be re-used for each execution, in case # the expression is part of a test that relies on the random seed. random_state = random.getstate() # Run with all supported partitionings in order of ascending # "difficulty". This way the final result is computed with the # most challenging partitioning. Avoids heisenbugs where sometimes # the result is computed trivially with Singleton partitioning and # passes. for input_partitioning in sorted(set([expr.requires_partition_by(), partitionings.Nothing(), partitionings.Index(), partitionings.Singleton()]), key=difficulty): if not input_partitioning.is_subpartitioning_of( expr.requires_partition_by()): continue random.setstate(random_state) result = evaluate_with(input_partitioning) self._bindings[expr] = result return self._bindings[expr]
def _proxy_function( func, # type: Union[Callable, str] name=None, # type: Optional[str] restrictions=None, # type: Optional[Dict[str, Union[Any, List[Any], Callable[[Any], bool]]]] inplace=False, # type: bool requires_partition_by=partitionings.Singleton(), # type: partitionings.Partitioning preserves_partition_by=partitionings.Nothing(), # type: partitionings.Partitioning ): if name is None: if isinstance(func, str): name = func else: name = func.__name__ if restrictions is None: restrictions = {} def wrapper(*args, **kwargs): for key, values in restrictions.items(): if key in kwargs: value = kwargs[key] else: try: ix = _getargspec(func).args.index(key) except ValueError: # TODO: fix for delegation? continue if len(args) <= ix: continue value = args[ix] if callable(values): check = values elif isinstance(values, list): check = lambda x, values=values: x in values else: check = lambda x, value=value: x == value if not check(value): raise NotImplementedError( '%s=%s not supported for %s' % (key, value, name)) deferred_arg_indices = [] deferred_arg_exprs = [] constant_args = [None] * len(args) for ix, arg in enumerate(args): if isinstance(arg, DeferredBase): deferred_arg_indices.append(ix) deferred_arg_exprs.append(arg._expr) elif isinstance(arg, pd.core.generic.NDFrame): deferred_arg_indices.append(ix) deferred_arg_exprs.append(expressions.ConstantExpression(arg, arg[0:0])) else: constant_args[ix] = arg deferred_kwarg_keys = [] deferred_kwarg_exprs = [] constant_kwargs = {key: None for key in kwargs} for key, arg in kwargs.items(): if isinstance(arg, DeferredBase): deferred_kwarg_keys.append(key) deferred_kwarg_exprs.append(arg._expr) elif isinstance(arg, pd.core.generic.NDFrame): deferred_kwarg_keys.append(key) deferred_kwarg_exprs.append( expressions.ConstantExpression(arg, arg[0:0])) else: constant_kwargs[key] = arg deferred_exprs = deferred_arg_exprs + deferred_kwarg_exprs if inplace: actual_func = copy_and_mutate(func) else: actual_func = func def apply(*actual_args): actual_args, actual_kwargs = (actual_args[:len(deferred_arg_exprs)], actual_args[len(deferred_arg_exprs):]) full_args = list(constant_args) for ix, arg in zip(deferred_arg_indices, actual_args): full_args[ix] = arg full_kwargs = dict(constant_kwargs) for key, arg in zip(deferred_kwarg_keys, actual_kwargs): full_kwargs[key] = arg return actual_func(*full_args, **full_kwargs) if any(isinstance(arg.proxy(), pd.core.generic.NDFrame) for arg in deferred_exprs) and not requires_partition_by.is_subpartitioning_of( partitionings.Index()): # Implicit join on index. actual_requires_partition_by = partitionings.Index() else: actual_requires_partition_by = requires_partition_by result_expr = expressions.ComputedExpression( name, apply, deferred_exprs, requires_partition_by=actual_requires_partition_by, preserves_partition_by=preserves_partition_by) if inplace: args[0]._expr = result_expr else: return DeferredFrame.wrap(result_expr) return wrapper