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 aggregate(self, func, axis=0, *args, **kwargs): if isinstance(func, list) and len(func) > 1: # Aggregate each column separately, then stick them all together. rows = [self.agg([f], *args, **kwargs) for f in func] return frame_base.DeferredFrame.wrap( expressions.ComputedExpression('join_aggregate', lambda *rows: pd.concat(rows), [row._expr for row in rows])) else: # We're only handling a single column. base_func = func[0] if isinstance(func, list) else func if _is_associative(base_func) and not args and not kwargs: intermediate = expressions.elementwise_expression( 'pre_aggregate', lambda s: s.agg([base_func], *args, **kwargs), [self._expr]) allow_nonparallel_final = True else: intermediate = self._expr allow_nonparallel_final = None # i.e. don't change the value with expressions.allow_non_parallel_operations( allow_nonparallel_final): return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'aggregate', lambda s: s.agg(func, *args, **kwargs), [intermediate], preserves_partition_by=partitionings.Singleton(), requires_partition_by=partitionings.Singleton()))
def test_only_caches_same_input(self): arg_0_expr = expressions.ConstantExpression(0) ident_val = expressions.ComputedExpression('ident', lambda x: x, [arg_0_expr]) comp_expr = expressions.ComputedExpression('add', lambda x, y: x + y, [ident_val, arg_0_expr]) self.mock_cache(arg_0_expr) replaced = self.cache.replace_with_cached(comp_expr) # Assert that arg_0_expr, being an input to two computations, was replaced # with the same placeholder expression. expected_trace = [ expressions.ComputedExpression, expressions.ComputedExpression, expressions.PlaceholderExpression, expressions.PlaceholderExpression ] actual_trace = self.create_trace(comp_expr) unique_placeholders = set( t for t in actual_trace if isinstance(t, expressions.PlaceholderExpression)) self.assertTraceTypes(comp_expr, expected_trace) self.assertTrue( all(e == replaced[arg_0_expr._id] for e in unique_placeholders)) self.assertIn(arg_0_expr._id, replaced)
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)))
def nsmallest(self, **kwargs): if 'keep' in kwargs and kwargs['keep'] != 'all': raise frame_base.WontImplementError('order-sensitive') 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()))
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
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))
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 wrapper(self, *args, **kargs): assert isinstance(self, DeferredGroupBy) ungrouped = self._expr.args()[0] pre_agg = expressions.ComputedExpression( 'pre_combine_' + name, lambda df: func(df.groupby(level=list(range(df.index.nlevels)))), [ungrouped], requires_partition_by=partitionings.Nothing(), preserves_partition_by=partitionings.Singleton()) post_agg = expressions.ComputedExpression( 'post_combine_' + name, lambda df: func(df.groupby(level=list(range(df.index.nlevels)))), [pre_agg], requires_partition_by=partitionings.Index(), preserves_partition_by=partitionings.Singleton()) return frame_base.DeferredFrame.wrap(post_agg)
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 test_only_replaces_cached(self): in_expr = expressions.ConstantExpression(0) comp_expr = expressions.ComputedExpression('test', lambda x: x, [in_expr]) # Expect that no replacement of expressions is performed. expected_trace = [ expressions.ComputedExpression, expressions.ConstantExpression ] self.assertTraceTypes(comp_expr, expected_trace) self.cache.replace_with_cached(comp_expr) self.assertTraceTypes(comp_expr, expected_trace) # Now "cache" the expression and assert that the cached expression was # replaced with a placeholder. self.mock_cache(in_expr) replaced = self.cache.replace_with_cached(comp_expr) expected_trace = [ expressions.ComputedExpression, expressions.PlaceholderExpression ] self.assertTraceTypes(comp_expr, expected_trace) self.assertIn(in_expr._id, replaced)
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 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 test_computed_expression(self): a = expressions.PlaceholderExpression(0) b = expressions.PlaceholderExpression(0) a_plus_b = expressions.ComputedExpression('add', lambda a, b: a + b, [a, b]) session = expressions.Session({a: 1, b: 2}) self.assertEqual(session.evaluate(a_plus_b), 3)
def dropna(self, **kwargs): return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'dropna', lambda df: df.dropna(**kwargs), [self._expr], preserves_partition_by=partitionings.Singleton(), requires_partition_by=partitionings.Nothing()))
def str(self): expr = expressions.ComputedExpression( 'str', lambda df: df.str, [self._expr], requires_partition_by=partitionings.Nothing(), preserves_partition_by=partitionings.Singleton()) return _DeferredStringMethods(expr)
def agg(self, *args, **kwargs): return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'agg', lambda df: df.agg(*args, **kwargs), [self._expr], preserves_partition_by=partitionings.Singleton(), requires_partition_by=partitionings.Singleton()))
def get(ix): return expressions.ComputedExpression( # yapf: disable 'get_%d' % ix, lambda t: t[ix], [expr], requires_partition_by=partitionings.Nothing(), preserves_partition_by=partitionings.Singleton())
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 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 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()))
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_index=True, preserves_partition_by_index=True) 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 aggregate(self, axis, **kwargs): if axis is None: return self.agg(axis=1, **kwargs).agg(axis=0, **kwargs) return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'aggregate', lambda df: df.agg(axis=axis, **kwargs), [self._expr], # TODO(robertwb): Sub-aggregate when possible. requires_partition_by=partitionings.Singleton()))
def apply(self, func, name=None, args=()): if name is None: name = func.__name__ with expressions.allow_non_parallel_operations( all(isinstance(arg, _DeferredScalar) for arg in args) or None): return DeferredFrame.wrap( expressions.ComputedExpression( name, func, [self._expr] + [arg._expr for arg in args], requires_partition_by=partitionings.Singleton()))
def aggregate(self, func, axis=0, *args, **kwargs): if axis != 0: raise NotImplementedError() return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'aggregate', lambda df: df.agg(func, axis, *args, **kwargs), [self._expr], # TODO(robertwb): Sub-aggregate when possible. requires_partition_by=partitionings.Singleton()))
def unique(self, as_series=False): if not as_series: raise frame_base.WontImplementError( 'pass as_series=True to get the result as a (deferred) Series') return frame_base.DeferredFrame.wrap( expressions.ComputedExpression( 'unique', lambda df: pd.Series(df.unique()), [self._expr], preserves_partition_by=partitionings.Singleton(), requires_partition_by=partitionings.Singleton()))
def wrapper(*args, **kwargs): for key, values in (): #restrictions.items(): if key in kwargs: value = kwargs[key] else: try: # pylint: disable=deprecated-method ix = inspect.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)
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()))
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()))