def test_preserves_singleton_output_partitioning(self): # Empty DataFrame with one column and two index levels input_expr = expressions.ConstantExpression( pd.DataFrame(columns=["column"], index=[[], []])) preserves_only_singleton = expressions.ComputedExpression( 'preserves_only_singleton', # index is replaced with an entirely new one, so # if we were partitioned by Index we're not anymore. lambda df: df.set_index('column'), [input_expr], requires_partition_by=partitionings.Arbitrary(), preserves_partition_by=partitionings.Singleton()) for partitioning in (partitionings.Singleton(), ): self.assertEqual( expressions.output_partitioning(preserves_only_singleton, partitioning), partitioning, f"Should preserve {partitioning}") for partitioning in (partitionings.Index([0]), partitionings.Index(), partitionings.Arbitrary()): self.assertEqual( expressions.output_partitioning(preserves_only_singleton, partitioning), partitionings.Arbitrary(), f"Should NOT preserve {partitioning}")
def concat( objs, axis, join, ignore_index, keys, levels, names, verify_integrity, sort, copy): if ignore_index: raise NotImplementedError('concat(ignore_index)') if levels: raise NotImplementedError('concat(levels)') if isinstance(objs, Mapping): if keys is None: keys = list(objs.keys()) objs = [objs[k] for k in keys] else: objs = list(objs) if keys is None: preserves_partitioning = partitionings.Arbitrary() else: # Index 0 will be a new index for keys, only partitioning by the original # indexes (1 to N) will be preserved. nlevels = min(o._expr.proxy().index.nlevels for o in objs) preserves_partitioning = partitionings.Index( [i for i in range(1, nlevels + 1)]) deferred_none = expressions.ConstantExpression(None) exprs = [deferred_none if o is None else o._expr for o in objs] if axis in (1, 'columns'): required_partitioning = partitionings.Index() elif verify_integrity: required_partitioning = partitionings.Index() else: required_partitioning = partitionings.Arbitrary() return frame_base.DeferredBase.wrap( expressions.ComputedExpression( 'concat', lambda *objs: pd.concat( objs, axis=axis, join=join, ignore_index=ignore_index, keys=keys, levels=levels, names=names, verify_integrity=verify_integrity), # yapf break exprs, requires_partition_by=required_partitioning, preserves_partition_by=preserves_partitioning))
def __init__(self, inputs, partitioning): self.inputs = set(inputs) if (len(self.inputs) > 1 and partitioning.is_subpartitioning_of(partitionings.Index())): # We have to shuffle to co-locate, might as well partition. self.partitioning = partitionings.Index() elif isinstance(partitioning, partitionings.JoinIndex): # Not an actionable partitioning, use index. self.partitioning = partitionings.Index() else: self.partitioning = partitioning self.ops = [] self.outputs = set()
def concat( objs, axis, join, ignore_index, keys, levels, names, verify_integrity, sort, copy): if ignore_index: raise NotImplementedError('concat(ignore_index)') if levels: raise NotImplementedError('concat(levels)') if isinstance(objs, Mapping): if keys is None: keys = list(objs.keys()) objs = [objs[k] for k in keys] else: objs = list(objs) deferred_none = expressions.ConstantExpression(None) exprs = [deferred_none if o is None else o._expr for o in objs] if axis in (1, 'columns'): required_partitioning = partitionings.Index() elif verify_integrity: required_partitioning = partitionings.Index() else: required_partitioning = partitionings.Nothing() return frame_base.DeferredBase.wrap( expressions.ComputedExpression( 'concat', lambda *objs: pd.concat( objs, axis=axis, join=join, ignore_index=ignore_index, keys=keys, levels=levels, names=names, verify_integrity=verify_integrity), # yapf break exprs, requires_partition_by=required_partitioning, preserves_partition_by=partitionings.Index()))
def dot(self, other): # We want to broadcast the right hand side to all partitions of the left. # This is OK, as its index must be the same size as the columns set of self, # so cannot be too large. class AsScalar(object): def __init__(self, value): self.value = value if isinstance(other, frame_base.DeferredFrame): proxy = other._expr.proxy() with expressions.allow_non_parallel_operations(): side = expressions.ComputedExpression( 'as_scalar', lambda df: AsScalar(df), [other._expr], requires_partition_by=partitionings.Singleton()) else: proxy = pd.DataFrame(columns=range(len(other[0]))) side = expressions.ConstantExpression(AsScalar(other)) return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'dot', lambda left, right: left @ right.value, [self._expr, side], requires_partition_by=partitionings.Nothing(), preserves_partition_by=partitionings.Index(), proxy=proxy))
def update(self, other): self._expr = expressions.ComputedExpression( 'update', lambda df, other: df.update(other) or df, [self._expr, other._expr], preserves_partition_by=partitionings.Singleton(), requires_partition_by=partitionings.Index())
def join(self, other, on, **kwargs): if on is not None: reindex, revert = self._cols_as_temporary_index(on) return revert(reindex(self).join(other, **kwargs)) if isinstance(other, list): other_is_list = True else: other = [other] other_is_list = False placeholder = object() other_exprs = [ df._expr for df in other if isinstance(df, frame_base.DeferredFrame)] const_others = [ placeholder if isinstance(df, frame_base.DeferredFrame) else df for df in other] def fill_placeholders(values): values = iter(values) filled = [ next(values) if df is placeholder else df for df in const_others] if other_is_list: return filled else: return filled[0] return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'join', lambda df, *deferred_others: df.join( fill_placeholders(deferred_others), **kwargs), [self._expr] + other_exprs, preserves_partition_by=partitionings.Singleton(), requires_partition_by=partitionings.Index()))
def __init__(self, inputs, partitioning): self.inputs = set(inputs) if len(self.inputs) > 1 and partitioning == partitionings.Nothing(): # We have to shuffle to co-locate, might as well partition. self.partitioning = partitionings.Index() else: self.partitioning = partitioning self.ops = [] self.outputs = set()
def agg(self, fn): if not callable(fn): raise NotImplementedError(fn) return DeferredDataFrame( expressions.ComputedExpression( 'agg', lambda df: df.agg(fn), [self._expr], requires_partition_by=partitionings.Index(), preserves_partition_by=partitionings.Singleton()))
def wrapper(self, *args, **kargs): assert isinstance(self, DeferredGroupBy) ungrouped = self._expr.args()[0] post_agg = expressions.ComputedExpression( name, lambda df: func(df.groupby(level=list(range(df.index.nlevels)))), [ungrouped], requires_partition_by=partitionings.Index(), preserves_partition_by=partitionings.Singleton()) return frame_base.DeferredFrame.wrap(post_agg)
def unstack(self, *args, **kwargs): if self._expr.proxy().index.nlevels == 1: return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'unstack', lambda df: df.unstack(*args, **kwargs), [self._expr], requires_partition_by=partitionings.Index())) else: raise frame_base.WontImplementError('non-deferred column values')
def merge( self, right, on, left_on, right_on, left_index, right_index, **kwargs): self_proxy = self._expr.proxy() right_proxy = right._expr.proxy() # Validate with a pandas call. _ = self_proxy.merge( right_proxy, on=on, left_on=left_on, right_on=right_on, left_index=left_index, right_index=right_index, **kwargs) if not any([on, left_on, right_on, left_index, right_index]): on = [col for col in self_proxy.columns() if col in right_proxy.columns()] if not left_on: left_on = on elif not isinstance(left_on, list): left_on = [left_on] if not right_on: right_on = on elif not isinstance(right_on, list): right_on = [right_on] if left_index: indexed_left = self else: indexed_left = self.set_index(left_on, drop=False) if right_index: indexed_right = right else: indexed_right = right.set_index(right_on, drop=False) merged = frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'merge', lambda left, right: left.merge( right, left_index=True, right_index=True, **kwargs), [indexed_left._expr, indexed_right._expr], preserves_partition_by=partitionings.Singleton(), requires_partition_by=partitionings.Index())) if left_index or right_index: return merged else: return merged.reset_index(drop=True)
def groupby(self, cols): # TODO: what happens to the existing index? # We set the columns to index as we have a notion of being partitioned by # index, but not partitioned by an arbitrary subset of columns. return DeferredGroupBy( expressions.ComputedExpression( 'groupbyindex', lambda df: df.groupby(level=list(range(df.index.nlevels))), [self.set_index(cols)._expr], requires_partition_by=partitionings.Index(), preserves_partition_by=partitionings.Singleton()))
def agg(self, fn): if not callable(fn): # TODO: Add support for strings in (UN)LIFTABLE_AGGREGATIONS. Test by # running doctests for pandas.core.groupby.generic raise NotImplementedError('GroupBy.agg currently only supports callable ' 'arguments') return DeferredDataFrame( expressions.ComputedExpression( 'agg', lambda df: df.agg(fn), [self._expr], requires_partition_by=partitionings.Index(), preserves_partition_by=partitionings.Singleton()))
def test_preserves_index_output_partitioning(self): # Empty DataFrame with two columns and two index levels input_expr = expressions.ConstantExpression( pd.DataFrame(columns=["foo", "bar"], index=[[], []])) preserves_partial_index = expressions.ComputedExpression( 'preserves_partial_index', # This adds an additional index level, so we'd only preserve # partitioning on the two index levels that existed before. lambda df: df.set_index('foo', append=True), [input_expr], requires_partition_by=partitionings.Arbitrary(), preserves_partition_by=partitionings.Index([0, 1])) for partitioning in ( partitionings.Singleton(), partitionings.Index([0]), partitionings.Index([1]), partitionings.Index([0, 1]), ): self.assertEqual( expressions.output_partitioning(preserves_partial_index, partitioning), partitioning, f"Should preserve {partitioning}") for partitioning in (partitionings.Index([0, 1, 2]), partitionings.Index(), partitionings.Arbitrary()): self.assertEqual( expressions.output_partitioning(preserves_partial_index, partitioning), partitionings.Arbitrary(), f"Should NOT preserve {partitioning}")
def dot(self, other): left = self._expr if isinstance(other, DeferredSeries): right = expressions.ComputedExpression( 'to_dataframe', pd.DataFrame, [other._expr], requires_partition_by=partitionings.Nothing(), preserves_partition_by=partitionings.Index()) right_is_series = True elif isinstance(other, DeferredDataFrame): right = other._expr right_is_series = False else: raise frame_base.WontImplementError('non-deferred result') dots = expressions.ComputedExpression( 'dot', # Transpose so we can sum across rows. (lambda left, right: pd.DataFrame(left @ right).T), [left, right], requires_partition_by=partitionings.Index()) with expressions.allow_non_parallel_operations(True): sums = expressions.ComputedExpression( 'sum', lambda dots: dots.sum(), # [dots], requires_partition_by=partitionings.Singleton()) if right_is_series: result = expressions.ComputedExpression( 'extract', lambda df: df[0], [sums], requires_partition_by=partitionings.Singleton()) else: result = sums return frame_base.DeferredFrame.wrap(result)
def __init__( self, # type: ComputedExpression name, # type: str func, # type: Callable[...,T] args, # type: Iterable[Expression] proxy=None, # type: Optional[T] _id=None, # type: Optional[str] requires_partition_by=partitionings.Index( ), # type: partitionings.Partitioning preserves_partition_by=partitionings.Singleton( ), # type: partitionings.Partitioning ): """Initialize a computed expression. Args: name: The name of this expression. func: The function that will be used to compute the value of this expression. Should accept arguments of the types returned when evaluating the `args` expressions. args: The list of expressions that will be used to produce inputs to `func`. proxy: (Optional) a proxy object with same type as the objects that this ComputedExpression will produce at execution time. If not provided, a proxy will be generated using `func` and the proxies of `args`. _id: (Optional) a string to uniquely identify this expression. requires_partition_by: The required (common) partitioning of the args. preserves_partition_by: The level of partitioning preserved. """ if (not _get_allow_non_parallel() and isinstance( requires_partition_by, partitionings.Singleton)): reason = requires_partition_by.reason or ( f"Encountered non-parallelizable form of {name!r}.") raise NonParallelOperation( f"{reason}\n" "Consider using an allow_non_parallel_operations block if you're " "sure you want to do this. See " "https://s.apache.org/dataframe-non-parallel-operations for more " "information.") args = tuple(args) if proxy is None: proxy = func(*(arg.proxy() for arg in args)) super(ComputedExpression, self).__init__(name, proxy, _id) self._func = func self._args = args self._requires_partition_by = requires_partition_by self._preserves_partition_by = preserves_partition_by
def __getitem__(self, index): if isinstance(index, tuple): rows, cols = index return self[rows][cols] elif isinstance(index, list) and index and isinstance(index[0], bool): # Aligned by numerical index. raise NotImplementedError(type(index)) elif isinstance(index, list): # Select rows, but behaves poorly on missing values. raise NotImplementedError(type(index)) elif isinstance(index, slice): args = [self._frame._expr] func = lambda df: df.loc[index] elif isinstance(index, frame_base.DeferredFrame): args = [self._frame._expr, index._expr] func = lambda df, index: df.loc[index] elif callable(index): def checked_callable_index(df): computed_index = index(df) if isinstance(computed_index, tuple): row_index, _ = computed_index else: row_index = computed_index if isinstance(row_index, list) and row_index and isinstance( row_index[0], bool): raise NotImplementedError(type(row_index)) elif not isinstance(row_index, (slice, pd.Series)): raise NotImplementedError(type(row_index)) return computed_index args = [self._frame._expr] func = lambda df: df.loc[checked_callable_index] else: raise NotImplementedError(type(index)) return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'loc', func, args, requires_partition_by=( partitionings.Index() if len(args) > 1 else partitionings.Nothing()), preserves_partition_by=partitionings.Singleton()))
def __init__( self, # type: ComputedExpression name, # type: str func, # type: Callable[...,T] args, # type: Iterable[Expression] proxy=None, # type: Optional[T] _id=None, # type: Optional[str] requires_partition_by=partitionings.Index( ), # type: partitionings.Partitioning preserves_partition_by=partitionings.Nothing( ), # type: partitionings.Partitioning ): """Initialize a computed expression. Args: name: The name of this expression. func: The function that will be used to compute the value of this expression. Should accept arguments of the types returned when evaluating the `args` expressions. args: The list of expressions that will be used to produce inputs to `func`. proxy: (Optional) a proxy object with same type as the objects that this ComputedExpression will produce at execution time. If not provided, a proxy will be generated using `func` and the proxies of `args`. _id: (Optional) a string to uniquely identify this expression. requires_partition_by_index: Whether this expression requires its argument(s) to be partitioned by index. preserves_partition_by_index: Whether the result of this expression will be partitioned by index whenever all of its inputs are partitioned by index. """ if (not _get_allow_non_parallel() and requires_partition_by == partitionings.Singleton()): raise NonParallelOperation( "Using non-parallel form of %s " "outside of allow_non_parallel_operations block." % name) args = tuple(args) if proxy is None: proxy = func(*(arg.proxy() for arg in args)) super(ComputedExpression, self).__init__(name, proxy, _id) self._func = func self._args = args self._requires_partition_by = requires_partition_by self._preserves_partition_by = preserves_partition_by
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 = 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() for input_partitioning in set([ expr.requires_partition_by(), partitionings.Nothing(), partitionings.Index(), partitionings.Singleton() ]): if not input_partitioning.is_subpartitioning_of( expr.requires_partition_by()): continue random.setstate(random_state) # TODO(BEAM-11324): Consider verifying result is always the same result = evaluate_with(input_partitioning) self._bindings[expr] = result return self._bindings[expr]
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) if any( isinstance(arg.proxy(), pd.core.generic.NDFrame) for arg in deferred_arg_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_arg_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 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 preserves_partition_by(self): return partitionings.Index()
class DeferredDataFrame(frame_base.DeferredFrame): @property def T(self): return self.transpose() def groupby(self, cols): # TODO: what happens to the existing index? # We set the columns to index as we have a notion of being partitioned by # index, but not partitioned by an arbitrary subset of columns. return DeferredGroupBy( expressions.ComputedExpression( 'groupbyindex', lambda df: df.groupby(level=list(range(df.index.nlevels))), [self.set_index(cols)._expr], requires_partition_by=partitionings.Index(), preserves_partition_by=partitionings.Singleton())) def __getattr__(self, name): # Column attribute access. if name in self._expr.proxy().columns: return self[name] else: return object.__getattribute__(self, name) def __getitem__(self, key): if key in self._expr.proxy().columns: return self._elementwise(lambda df: df[key], 'get_column') else: raise NotImplementedError(key) def __setitem__(self, key, value): if isinstance(key, str): # yapf: disable return self._elementwise( lambda df, key, value: df.__setitem__(key, value), 'set_column', (key, value), inplace=True) else: raise NotImplementedError(key) def set_index(self, keys, **kwargs): if isinstance(keys, str): keys = [keys] if not set(keys).issubset(self._expr.proxy().columns): raise NotImplementedError(keys) return self._elementwise( lambda df: df.set_index(keys, **kwargs), 'set_index', inplace=kwargs.get('inplace', False)) def at(self, *args, **kwargs): raise NotImplementedError() @property def loc(self): return _DeferredLoc(self) def aggregate(self, func, axis=0, *args, **kwargs): if axis is None: # Aggregate across all elements by first aggregating across columns, # then across rows. return self.agg(func, *args, **dict(kwargs, axis=1)).agg( func, *args, **dict(kwargs, axis=0)) elif axis in (1, 'columns'): # This is an easy elementwise aggregation. return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'aggregate', lambda df: df.agg(func, axis=1, *args, **kwargs), [self._expr], requires_partition_by=partitionings.Nothing())) elif len(self._expr.proxy().columns) == 0 or args or kwargs: # For these corner cases, just colocate everything. return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'aggregate', lambda df: df.agg(func, *args, **kwargs), [self._expr], requires_partition_by=partitionings.Singleton())) else: # In the general case, compute the aggregation of each column separately, # then recombine. if not isinstance(func, dict): col_names = list(self._expr.proxy().columns) func = {col: func for col in col_names} else: col_names = list(func.keys()) aggregated_cols = [] for col in col_names: funcs = func[col] if not isinstance(funcs, list): funcs = [funcs] aggregated_cols.append(self[col].agg(funcs, *args, **kwargs)) # The final shape is different depending on whether any of the columns # were aggregated by a list of aggregators. with expressions.allow_non_parallel_operations(): if any(isinstance(funcs, list) for funcs in func.values()): return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'join_aggregate', lambda *cols: pd.DataFrame( {col: value for col, value in zip(col_names, cols)}), [col._expr for col in aggregated_cols], requires_partition_by=partitionings.Singleton())) else: return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'join_aggregate', lambda *cols: pd.Series( {col: value[0] for col, value in zip(col_names, cols)}), [col._expr for col in aggregated_cols], requires_partition_by=partitionings.Singleton(), proxy=self._expr.proxy().agg(func, *args, **kwargs))) agg = aggregate applymap = frame_base._elementwise_method('applymap') memory_usage = frame_base.wont_implement_method('non-deferred value') all = frame_base._agg_method('all') any = frame_base._agg_method('any') cummax = cummin = cumsum = cumprod = frame_base.wont_implement_method( 'order-sensitive') diff = frame_base.wont_implement_method('order-sensitive') max = frame_base._agg_method('max') min = frame_base._agg_method('min') def mode(self, axis=0, *args, **kwargs): if axis == 1 or axis == 'columns': raise frame_base.WontImplementError('non-deferred column values') return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'mode', lambda df: df.mode(*args, **kwargs), [self._expr], #TODO(robertwb): Approximate? requires_partition_by=partitionings.Singleton(), preserves_partition_by=partitionings.Singleton())) @frame_base.args_to_kwargs(pd.DataFrame) @frame_base.populate_defaults(pd.DataFrame) @frame_base.maybe_inplace def dropna(self, axis, **kwargs): # TODO(robertwb): This is a common pattern. Generalize? if axis == 1 or axis == 'columns': requires_partition_by = partitionings.Singleton() else: requires_partition_by = partitionings.Nothing() return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'dropna', lambda df: df.dropna(axis=axis, **kwargs), [self._expr], preserves_partition_by=partitionings.Singleton(), requires_partition_by=requires_partition_by)) items = itertuples = iterrows = iteritems = frame_base.wont_implement_method( 'non-lazy') isna = frame_base._elementwise_method('isna') notnull = notna = frame_base._elementwise_method('notna') @frame_base.args_to_kwargs(pd.DataFrame) @frame_base.populate_defaults(pd.DataFrame) @frame_base.maybe_inplace def fillna(self, value, method, axis, **kwargs): if method is not None and axis in (0, 'index'): raise frame_base.WontImplementError('order-sensitive') if isinstance(value, frame_base.DeferredBase): value_expr = value._expr else: value_expr = expressions.ConstantExpression(value) return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'fillna', lambda df, value: df.fillna( value, method=method, axis=axis, **kwargs), [self._expr, value_expr], preserves_partition_by=partitionings.Singleton(), requires_partition_by=partitionings.Nothing())) prod = product = frame_base._agg_method('prod') @frame_base.args_to_kwargs(pd.DataFrame) @frame_base.populate_defaults(pd.DataFrame) def quantile(self, axis, **kwargs): if axis == 1 or axis == 'columns': raise frame_base.WontImplementError('non-deferred column values') return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'quantile', lambda df: df.quantile(axis=axis, **kwargs), [self._expr], #TODO(robertwb): Approximate quantiles? requires_partition_by=partitionings.Singleton(), preserves_partition_by=partitionings.Singleton())) query = frame_base._elementwise_method('query') @frame_base.args_to_kwargs(pd.DataFrame) @frame_base.populate_defaults(pd.DataFrame) @frame_base.maybe_inplace def replace(self, limit, **kwargs): if limit is None: requires_partition_by = partitionings.Nothing() else: requires_partition_by = partitionings.Singleton() return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'replace', lambda df: df.replace(limit=limit, **kwargs), [self._expr], preserves_partition_by=partitionings.Singleton(), requires_partition_by=requires_partition_by)) @frame_base.args_to_kwargs(pd.DataFrame) @frame_base.populate_defaults(pd.DataFrame) @frame_base.maybe_inplace def reset_index(self, level, **kwargs): if level is not None and not isinstance(level, (tuple, list)): level = [level] if level is None or len(level) == len(self._expr.proxy().index.levels): # TODO: Could do distributed re-index with offsets. requires_partition_by = partitionings.Singleton() else: requires_partition_by = partitionings.Nothing() return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'reset_index', lambda df: df.reset_index(level=level, **kwargs), [self._expr], preserves_partition_by=partitionings.Singleton(), requires_partition_by=requires_partition_by)) round = frame_base._elementwise_method('round') select_dtypes = frame_base._elementwise_method('select_dtypes') @frame_base.args_to_kwargs(pd.DataFrame) @frame_base.populate_defaults(pd.DataFrame) def shift(self, axis, **kwargs): if axis == 1 or axis == 'columns': requires_partition_by = partitionings.Nothing() else: requires_partition_by = partitionings.Singleton() return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'shift', lambda df: df.shift(axis=axis, **kwargs), [self._expr], preserves_partition_by=partitionings.Singleton(), requires_partition_by=requires_partition_by)) @property def shape(self): raise frame_base.WontImplementError('scalar value') @frame_base.args_to_kwargs(pd.DataFrame) @frame_base.populate_defaults(pd.DataFrame) @frame_base.maybe_inplace def sort_values(self, axis, **kwargs): if axis == 1 or axis == 'columns': requires_partition_by = partitionings.Nothing() else: requires_partition_by = partitionings.Singleton() return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'sort_values', lambda df: df.sort_values(axis=axis, **kwargs), [self._expr], preserves_partition_by=partitionings.Singleton(), requires_partition_by=requires_partition_by)) stack = frame_base._elementwise_method('stack') sum = frame_base._agg_method('sum') take = frame_base.wont_implement_method('deprecated') to_records = to_dict = to_numpy = to_string = ( frame_base.wont_implement_method('non-deferred value')) to_sparse = to_string # frame_base._elementwise_method('to_sparse') transform = frame_base._elementwise_method( 'transform', restrictions={'axis': 0}) def transpose(self, *args, **kwargs): raise frame_base.WontImplementError('non-deferred column values') def unstack(self, *args, **kwargs): if self._expr.proxy().index.nlevels == 1: return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'unstack', lambda df: df.unstack(*args, **kwargs), [self._expr], requires_partition_by=partitionings.Index())) else: raise frame_base.WontImplementError('non-deferred column values') update = frame_base._proxy_method( 'update', inplace=True, requires_partition_by=partitionings.Index(), preserves_partition_by=partitionings.Index())
class DeferredDataFrame(frame_base.DeferredFrame): @property def T(self): return self.transpose() @property def columns(self): return self._expr.proxy().columns def groupby(self, by): # TODO: what happens to the existing index? # We set the columns to index as we have a notion of being partitioned by # index, but not partitioned by an arbitrary subset of columns. return DeferredGroupBy( expressions.ComputedExpression( 'groupbyindex', lambda df: df.groupby(level=list(range(df.index.nlevels))), [self.set_index(by)._expr], requires_partition_by=partitionings.Index(), preserves_partition_by=partitionings.Singleton())) def __getattr__(self, name): # Column attribute access. if name in self._expr.proxy().columns: return self[name] else: return object.__getattribute__(self, name) def __getitem__(self, key): # TODO: Replicate pd.DataFrame.__getitem__ logic if isinstance(key, frame_base.DeferredBase): # Fail early if key is a DeferredBase as it interacts surprisingly with # key in self._expr.proxy().columns raise NotImplementedError( "Indexing with a deferred frame is not yet supported. Consider " "using df.loc[...]") if (isinstance(key, list) and all(key_column in self._expr.proxy().columns for key_column in key)) or key in self._expr.proxy().columns: return self._elementwise(lambda df: df[key], 'get_column') else: raise NotImplementedError(key) def __contains__(self, key): # Checks if proxy has the given column return self._expr.proxy().__contains__(key) def __setitem__(self, key, value): if isinstance(key, str): # yapf: disable return self._elementwise( lambda df, key, value: df.__setitem__(key, value), 'set_column', (key, value), inplace=True) else: raise NotImplementedError(key) @frame_base.args_to_kwargs(pd.DataFrame) @frame_base.populate_defaults(pd.DataFrame) @frame_base.maybe_inplace def set_index(self, keys, **kwargs): if isinstance(keys, str): keys = [keys] if not set(keys).issubset(self._expr.proxy().columns): raise NotImplementedError(keys) return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'set_index', lambda df: df.set_index(keys, **kwargs), [self._expr], requires_partition_by=partitionings.Nothing(), preserves_partition_by=partitionings.Nothing())) at = frame_base.not_implemented_method('at') @property def loc(self): return _DeferredLoc(self) _get_index = _set_index = frame_base.not_implemented_method('index') index = property(_get_index, _set_index) @property def axes(self): return (self.index, self.columns) apply = frame_base.not_implemented_method('apply') explode = frame_base.not_implemented_method('explode') isin = frame_base.not_implemented_method('isin') assign = frame_base.not_implemented_method('assign') append = frame_base.not_implemented_method('append') combine = frame_base.not_implemented_method('combine') combine_first = frame_base.not_implemented_method('combine_first') cov = frame_base.not_implemented_method('cov') corr = frame_base.not_implemented_method('corr') count = frame_base.not_implemented_method('count') drop = frame_base.not_implemented_method('drop') eval = frame_base.not_implemented_method('eval') reindex = frame_base.not_implemented_method('reindex') melt = frame_base.not_implemented_method('melt') pivot = frame_base.not_implemented_method('pivot') pivot_table = frame_base.not_implemented_method('pivot_table') def aggregate(self, func, axis=0, *args, **kwargs): if axis is None: # Aggregate across all elements by first aggregating across columns, # then across rows. return self.agg(func, *args, **dict(kwargs, axis=1)).agg( func, *args, **dict(kwargs, axis=0)) elif axis in (1, 'columns'): # This is an easy elementwise aggregation. return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'aggregate', lambda df: df.agg(func, axis=1, *args, **kwargs), [self._expr], requires_partition_by=partitionings.Nothing())) elif len(self._expr.proxy().columns) == 0 or args or kwargs: # For these corner cases, just colocate everything. return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'aggregate', lambda df: df.agg(func, *args, **kwargs), [self._expr], requires_partition_by=partitionings.Singleton())) else: # In the general case, compute the aggregation of each column separately, # then recombine. if not isinstance(func, dict): col_names = list(self._expr.proxy().columns) func = {col: func for col in col_names} else: col_names = list(func.keys()) aggregated_cols = [] for col in col_names: funcs = func[col] if not isinstance(funcs, list): funcs = [funcs] aggregated_cols.append(self[col].agg(funcs, *args, **kwargs)) # The final shape is different depending on whether any of the columns # were aggregated by a list of aggregators. with expressions.allow_non_parallel_operations(): if any(isinstance(funcs, list) for funcs in func.values()): return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'join_aggregate', lambda *cols: pd.DataFrame( {col: value for col, value in zip(col_names, cols)}), [col._expr for col in aggregated_cols], requires_partition_by=partitionings.Singleton())) else: return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'join_aggregate', lambda *cols: pd.Series( {col: value[0] for col, value in zip(col_names, cols)}), [col._expr for col in aggregated_cols], requires_partition_by=partitionings.Singleton(), proxy=self._expr.proxy().agg(func, *args, **kwargs))) agg = aggregate applymap = frame_base._elementwise_method('applymap') memory_usage = frame_base.wont_implement_method('non-deferred value') info = frame_base.wont_implement_method('non-deferred value') all = frame_base._agg_method('all') any = frame_base._agg_method('any') cummax = cummin = cumsum = cumprod = frame_base.wont_implement_method( 'order-sensitive') diff = frame_base.wont_implement_method('order-sensitive') def dot(self, other): # We want to broadcast the right hand side to all partitions of the left. # This is OK, as its index must be the same size as the columns set of self, # so cannot be too large. class AsScalar(object): def __init__(self, value): self.value = value if isinstance(other, frame_base.DeferredFrame): proxy = other._expr.proxy() with expressions.allow_non_parallel_operations(): side = expressions.ComputedExpression( 'as_scalar', lambda df: AsScalar(df), [other._expr], requires_partition_by=partitionings.Singleton()) else: proxy = pd.DataFrame(columns=range(len(other[0]))) side = expressions.ConstantExpression(AsScalar(other)) return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'dot', lambda left, right: left @ right.value, [self._expr, side], requires_partition_by=partitionings.Nothing(), preserves_partition_by=partitionings.Index(), proxy=proxy)) __matmul__ = dot head = tail = frame_base.wont_implement_method('order-sensitive') max = frame_base._agg_method('max') min = frame_base._agg_method('min') def mode(self, axis=0, *args, **kwargs): if axis == 1 or axis == 'columns': # Number of columns is max(number mode values for each row), so we can't # determine how many there will be before looking at the data. raise frame_base.WontImplementError('non-deferred column values') return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'mode', lambda df: df.mode(*args, **kwargs), [self._expr], #TODO(robertwb): Approximate? requires_partition_by=partitionings.Singleton(), preserves_partition_by=partitionings.Singleton())) @frame_base.args_to_kwargs(pd.DataFrame) @frame_base.populate_defaults(pd.DataFrame) @frame_base.maybe_inplace def dropna(self, axis, **kwargs): # TODO(robertwb): This is a common pattern. Generalize? if axis == 1 or axis == 'columns': requires_partition_by = partitionings.Singleton() else: requires_partition_by = partitionings.Nothing() return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'dropna', lambda df: df.dropna(axis=axis, **kwargs), [self._expr], preserves_partition_by=partitionings.Singleton(), requires_partition_by=requires_partition_by)) @frame_base.args_to_kwargs(pd.DataFrame) @frame_base.populate_defaults(pd.DataFrame) @frame_base.maybe_inplace def fillna(self, value, method, axis, **kwargs): if method is not None and axis in (0, 'index'): raise frame_base.WontImplementError('order-sensitive') if isinstance(value, frame_base.DeferredBase): value_expr = value._expr else: value_expr = expressions.ConstantExpression(value) return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'fillna', lambda df, value: df.fillna( value, method=method, axis=axis, **kwargs), [self._expr, value_expr], preserves_partition_by=partitionings.Singleton(), requires_partition_by=partitionings.Nothing())) isna = frame_base._elementwise_method('isna') notnull = notna = frame_base._elementwise_method('notna') items = itertuples = iterrows = iteritems = frame_base.wont_implement_method( 'non-lazy') def _cols_as_temporary_index(self, cols, suffix=''): original_index_names = list(self._expr.proxy().index.names) new_index_names = [ '__apache_beam_temp_%d_%s' % (ix, suffix) for (ix, _) in enumerate(original_index_names)] def reindex(df): return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'reindex', lambda df: df.rename_axis(index=new_index_names, copy=False) .reset_index().set_index(cols), [df._expr], preserves_partition_by=partitionings.Nothing(), requires_partition_by=partitionings.Nothing())) def revert(df): return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'join_restoreindex', lambda df: df.reset_index().set_index(new_index_names) .rename_axis(index=original_index_names, copy=False), [df._expr], preserves_partition_by=partitionings.Nothing(), requires_partition_by=partitionings.Nothing())) return reindex, revert @frame_base.args_to_kwargs(pd.DataFrame) @frame_base.populate_defaults(pd.DataFrame) def join(self, other, on, **kwargs): if on is not None: reindex, revert = self._cols_as_temporary_index(on) return revert(reindex(self).join(other, **kwargs)) if isinstance(other, list): other_is_list = True else: other = [other] other_is_list = False placeholder = object() other_exprs = [ df._expr for df in other if isinstance(df, frame_base.DeferredFrame)] const_others = [ placeholder if isinstance(df, frame_base.DeferredFrame) else df for df in other] def fill_placeholders(values): values = iter(values) filled = [ next(values) if df is placeholder else df for df in const_others] if other_is_list: return filled else: return filled[0] return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'join', lambda df, *deferred_others: df.join( fill_placeholders(deferred_others), **kwargs), [self._expr] + other_exprs, preserves_partition_by=partitionings.Singleton(), requires_partition_by=partitionings.Index())) @frame_base.args_to_kwargs(pd.DataFrame) @frame_base.populate_defaults(pd.DataFrame) def merge( self, right, on, left_on, right_on, left_index, right_index, **kwargs): self_proxy = self._expr.proxy() right_proxy = right._expr.proxy() # Validate with a pandas call. _ = self_proxy.merge( right_proxy, on=on, left_on=left_on, right_on=right_on, left_index=left_index, right_index=right_index, **kwargs) if not any([on, left_on, right_on, left_index, right_index]): on = [col for col in self_proxy.columns() if col in right_proxy.columns()] if not left_on: left_on = on elif not isinstance(left_on, list): left_on = [left_on] if not right_on: right_on = on elif not isinstance(right_on, list): right_on = [right_on] if left_index: indexed_left = self else: indexed_left = self.set_index(left_on, drop=False) if right_index: indexed_right = right else: indexed_right = right.set_index(right_on, drop=False) merged = frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'merge', lambda left, right: left.merge( right, left_index=True, right_index=True, **kwargs), [indexed_left._expr, indexed_right._expr], preserves_partition_by=partitionings.Singleton(), requires_partition_by=partitionings.Index())) if left_index or right_index: return merged else: return merged.reset_index(drop=True) @frame_base.args_to_kwargs(pd.DataFrame) @frame_base.populate_defaults(pd.DataFrame) def nlargest(self, keep, **kwargs): if keep == 'any': keep = 'first' elif keep != 'all': raise frame_base.WontImplementError('order-sensitive') kwargs['keep'] = keep per_partition = expressions.ComputedExpression( 'nlargest-per-partition', lambda df: df.nlargest(**kwargs), [self._expr], preserves_partition_by=partitionings.Singleton(), requires_partition_by=partitionings.Nothing()) with expressions.allow_non_parallel_operations(True): return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'nlargest', lambda df: df.nlargest(**kwargs), [per_partition], preserves_partition_by=partitionings.Singleton(), requires_partition_by=partitionings.Singleton())) @frame_base.args_to_kwargs(pd.DataFrame) @frame_base.populate_defaults(pd.DataFrame) def nsmallest(self, keep, **kwargs): if keep == 'any': keep = 'first' elif keep != 'all': raise frame_base.WontImplementError('order-sensitive') kwargs['keep'] = keep per_partition = expressions.ComputedExpression( 'nsmallest-per-partition', lambda df: df.nsmallest(**kwargs), [self._expr], preserves_partition_by=partitionings.Singleton(), requires_partition_by=partitionings.Nothing()) with expressions.allow_non_parallel_operations(True): return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'nsmallest', lambda df: df.nsmallest(**kwargs), [per_partition], preserves_partition_by=partitionings.Singleton(), requires_partition_by=partitionings.Singleton())) @frame_base.args_to_kwargs(pd.DataFrame) def nunique(self, **kwargs): if kwargs.get('axis', None) in (1, 'columns'): requires_partition_by = partitionings.Nothing() else: requires_partition_by = partitionings.Singleton() return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'nunique', lambda df: df.nunique(**kwargs), [self._expr], preserves_partition_by=partitionings.Singleton(), requires_partition_by=requires_partition_by)) prod = product = frame_base._agg_method('prod') @frame_base.args_to_kwargs(pd.DataFrame) @frame_base.populate_defaults(pd.DataFrame) def quantile(self, axis, **kwargs): if axis == 1 or axis == 'columns': raise frame_base.WontImplementError('non-deferred column values') return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'quantile', lambda df: df.quantile(axis=axis, **kwargs), [self._expr], #TODO(robertwb): Approximate quantiles? requires_partition_by=partitionings.Singleton(), preserves_partition_by=partitionings.Singleton())) query = frame_base._elementwise_method('query') @frame_base.args_to_kwargs(pd.DataFrame) @frame_base.maybe_inplace def rename(self, **kwargs): rename_index = ( 'index' in kwargs or kwargs.get('axis', None) in (0, 'index') or ('columns' not in kwargs and 'axis' not in kwargs)) if rename_index: # Technically, it's still partitioned by index, but it's no longer # partitioned by the hash of the index. preserves_partition_by = partitionings.Nothing() else: preserves_partition_by = partitionings.Singleton() if kwargs.get('errors', None) == 'raise' and rename_index: # Renaming index with checking. requires_partition_by = partitionings.Singleton() proxy = self._expr.proxy() else: requires_partition_by = partitionings.Nothing() proxy = None return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'rename', lambda df: df.rename(**kwargs), [self._expr], proxy=proxy, preserves_partition_by=preserves_partition_by, requires_partition_by=requires_partition_by)) rename_axis = frame_base._elementwise_method('rename_axis') @frame_base.args_to_kwargs(pd.DataFrame) @frame_base.populate_defaults(pd.DataFrame) @frame_base.maybe_inplace def replace(self, limit, **kwargs): if limit is None: requires_partition_by = partitionings.Nothing() else: requires_partition_by = partitionings.Singleton() return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'replace', lambda df: df.replace(limit=limit, **kwargs), [self._expr], preserves_partition_by=partitionings.Singleton(), requires_partition_by=requires_partition_by)) @frame_base.args_to_kwargs(pd.DataFrame) @frame_base.populate_defaults(pd.DataFrame) @frame_base.maybe_inplace def reset_index(self, level=None, **kwargs): if level is not None and not isinstance(level, (tuple, list)): level = [level] if level is None or len(level) == len(self._expr.proxy().index.levels): # TODO: Could do distributed re-index with offsets. requires_partition_by = partitionings.Singleton() else: requires_partition_by = partitionings.Nothing() return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'reset_index', lambda df: df.reset_index(level=level, **kwargs), [self._expr], preserves_partition_by=partitionings.Singleton(), requires_partition_by=requires_partition_by)) round = frame_base._elementwise_method('round') select_dtypes = frame_base._elementwise_method('select_dtypes') @frame_base.args_to_kwargs(pd.DataFrame) @frame_base.populate_defaults(pd.DataFrame) def shift(self, axis, **kwargs): if 'freq' in kwargs: raise frame_base.WontImplementError('data-dependent') if axis == 1 or axis == 'columns': requires_partition_by = partitionings.Nothing() else: requires_partition_by = partitionings.Singleton() return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'shift', lambda df: df.shift(axis=axis, **kwargs), [self._expr], preserves_partition_by=partitionings.Singleton(), requires_partition_by=requires_partition_by)) @property def shape(self): raise frame_base.WontImplementError('scalar value') @frame_base.args_to_kwargs(pd.DataFrame) @frame_base.populate_defaults(pd.DataFrame) @frame_base.maybe_inplace def sort_values(self, axis, **kwargs): if axis == 1 or axis == 'columns': requires_partition_by = partitionings.Nothing() else: requires_partition_by = partitionings.Singleton() return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'sort_values', lambda df: df.sort_values(axis=axis, **kwargs), [self._expr], preserves_partition_by=partitionings.Singleton(), requires_partition_by=requires_partition_by)) stack = frame_base._elementwise_method('stack') sum = frame_base._agg_method('sum') take = frame_base.wont_implement_method('deprecated') to_records = to_dict = to_numpy = to_string = ( frame_base.wont_implement_method('non-deferred value')) to_sparse = to_string # frame_base._elementwise_method('to_sparse') transform = frame_base._elementwise_method( 'transform', restrictions={'axis': 0}) transpose = frame_base.wont_implement_method('non-deferred column values') def unstack(self, *args, **kwargs): if self._expr.proxy().index.nlevels == 1: return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'unstack', lambda df: df.unstack(*args, **kwargs), [self._expr], requires_partition_by=partitionings.Index())) else: raise frame_base.WontImplementError('non-deferred column values') update = frame_base._proxy_method( 'update', inplace=True, requires_partition_by=partitionings.Index(), preserves_partition_by=partitionings.Index())
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) 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.Nothing())) 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 (not 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)
class DeferredDataFrame(frame_base.DeferredFrame): @property def T(self): return self.transpose() def groupby(self, cols): # TODO: what happens to the existing index? # We set the columns to index as we have a notion of being partitioned by # index, but not partitioned by an arbitrary subset of columns. return DeferredGroupBy( expressions.ComputedExpression( 'groupbyindex', lambda df: df.groupby(level=list(range(df.index.nlevels))), [self.set_index(cols)._expr], requires_partition_by=partitionings.Index(), preserves_partition_by=partitionings.Singleton())) def __getattr__(self, name): # Column attribute access. if name in self._expr.proxy().columns: return self[name] else: return object.__getattribute__(self, name) def __getitem__(self, key): if key in self._expr.proxy().columns: return self._elementwise(lambda df: df[key], 'get_column') else: raise NotImplementedError(key) def __setitem__(self, key, value): if isinstance(key, str): # yapf: disable return self._elementwise( lambda df, key, value: df.__setitem__(key, value), 'set_column', (key, value), inplace=True) else: raise NotImplementedError(key) def set_index(self, keys, **kwargs): if isinstance(keys, str): keys = [keys] if not set(keys).issubset(self._expr.proxy().columns): raise NotImplementedError(keys) return self._elementwise( lambda df: df.set_index(keys, **kwargs), 'set_index', inplace=kwargs.get('inplace', False)) def at(self, *args, **kwargs): raise NotImplementedError() @property def loc(self): return _DeferredLoc(self) def aggregate(self, *args, **kwargs): if 'axis' in kwargs and kwargs['axis'] is None: return self.agg(*args, **dict(kwargs, axis=1)).agg( *args, **dict(kwargs, axis=0)) return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'aggregate', lambda df: df.agg(*args, **kwargs), [self._expr], # TODO(robertwb): Sub-aggregate when possible. requires_partition_by=partitionings.Singleton())) agg = aggregate applymap = frame_base._elementwise_method('applymap') memory_usage = frame_base.wont_implement_method('non-deferred value') all = frame_base._associative_agg_method('all') any = frame_base._associative_agg_method('any') cummax = cummin = cumsum = cumprod = frame_base.wont_implement_method( 'order-sensitive') diff = frame_base.wont_implement_method('order-sensitive') max = frame_base._associative_agg_method('max') min = frame_base._associative_agg_method('min') mode = frame_base._agg_method('mode') def dropna( self, axis=0, how='any', thresh=None, subset=None, inplace=False, *args, **kwargs): # TODO(robertwb): This is a common pattern. Generalize? if axis == 1 or axis == 'columns': requires_partition_by = partitionings.Singleton() else: requires_partition_by = partitionings.Nothing() result = frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'dropna', lambda df: df.dropna( axis, how, thresh, subset, False, *args, **kwargs), [self._expr], preserves_partition_by=partitionings.Singleton(), requires_partition_by=requires_partition_by)) if inplace: self._expr = result._expr else: return result items = itertuples = iterrows = iteritems = frame_base.wont_implement_method( 'non-lazy') isna = frame_base._elementwise_method('isna') notnull = notna = frame_base._elementwise_method('notna') prod = product = frame_base._associative_agg_method('prod') def quantile(self, q=0.5, axis=0, *args, **kwargs): if axis != 0: raise frame_base.WontImplementError('non-deferred column values') return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'quantile', lambda df: df.quantile(q, axis, *args, **kwargs), [self._expr], #TODO(robertwb): Approximate quantiles? requires_partition_by=partitionings.Singleton(), preserves_partition_by=partitionings.Singleton())) query = frame_base._elementwise_method('query') def replace(self, to_replace=None, value=None, inplace=False, limit=None, *args, **kwargs): if limit is None: requires_partition_by = partitionings.Nothing() else: requires_partition_by = partitionings.Singleton() result = frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'replace', lambda df: df.replace( to_replace, value, False, limit, *args, **kwargs), [self._expr], preserves_partition_by=partitionings.Singleton(), requires_partition_by=requires_partition_by)) if inplace: self._expr = result._expr else: return result def reset_index(self, level=None, drop=False, inplace=False, *args, **kwargs): if level is not None and not isinstance(level, (tuple, list)): level = [level] if level is None or len(level) == len(self._expr.proxy().index.levels): # TODO: Could do distributed re-index with offsets. requires_partition_by = partitionings.Singleton() else: requires_partition_by = partitionings.Nothing() result = frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'reset_index', lambda df: df.reset_index(level, drop, False, *args, **kwargs), [self._expr], preserves_partition_by=partitionings.Singleton(), requires_partition_by=requires_partition_by)) if inplace: self._expr = result._expr else: return result round = frame_base._elementwise_method('round') select_dtypes = frame_base._elementwise_method('select_dtypes') def shift(self, periods=1, freq=None, axis=0, *args, **kwargs): if axis == 1 or axis == 'columns': requires_partition_by = partitionings.Nothing() else: requires_partition_by = partitionings.Singleton() return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'shift', lambda df: df.shift(periods, freq, axis, *args, **kwargs), [self._expr], preserves_partition_by=partitionings.Singleton(), requires_partition_by=requires_partition_by)) @property def shape(self): raise frame_base.WontImplementError('scalar value') def sort_values( self, by, axis=0, ascending=True, inplace=False, *args, **kwargs): if axis == 1 or axis == 'columns': requires_partition_by = partitionings.Nothing() else: requires_partition_by = partitionings.Singleton() result = frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'sort_values', lambda df: df.sort_values( by, axis, ascending, False, *args, **kwargs), [self._expr], preserves_partition_by=partitionings.Singleton(), requires_partition_by=requires_partition_by)) if inplace: self._expr = result._expr else: return result stack = frame_base._elementwise_method('stack') sum = frame_base._associative_agg_method('sum') to_records = to_dict = to_numpy = to_string = ( frame_base.wont_implement_method('non-deferred value')) to_sparse = to_string # frame_base._elementwise_method('to_sparse') transform = frame_base._elementwise_method( 'transform', restrictions={'axis': 0}) def transpose(self, *args, **kwargs): raise frame_base.WontImplementError('non-deferred column values') def unstack(self, *args, **kwargs): if self._expr.proxy().index.nlevels == 1: return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'unstack', lambda df: df.unstack(*args, **kwargs), [self._expr], requires_partition_by=partitionings.Index())) else: raise frame_base.WontImplementError('non-deferred column values') update = frame_base._proxy_method( 'update', inplace=True, requires_partition_by=partitionings.Index(), preserves_partition_by=partitionings.Index())
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]