github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/dataframe/transforms.py (about)

     1  #
     2  # Licensed to the Apache Software Foundation (ASF) under one or more
     3  # contributor license agreements.  See the NOTICE file distributed with
     4  # this work for additional information regarding copyright ownership.
     5  # The ASF licenses this file to You under the Apache License, Version 2.0
     6  # (the "License"); you may not use this file except in compliance with
     7  # the License.  You may obtain a copy of the License at
     8  #
     9  #    http://www.apache.org/licenses/LICENSE-2.0
    10  #
    11  # Unless required by applicable law or agreed to in writing, software
    12  # distributed under the License is distributed on an "AS IS" BASIS,
    13  # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    14  # See the License for the specific language governing permissions and
    15  # limitations under the License.
    16  
    17  import collections
    18  import logging
    19  from typing import TYPE_CHECKING
    20  from typing import Any
    21  from typing import Dict
    22  from typing import List
    23  from typing import Mapping
    24  from typing import Tuple
    25  from typing import TypeVar
    26  from typing import Union
    27  
    28  import pandas as pd
    29  
    30  import apache_beam as beam
    31  from apache_beam import transforms
    32  from apache_beam.dataframe import expressions
    33  from apache_beam.dataframe import frames  # pylint: disable=unused-import
    34  from apache_beam.dataframe import partitionings
    35  from apache_beam.utils import windowed_value
    36  
    37  __all__ = [
    38      'DataframeTransform',
    39  ]
    40  
    41  if TYPE_CHECKING:
    42    # pylint: disable=ungrouped-imports
    43    from apache_beam.pvalue import PCollection
    44  
    45  T = TypeVar('T')
    46  
    47  TARGET_PARTITION_SIZE = 1 << 23  # 8M
    48  MIN_PARTITION_SIZE = 1 << 19  # 0.5M
    49  MAX_PARTITIONS = 1000
    50  DEFAULT_PARTITIONS = 100
    51  MIN_PARTITIONS = 10
    52  PER_COL_OVERHEAD = 1000
    53  
    54  
    55  class DataframeTransform(transforms.PTransform):
    56    """A PTransform for applying function that takes and returns dataframes
    57    to one or more PCollections.
    58  
    59    :class:`DataframeTransform` will accept a PCollection with a `schema`_ and
    60    batch it into :class:`~pandas.DataFrame` instances if necessary::
    61  
    62        (pcoll | beam.Select(key=..., foo=..., bar=...)
    63               | DataframeTransform(lambda df: df.group_by('key').sum()))
    64  
    65    It is also possible to process a PCollection of :class:`~pandas.DataFrame`
    66    instances directly, in this case a "proxy" must be provided. For example, if
    67    ``pcoll`` is a PCollection of DataFrames, one could write::
    68  
    69        pcoll | DataframeTransform(lambda df: df.group_by('key').sum(), proxy=...)
    70  
    71    To pass multiple PCollections, pass a tuple of PCollections wich will be
    72    passed to the callable as positional arguments, or a dictionary of
    73    PCollections, in which case they will be passed as keyword arguments.
    74  
    75    Args:
    76      yield_elements: (optional, default: "schemas") If set to ``"pandas"``,
    77          return PCollection(s) containing the raw Pandas objects
    78          (:class:`~pandas.DataFrame` or :class:`~pandas.Series` as appropriate).
    79          If set to ``"schemas"``, return an element-wise PCollection, where
    80          DataFrame and Series instances are expanded to one element per row.
    81          DataFrames are converted to `schema-aware`_ PCollections, where column
    82          values can be accessed by attribute.
    83      include_indexes: (optional, default: False) When
    84         ``yield_elements="schemas"``, if ``include_indexes=True``, attempt to
    85         include index columns in the output schema for expanded DataFrames.
    86         Raises an error if any of the index levels are unnamed (name=None), or if
    87         any of the names are not unique among all column and index names.
    88      proxy: (optional) An empty :class:`~pandas.DataFrame` or
    89          :class:`~pandas.Series` instance with the same ``dtype`` and ``name``
    90          as the elements of the input PCollection. Required when input
    91          PCollection :class:`~pandas.DataFrame` or :class:`~pandas.Series`
    92          elements. Ignored when input PCollection has a `schema`_.
    93  
    94    .. _schema:
    95      https://beam.apache.org/documentation/programming-guide/#what-is-a-schema
    96    .. _schema-aware:
    97      https://beam.apache.org/documentation/programming-guide/#what-is-a-schema
    98    """
    99    def __init__(
   100        self, func, proxy=None, yield_elements="schemas", include_indexes=False):
   101      self._func = func
   102      self._proxy = proxy
   103      self._yield_elements = yield_elements
   104      self._include_indexes = include_indexes
   105  
   106    def expand(self, input_pcolls):
   107      # Avoid circular import.
   108      from apache_beam.dataframe import convert
   109  
   110      # Convert inputs to a flat dict.
   111      input_dict = _flatten(input_pcolls)  # type: Dict[Any, PCollection]
   112      proxies = _flatten(self._proxy) if self._proxy is not None else {
   113          tag: None
   114          for tag in input_dict
   115      }
   116      input_frames = {
   117          k: convert.to_dataframe(pc, proxies[k])
   118          for k, pc in input_dict.items()
   119      }  # type: Dict[Any, DeferredFrame] # noqa: F821
   120  
   121      # Apply the function.
   122      frames_input = _substitute(input_pcolls, input_frames)
   123      if isinstance(frames_input, dict):
   124        result_frames = self._func(**frames_input)
   125      elif isinstance(frames_input, tuple):
   126        result_frames = self._func(*frames_input)
   127      else:
   128        result_frames = self._func(frames_input)
   129  
   130      # Compute results as a tuple.
   131      result_frames_dict = _flatten(result_frames)
   132      keys = list(result_frames_dict.keys())
   133      result_frames_tuple = tuple(result_frames_dict[key] for key in keys)
   134      result_pcolls_tuple = convert.to_pcollection(
   135          *result_frames_tuple,
   136          label='Eval',
   137          always_return_tuple=True,
   138          yield_elements=self._yield_elements,
   139          include_indexes=self._include_indexes)
   140  
   141      # Convert back to the structure returned by self._func.
   142      result_pcolls_dict = dict(zip(keys, result_pcolls_tuple))
   143      return _substitute(result_frames, result_pcolls_dict)
   144  
   145  
   146  class _DataframeExpressionsTransform(transforms.PTransform):
   147    def __init__(self, outputs):
   148      self._outputs = outputs
   149  
   150    def expand(self, inputs):
   151      return self._apply_deferred_ops(inputs, self._outputs)
   152  
   153    def _apply_deferred_ops(
   154        self,
   155        inputs,  # type: Dict[expressions.Expression, PCollection]
   156        outputs,  # type: Dict[Any, expressions.Expression]
   157        ):  # -> Dict[Any, PCollection]
   158      """Construct a Beam graph that evaluates a set of expressions on a set of
   159      input PCollections.
   160  
   161      :param inputs: A mapping of placeholder expressions to PCollections.
   162      :param outputs: A mapping of keys to expressions defined in terms of the
   163          placeholders of inputs.
   164  
   165      Returns a dictionary whose keys are those of outputs, and whose values are
   166      PCollections corresponding to the values of outputs evaluated at the
   167      values of inputs.
   168  
   169      Logically, `_apply_deferred_ops({x: a, y: b}, {f: F(x, y), g: G(x, y)})`
   170      returns `{f: F(a, b), g: G(a, b)}`.
   171      """
   172      class ComputeStage(beam.PTransform):
   173        """A helper transform that computes a single stage of operations.
   174        """
   175        def __init__(self, stage):
   176          self.stage = stage
   177  
   178        def default_label(self):
   179          return '%s:%s' % (self.stage.ops, id(self))
   180  
   181        def expand(self, pcolls):
   182          logging.info('Computing dataframe stage %s for %s', self, self.stage)
   183          scalar_inputs = [expr for expr in self.stage.inputs if is_scalar(expr)]
   184          tabular_inputs = [
   185              expr for expr in self.stage.inputs if not is_scalar(expr)
   186          ]
   187  
   188          if len(tabular_inputs) == 0:
   189            partitioned_pcoll = next(iter(
   190                pcolls.values())).pipeline | beam.Create([{}])
   191  
   192          elif self.stage.partitioning != partitionings.Arbitrary():
   193            # Partitioning required for these operations.
   194            # Compute the number of partitions to use for the inputs based on
   195            # the estimated size of the inputs.
   196            if self.stage.partitioning == partitionings.Singleton():
   197              # Always a single partition, don't waste time computing sizes.
   198              num_partitions = 1
   199            else:
   200              # Estimate the sizes from the outputs of a *previous* stage such
   201              # that using these estimates will not cause a fusion break.
   202              input_sizes = [
   203                  estimate_size(input, same_stage_ok=False)
   204                  for input in tabular_inputs
   205              ]
   206              if None in input_sizes:
   207                # We were unable to (cheaply) compute the size of one or more
   208                # inputs.
   209                num_partitions = DEFAULT_PARTITIONS
   210              else:
   211                num_partitions = beam.pvalue.AsSingleton(
   212                    input_sizes
   213                    | 'FlattenSizes' >> beam.Flatten()
   214                    | 'SumSizes' >> beam.CombineGlobally(sum)
   215                    | 'NumPartitions' >> beam.Map(
   216                        lambda size: max(
   217                            MIN_PARTITIONS,
   218                            min(MAX_PARTITIONS, size // TARGET_PARTITION_SIZE))))
   219  
   220            partition_fn = self.stage.partitioning.partition_fn
   221  
   222            class Partition(beam.PTransform):
   223              def expand(self, pcoll):
   224                return (
   225                    pcoll
   226                    # Attempt to create batches of reasonable size.
   227                    | beam.ParDo(_PreBatch())
   228                    # Actually partition.
   229                    | beam.FlatMap(partition_fn, num_partitions)
   230                    # Don't bother shuffling empty partitions.
   231                    | beam.Filter(lambda k_df: len(k_df[1])))
   232  
   233            # Arrange such that partitioned_pcoll is properly partitioned.
   234            main_pcolls = {
   235                expr._id: pcolls[expr._id] | 'Partition_%s_%s' %
   236                (self.stage.partitioning, expr._id) >> Partition()
   237                for expr in tabular_inputs
   238            } | beam.CoGroupByKey()
   239            partitioned_pcoll = main_pcolls | beam.ParDo(_ReBatch())
   240  
   241          else:
   242            # Already partitioned, or no partitioning needed.
   243            assert len(tabular_inputs) == 1
   244            tag = tabular_inputs[0]._id
   245            partitioned_pcoll = pcolls[tag] | beam.Map(lambda df: {tag: df})
   246  
   247          side_pcolls = {
   248              expr._id: beam.pvalue.AsSingleton(pcolls[expr._id])
   249              for expr in scalar_inputs
   250          }
   251  
   252          # Actually evaluate the expressions.
   253          def evaluate(partition, stage=self.stage, **side_inputs):
   254            def lookup(expr):
   255              # Use proxy if there's no data in this partition
   256              return expr.proxy(
   257              ).iloc[:0] if partition[expr._id] is None else partition[expr._id]
   258  
   259            session = expressions.Session(
   260                dict([(expr, lookup(expr)) for expr in tabular_inputs] +
   261                     [(expr, side_inputs[expr._id]) for expr in scalar_inputs]))
   262            for expr in stage.outputs:
   263              yield beam.pvalue.TaggedOutput(expr._id, expr.evaluate_at(session))
   264  
   265          return partitioned_pcoll | beam.FlatMap(evaluate, **
   266                                                  side_pcolls).with_outputs()
   267  
   268      class Stage(object):
   269        """Used to build up a set of operations that can be fused together.
   270  
   271        Note that these Dataframe "stages" contain a CoGBK and hence are often
   272        split across multiple "executable" stages.
   273        """
   274        def __init__(self, inputs, partitioning):
   275          self.inputs = set(inputs)
   276          if (len(self.inputs) > 1 and
   277              partitioning.is_subpartitioning_of(partitionings.Index())):
   278            # We have to shuffle to co-locate, might as well partition.
   279            self.partitioning = partitionings.Index()
   280          elif isinstance(partitioning, partitionings.JoinIndex):
   281            # Not an actionable partitioning, use index.
   282            self.partitioning = partitionings.Index()
   283          else:
   284            self.partitioning = partitioning
   285          self.ops = []
   286          self.outputs = set()
   287  
   288        def __repr__(self, indent=0):
   289          if indent:
   290            sep = '\n' + ' ' * indent
   291          else:
   292            sep = ''
   293          return (
   294              "Stage[%sinputs=%s, %spartitioning=%s, %sops=%s, %soutputs=%s]" % (
   295                  sep,
   296                  self.inputs,
   297                  sep,
   298                  self.partitioning,
   299                  sep,
   300                  self.ops,
   301                  sep,
   302                  self.outputs))
   303  
   304      # First define some helper functions.
   305      def output_partitioning_in_stage(expr, stage):
   306        """Return the output partitioning of expr when computed in stage,
   307        or returns None if the expression cannot be computed in this stage.
   308        """
   309        def maybe_upgrade_to_join_index(partitioning):
   310          if partitioning.is_subpartitioning_of(partitionings.JoinIndex()):
   311            return partitionings.JoinIndex(expr)
   312          else:
   313            return partitioning
   314  
   315        if expr in stage.inputs or expr in inputs:
   316          # Inputs are all partitioned by stage.partitioning.
   317          return maybe_upgrade_to_join_index(stage.partitioning)
   318  
   319        # Anything that's not an input must have arguments
   320        assert len(expr.args())
   321  
   322        arg_partitionings = set(
   323            output_partitioning_in_stage(arg, stage) for arg in expr.args()
   324            if not is_scalar(arg))
   325  
   326        if len(arg_partitionings) == 0:
   327          # All inputs are scalars, output partitioning isn't dependent on the
   328          # input.
   329          return maybe_upgrade_to_join_index(expr.preserves_partition_by())
   330  
   331        if len(arg_partitionings) > 1:
   332          # Arguments must be identically partitioned, can't compute this
   333          # expression here.
   334          return None
   335  
   336        arg_partitioning = arg_partitionings.pop()
   337  
   338        if not expr.requires_partition_by().is_subpartitioning_of(
   339            arg_partitioning):
   340          # Arguments aren't partitioned sufficiently for this expression
   341          return None
   342  
   343        return maybe_upgrade_to_join_index(
   344            expressions.output_partitioning(expr, arg_partitioning))
   345  
   346      def is_computable_in_stage(expr, stage):
   347        return output_partitioning_in_stage(expr, stage) is not None
   348  
   349      def common_stages(stage_lists):
   350        # Set intersection, with a preference for earlier items in the list.
   351        if stage_lists:
   352          for stage in stage_lists[0]:
   353            if all(stage in other for other in stage_lists[1:]):
   354              yield stage
   355  
   356      @_memoize
   357      def is_scalar(expr):
   358        return not isinstance(expr.proxy(), pd.core.generic.NDFrame)
   359  
   360      @_memoize
   361      def expr_to_stages(expr):
   362        if expr in inputs:
   363          # Don't create a stage for each input, but it is still useful to record
   364          # what which stages inputs are available from.
   365          return []
   366  
   367        # First attempt to compute this expression as part of an existing stage,
   368        # if possible.
   369        if all(arg in inputs for arg in expr.args()):
   370          # All input arguments;  try to pick a stage that already has as many
   371          # of the inputs, correctly partitioned, as possible.
   372          inputs_by_stage = collections.defaultdict(int)
   373          for arg in expr.args():
   374            for stage in expr_to_stages(arg):
   375              if is_computable_in_stage(expr, stage):
   376                inputs_by_stage[stage] += 1 + 100 * (
   377                    expr.requires_partition_by() == stage.partitioning)
   378          if inputs_by_stage:
   379            # Take the stage with the largest count.
   380            stage = max(inputs_by_stage.items(), key=lambda kv: kv[1])[0]
   381          else:
   382            stage = None
   383        else:
   384          # Try to pick a stage that has all the available non-input expressions.
   385          # TODO(robertwb): Baring any that have all of them, we could try and
   386          # pick one that has the most, but we need to ensure it is not a
   387          # predecessor of any of the missing argument's stages.
   388          for stage in common_stages([expr_to_stages(arg) for arg in expr.args()
   389                                      if arg not in inputs]):
   390            if is_computable_in_stage(expr, stage):
   391              break
   392          else:
   393            stage = None
   394  
   395        if stage is None:
   396          # No stage available, compute this expression as part of a new stage.
   397          stage = Stage(expr.args(), expr.requires_partition_by())
   398          for arg in expr.args():
   399            # For each argument, declare that it is also available in
   400            # this new stage.
   401            expr_to_stages(arg).append(stage)
   402            # It also must be declared as an output of the producing stage.
   403            expr_to_stage(arg).outputs.add(arg)
   404        stage.ops.append(expr)
   405        # Ensure that any inputs for the overall transform are added
   406        # in downstream stages.
   407        for arg in expr.args():
   408          if arg in inputs:
   409            stage.inputs.add(arg)
   410        # This is a list as given expression may be available in many stages.
   411        return [stage]
   412  
   413      def expr_to_stage(expr):
   414        # Any will do; the first requires the fewest intermediate stages.
   415        return expr_to_stages(expr)[0]
   416  
   417      # Ensure each output is computed.
   418      for expr in outputs.values():
   419        if expr not in inputs:
   420          expr_to_stage(expr).outputs.add(expr)
   421  
   422      @_memoize
   423      def stage_to_result(stage):
   424        return {expr._id: expr_to_pcoll(expr)
   425                for expr in stage.inputs} | ComputeStage(stage)
   426  
   427      @_memoize
   428      def expr_to_pcoll(expr):
   429        if expr in inputs:
   430          return inputs[expr]
   431        else:
   432          return stage_to_result(expr_to_stage(expr))[expr._id]
   433  
   434      @_memoize
   435      def estimate_size(expr, same_stage_ok):
   436        # Returns a pcollection of ints whose sum is the estimated size of the
   437        # given expression.
   438        pipeline = next(iter(inputs.values())).pipeline
   439        label = 'Size[%s, %s]' % (expr._id, same_stage_ok)
   440        if is_scalar(expr):
   441          return pipeline | label >> beam.Create([0])
   442        elif same_stage_ok:
   443          return expr_to_pcoll(expr) | label >> beam.Map(_total_memory_usage)
   444        elif expr in inputs:
   445          return None
   446        else:
   447          # This is the stage to avoid.
   448          expr_stage = expr_to_stage(expr)
   449          # If the stage doesn't start with a shuffle, it's not safe to fuse
   450          # the computation into its parent either.
   451          has_shuffle = expr_stage.partitioning != partitionings.Arbitrary()
   452          # We assume the size of an expression is the sum of the size of its
   453          # inputs, which may be off by quite a bit, but the goal is to get
   454          # within an order of magnitude or two.
   455          arg_sizes = []
   456          for arg in expr.args():
   457            if is_scalar(arg):
   458              continue
   459            elif arg in inputs:
   460              return None
   461            arg_size = estimate_size(
   462                arg,
   463                same_stage_ok=has_shuffle and expr_to_stage(arg) != expr_stage)
   464            if arg_size is None:
   465              return None
   466            arg_sizes.append(arg_size)
   467          return arg_sizes | label >> beam.Flatten(pipeline=pipeline)
   468  
   469      # Now we can compute and return the result.
   470      return {k: expr_to_pcoll(expr) for k, expr in outputs.items()}
   471  
   472  
   473  def _total_memory_usage(frame):
   474    assert isinstance(frame, (pd.core.generic.NDFrame, pd.Index))
   475    try:
   476      size = frame.memory_usage()
   477      if not isinstance(size, int):
   478        size = size.sum() + PER_COL_OVERHEAD * len(size)
   479      else:
   480        size += PER_COL_OVERHEAD
   481      return size
   482    except AttributeError:
   483      # Don't know, assume it's really big.
   484      float('inf')
   485  
   486  
   487  class _PreBatch(beam.DoFn):
   488    def __init__(
   489        self, target_size=TARGET_PARTITION_SIZE, min_size=MIN_PARTITION_SIZE):
   490      self._target_size = target_size
   491      self._min_size = min_size
   492  
   493    def start_bundle(self):
   494      self._parts = collections.defaultdict(list)
   495      self._running_size = 0
   496  
   497    def process(
   498        self,
   499        part,
   500        window=beam.DoFn.WindowParam,
   501        timestamp=beam.DoFn.TimestampParam):
   502      part_size = _total_memory_usage(part)
   503      if part_size >= self._min_size:
   504        yield part
   505      else:
   506        self._running_size += part_size
   507        self._parts[window, timestamp].append(part)
   508        if self._running_size >= self._target_size:
   509          yield from self.finish_bundle()
   510  
   511    def finish_bundle(self):
   512      for (window, timestamp), parts in self._parts.items():
   513        yield windowed_value.WindowedValue(_concat(parts), timestamp, (window, ))
   514      self.start_bundle()
   515  
   516  
   517  class _ReBatch(beam.DoFn):
   518    """Groups all the parts from various workers into the same dataframe.
   519  
   520    Also groups across partitions, up to a given data size, to recover some
   521    efficiency in the face of over-partitioning.
   522    """
   523    def __init__(
   524        self, target_size=TARGET_PARTITION_SIZE, min_size=MIN_PARTITION_SIZE):
   525      self._target_size = target_size
   526      self._min_size = min_size
   527  
   528    def start_bundle(self):
   529      self._parts = collections.defaultdict(lambda: collections.defaultdict(list))
   530      self._running_size = 0
   531  
   532    def process(
   533        self,
   534        element,
   535        window=beam.DoFn.WindowParam,
   536        timestamp=beam.DoFn.TimestampParam):
   537      _, tagged_parts = element
   538      for tag, parts in tagged_parts.items():
   539        for part in parts:
   540          self._running_size += _total_memory_usage(part)
   541        self._parts[window, timestamp][tag].extend(parts)
   542      if self._running_size >= self._target_size:
   543        yield from self.finish_bundle()
   544  
   545    def finish_bundle(self):
   546      for (window, timestamp), tagged_parts in self._parts.items():
   547        yield windowed_value.WindowedValue(  # yapf break
   548        {
   549            tag: _concat(parts) if parts else None
   550            for (tag, parts) in tagged_parts.items()
   551        },
   552        timestamp, (window, ))
   553      self.start_bundle()
   554  
   555  
   556  def _memoize(f):
   557    cache = {}
   558  
   559    def wrapper(*args, **kwargs):
   560      key = args, tuple(sorted(kwargs.items()))
   561      if key not in cache:
   562        cache[key] = f(*args, **kwargs)
   563      return cache[key]
   564  
   565    return wrapper
   566  
   567  
   568  def _dict_union(dicts):
   569    result = {}
   570    for d in dicts:
   571      result.update(d)
   572    return result
   573  
   574  
   575  def _concat(parts):
   576    if len(parts) == 1:
   577      return parts[0]
   578    else:
   579      return pd.concat(parts)
   580  
   581  
   582  def _flatten(
   583      valueish,  # type: Union[T, List[T], Tuple[T], Dict[Any, T]]
   584      root=(),  # type: Tuple[Any, ...]
   585      ):
   586    # type: (...) -> Mapping[Tuple[Any, ...], T]
   587  
   588    """Given a nested structure of dicts, tuples, and lists, return a flat
   589    dictionary where the values are the leafs and the keys are the "paths" to
   590    these leaves.
   591  
   592    For example `{a: x, b: (y, z)}` becomes `{(a,): x, (b, 0): y, (b, 1): c}`.
   593    """
   594    if isinstance(valueish, dict):
   595      return _dict_union(_flatten(v, root + (k, )) for k, v in valueish.items())
   596    elif isinstance(valueish, (tuple, list)):
   597      return _dict_union(
   598          _flatten(v, root + (ix, )) for ix, v in enumerate(valueish))
   599    else:
   600      return {root: valueish}
   601  
   602  
   603  def _substitute(valueish, replacements, root=()):
   604    """Substitutes the values in valueish with those in replacements where the
   605    keys are as in _flatten.
   606  
   607    For example,
   608  
   609    ```
   610    _substitute(
   611        {a: x, b: (y, z)},
   612        {(a,): X, (b, 0): Y, (b, 1): Z})
   613    ```
   614  
   615    returns `{a: X, b: (Y, Z)}`.
   616    """
   617    if isinstance(valueish, dict):
   618      return type(valueish)({
   619          k: _substitute(v, replacements, root + (k, ))
   620          for (k, v) in valueish.items()
   621      })
   622    elif isinstance(valueish, (tuple, list)):
   623      return type(valueish)((
   624          _substitute(v, replacements, root + (ix, ))
   625          for (ix, v) in enumerate(valueish)))
   626    else:
   627      return replacements[root]