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_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}")
示例#3
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)

    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))
示例#4
0
def _elementwise_function(func,
                          name=None,
                          restrictions=None,
                          inplace=False,
                          base=None):
    return _proxy_function(func,
                           name,
                           restrictions,
                           inplace,
                           base,
                           requires_partition_by=partitionings.Arbitrary(),
                           preserves_partition_by=partitionings.Arbitrary())
示例#5
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.Arbitrary()
     # We assume the size of an expression is the sum of the size of its
     # inputs, which may be off by quite a bit, but the goal is to get
     # within an order of magnitude or two.
     arg_sizes = []
     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)
示例#6
0
def _elementwise_method(func, name=None, restrictions=None, inplace=False):
    return _proxy_method(func,
                         name,
                         restrictions,
                         inplace,
                         requires_partition_by=partitionings.Arbitrary(),
                         preserves_partition_by=partitionings.Singleton())
示例#7
0
 def get(ix):
     return expressions.ComputedExpression(
         # yapf: disable
         'get_%d' % ix,
         lambda t: t[ix],
         [expr],
         requires_partition_by=partitionings.Arbitrary(),
         preserves_partition_by=partitionings.Singleton())
示例#8
0
 def __init__(self, inputs, partitioning):
   self.inputs = set(inputs)
   if len(self.inputs) > 1 and partitioning == partitionings.Arbitrary():
     # We have to shuffle to co-locate, might as well partition.
     self.partitioning = partitionings.Index()
   else:
     self.partitioning = partitioning
   self.ops = []
   self.outputs = set()
示例#9
0
def output_partitioning(expr, input_partitioning):
    """ Return the expected output partitioning for `expr` when it's input is
  partitioned by `input_partitioning`.

  For internal use only; No backward compatibility guarantees """
    assert expr.requires_partition_by().is_subpartitioning_of(
        input_partitioning)

    if expr.preserves_partition_by().is_subpartitioning_of(input_partitioning):
        return min(input_partitioning, expr.preserves_partition_by())
    else:
        return partitionings.Arbitrary()
示例#10
0
def _proxy_method(func,
                  name=None,
                  restrictions=None,
                  inplace=False,
                  requires_partition_by=partitionings.Singleton(),
                  preserves_partition_by=partitionings.Arbitrary()):
    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)
示例#11
0
def _proxy_method(func,
                  name=None,
                  restrictions=None,
                  inplace=False,
                  base=None,
                  requires_partition_by=partitionings.Singleton(),
                  preserves_partition_by=partitionings.Arbitrary()):
    if name is None:
        name, func = name_and_func(func)
    if base is None:
        raise ValueError("base is required for _proxy_method")
    return _proxy_function(func, name, restrictions, inplace, base,
                           requires_partition_by, preserves_partition_by)
示例#12
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.Arbitrary():
                    # Partitioning required for these operations.
                    # Compute the number of partitions to use for the inputs based on
                    # the estimated size of the inputs.
                    if self.stage.partitioning == partitionings.Singleton():
                        # Always a single partition, don't waste time computing sizes.
                        num_partitions = 1
                    else:
                        # Estimate the sizes from the outputs of a *previous* stage such
                        # that using these estimates will not cause a fusion break.
                        input_sizes = [
                            estimate_size(input, same_stage_ok=False)
                            for input in tabular_inputs
                        ]
                        if None in input_sizes:
                            # We were unable to (cheaply) compute the size of one or more
                            # inputs.
                            num_partitions = DEFAULT_PARTITIONS
                        else:
                            num_partitions = beam.pvalue.AsSingleton(
                                input_sizes
                                | 'FlattenSizes' >> beam.Flatten()
                                | 'SumSizes' >> beam.CombineGlobally(sum)
                                | 'NumPartitions' >> beam.Map(lambda size: max(
                                    MIN_PARTITIONS,
                                    min(MAX_PARTITIONS, size //
                                        TARGET_PARTITION_SIZE))))

                    partition_fn = self.stage.partitioning.partition_fn

                    class Partition(beam.PTransform):
                        def expand(self, pcoll):
                            return (
                                pcoll
                                # Attempt to create batches of reasonable size.
                                | beam.ParDo(_PreBatch())
                                # Actually partition.
                                | beam.FlatMap(partition_fn, num_partitions)
                                # Don't bother shuffling empty partitions.
                                | beam.Filter(lambda k_df: len(k_df[1])))

                    # Arrange such that partitioned_pcoll is properly partitioned.
                    main_pcolls = {
                        expr._id: pcolls[expr._id] | 'Partition_%s_%s' %
                        (self.stage.partitioning, expr._id) >> Partition()
                        for expr in tabular_inputs
                    } | beam.CoGroupByKey()
                    partitioned_pcoll = main_pcolls | beam.ParDo(_ReBatch())

                else:
                    # Already partitioned, or no partitioning needed.
                    assert len(tabular_inputs) == 1
                    tag = tabular_inputs[0]._id
                    partitioned_pcoll = pcolls[tag] | beam.Map(
                        lambda df: {tag: df})

                side_pcolls = {
                    expr._id: beam.pvalue.AsSingleton(pcolls[expr._id])
                    for expr in scalar_inputs
                }

                # Actually evaluate the expressions.
                def evaluate(partition, stage=self.stage, **side_inputs):
                    def lookup(expr):
                        # Use proxy if there's no data in this partition
                        return expr.proxy().iloc[:0] if partition[
                            expr._id] is None else partition[expr._id]

                    session = expressions.Session(
                        dict([(expr, lookup(expr))
                              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()
示例#13
0
    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]
示例#14
0
def elementwise_expression(name, func, args):
    return ComputedExpression(name,
                              func,
                              args,
                              requires_partition_by=partitionings.Arbitrary(),
                              preserves_partition_by=partitionings.Arbitrary())
示例#15
0
 def preserves_partition_by(self):
     return partitionings.Arbitrary()
示例#16
0
 def requires_partition_by(self):
     return partitionings.Arbitrary()
示例#17
0
    def wrapper(*args, **kwargs):
        for key, values in restrictions.items():
            if key in kwargs:
                value = kwargs[key]
            else:
                try:
                    ix = getfullargspec(func).args.index(key)
                except ValueError:
                    # TODO: fix for delegation?
                    continue
                if len(args) <= ix:
                    continue
                value = args[ix]
            if callable(values):
                check = values
            elif isinstance(values, list):
                check = lambda x, values=values: x in values
            else:
                check = lambda x, value=value: x == value

            if not check(value):
                raise NotImplementedError('%s=%s not supported for %s' %
                                          (key, value, name))
        deferred_arg_indices = []
        deferred_arg_exprs = []
        constant_args = [None] * len(args)
        from apache_beam.dataframe.frames import _DeferredIndex
        for ix, arg in enumerate(args):
            if isinstance(arg, DeferredBase):
                deferred_arg_indices.append(ix)
                deferred_arg_exprs.append(arg._expr)
            elif isinstance(arg, _DeferredIndex):
                # TODO(robertwb): Consider letting indices pass through as indices.
                # This would require updating the partitioning code, as indices don't
                # have indices.
                deferred_arg_indices.append(ix)
                deferred_arg_exprs.append(
                    expressions.ComputedExpression(
                        'index_as_series',
                        lambda ix: ix.index.to_series(),  # yapf break
                        [arg._frame._expr],
                        preserves_partition_by=partitionings.Singleton(),
                        requires_partition_by=partitionings.Arbitrary()))
            elif isinstance(arg, pd.core.generic.NDFrame):
                deferred_arg_indices.append(ix)
                deferred_arg_exprs.append(
                    expressions.ConstantExpression(arg, arg[0:0]))
            else:
                constant_args[ix] = arg

        deferred_kwarg_keys = []
        deferred_kwarg_exprs = []
        constant_kwargs = {key: None for key in kwargs}
        for key, arg in kwargs.items():
            if isinstance(arg, DeferredBase):
                deferred_kwarg_keys.append(key)
                deferred_kwarg_exprs.append(arg._expr)
            elif isinstance(arg, pd.core.generic.NDFrame):
                deferred_kwarg_keys.append(key)
                deferred_kwarg_exprs.append(
                    expressions.ConstantExpression(arg, arg[0:0]))
            else:
                constant_kwargs[key] = arg

        deferred_exprs = deferred_arg_exprs + deferred_kwarg_exprs

        if inplace:
            actual_func = _copy_and_mutate(func)
        else:
            actual_func = func

        def apply(*actual_args):
            actual_args, actual_kwargs = (
                actual_args[:len(deferred_arg_exprs)],
                actual_args[len(deferred_arg_exprs):])

            full_args = list(constant_args)
            for ix, arg in zip(deferred_arg_indices, actual_args):
                full_args[ix] = arg

            full_kwargs = dict(constant_kwargs)
            for key, arg in zip(deferred_kwarg_keys, actual_kwargs):
                full_kwargs[key] = arg

            return actual_func(*full_args, **full_kwargs)

        if (requires_partition_by.is_subpartitioning_of(partitionings.Index())
                and sum(
                    isinstance(arg.proxy(), pd.core.generic.NDFrame)
                    for arg in deferred_exprs) > 1):
            # Implicit join on index if there is more than one indexed input.
            actual_requires_partition_by = partitionings.JoinIndex()
        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)