Пример #1
0
 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))
Пример #2
0
 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()))
Пример #3
0
 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()))
Пример #4
0
  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))
Пример #5
0
 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)
Пример #6
0
 def estimate_size(expr, same_stage_ok):
   # Returns a pcollection of ints whose sum is the estimated size of the
   # given expression.
   pipeline = next(iter(inputs.values())).pipeline
   label = 'Size[%s, %s]' % (expr._id, same_stage_ok)
   if is_scalar(expr):
     return pipeline | label >> beam.Create([0])
   elif same_stage_ok:
     return expr_to_pcoll(expr) | label >> beam.Map(_total_memory_usage)
   elif expr in inputs:
     return None
   else:
     # This is the stage to avoid.
     expr_stage = expr_to_stage(expr)
     # If the stage doesn't start with a shuffle, it's not safe to fuse
     # the computation into its parent either.
     has_shuffle = expr_stage.partitioning != partitionings.Nothing()
     # We assume the size of an expression is the sum of the size of its
     # inputs, which may be off by quite a bit, but the goal is to get
     # within an order of magnitude or two.
     arg_sizes = []
     for arg in expr.args():
       if is_scalar(arg):
         continue
       elif arg in inputs:
         return None
       arg_size = estimate_size(
           arg,
           same_stage_ok=has_shuffle and expr_to_stage(arg) != expr_stage)
       if arg_size is None:
         return None
       arg_sizes.append(arg_size)
     return arg_sizes | label >> beam.Flatten(pipeline=pipeline)
Пример #7
0
 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()))
Пример #8
0
def elementwise_expression(name, func, args):
  return ComputedExpression(
      name,
      func,
      args,
      requires_partition_by=partitionings.Nothing(),
      preserves_partition_by=partitionings.Singleton())
Пример #9
0
      def expand(self, pcolls):
        if self.stage.partitioning != partitionings.Nothing():
          # Arrange such that partitioned_pcoll is properly partitioned.
          input_pcolls = {
              tag: pcoll | 'Flat%s' % tag >> beam.FlatMap(
                  self.stage.partitioning.partition_fn)
              for (tag, pcoll) in pcolls.items()
          }
          partitioned_pcoll = input_pcolls | beam.CoGroupByKey(
          ) | beam.MapTuple(
              lambda _,
              inputs: {tag: pd.concat(vs)
                       for tag, vs in inputs.items()})
        else:
          # Already partitioned, or no partitioning needed.
          (tag, pcoll), = pcolls.items()
          partitioned_pcoll = pcoll | beam.Map(lambda df: {tag: df})

        # Actually evaluate the expressions.
        def evaluate(partition, stage=self.stage):
          session = expressions.Session(
              {expr: partition[expr._id]
               for expr in stage.inputs})
          for expr in stage.outputs:
            yield beam.pvalue.TaggedOutput(expr._id, expr.evaluate_at(session))

        return partitioned_pcoll | beam.FlatMap(evaluate).with_outputs()
Пример #10
0
def _elementwise_function(func, name=None, restrictions=None, inplace=False):
    return _proxy_function(func,
                           name,
                           restrictions,
                           inplace,
                           requires_partition_by=partitionings.Nothing(),
                           preserves_partition_by=partitionings.Singleton())
Пример #11
0
 def output_is_partitioned_by(expr, stage, partitioning):
     if partitioning == partitionings.Nothing():
         # Always satisfied.
         return True
     elif stage.partitioning == partitionings.Singleton():
         # Within a stage, the singleton partitioning is trivially preserved.
         return True
     elif expr in stage.inputs:
         # Inputs are all partitioned by stage.partitioning.
         return stage.partitioning.is_subpartitioning_of(partitioning)
     elif expr.preserves_partition_by().is_subpartitioning_of(
             partitioning):
         # Here expr preserves at least the requested partitioning; its outputs
         # will also have this partitioning iff its inputs do.
         if expr.requires_partition_by().is_subpartitioning_of(
                 partitioning):
             # If expr requires at least this partitioning, we will arrange such
             # that its inputs satisfy this.
             return True
         else:
             # Otherwise, recursively check all the inputs.
             return all(
                 output_is_partitioned_by(arg, stage, partitioning)
                 for arg in expr.args())
     else:
         return False
Пример #12
0
 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
Пример #13
0
 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())
Пример #14
0
 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()
Пример #15
0
 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)))
Пример #16
0
 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))
Пример #17
0
 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))
Пример #18
0
 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))
Пример #19
0
def _proxy_method(func,
                  name=None,
                  restrictions=None,
                  inplace=False,
                  requires_partition_by=partitionings.Singleton(),
                  preserves_partition_by=partitionings.Nothing()):
    if name is None:
        name, func = name_and_func(func)
    if restrictions is None:
        restrictions = {}
    return _proxy_function(func, name, restrictions, inplace,
                           requires_partition_by, preserves_partition_by)
Пример #20
0
 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))
Пример #21
0
 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))
Пример #22
0
 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))
Пример #23
0
            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():
                    # Arrange such that partitioned_pcoll is properly partitioned.
                    main_pcolls = {
                        expr._id: pcolls[expr._id] | 'Flat%s' % expr._id >>
                        beam.FlatMap(self.stage.partitioning.partition_fn)
                        for expr in tabular_inputs
                    } | beam.CoGroupByKey()
                    partitioned_pcoll = main_pcolls | beam.MapTuple(
                        lambda _, inputs:
                        {tag: pd.concat(vs)
                         for tag, vs in inputs.items()})

                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()
Пример #24
0
 def fillna(self, value, method):
     if method is not None:
         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),
             [self._expr, value_expr],
             preserves_partition_by=partitionings.Singleton(),
             requires_partition_by=partitionings.Nothing()))
Пример #25
0
 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()))
Пример #26
0
 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))
Пример #27
0
  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()))
Пример #28
0
 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)
Пример #29
0
    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()))
Пример #30
0
 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