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

     1  # Licensed to the Apache Software Foundation (ASF) under one or more
     2  # contributor license agreements.  See the NOTICE file distributed with
     3  # this work for additional information regarding copyright ownership.
     4  # The ASF licenses this file to You under the Apache License, Version 2.0
     5  # (the "License"); you may not use this file except in compliance with
     6  # the License.  You may obtain a copy of the License at
     7  #
     8  #    http://www.apache.org/licenses/LICENSE-2.0
     9  #
    10  # Unless required by applicable law or agreed to in writing, software
    11  # distributed under the License is distributed on an "AS IS" BASIS,
    12  # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    13  # See the License for the specific language governing permissions and
    14  # limitations under the License.
    15  #
    16  
    17  """A microbenchmark for measuring performance of coders.
    18  
    19  This runs a sequence of encode-decode operations on random inputs
    20  to collect performance of various coders.
    21  
    22  To evaluate coders performance we approximate the behavior
    23  how the coders are used in PCollections: we encode and decode
    24  a list of elements. An element can be a string, a list of integers,
    25  a windowed value, or any other object we want a coder to process.
    26  
    27  Run as:
    28    python -m apache_beam.tools.coders_microbenchmark
    29  
    30  """
    31  
    32  # pytype: skip-file
    33  
    34  import argparse
    35  import logging
    36  import random
    37  import re
    38  import string
    39  import sys
    40  
    41  import apache_beam as beam
    42  from apache_beam.coders import proto2_coder_test_messages_pb2 as test_message
    43  from apache_beam.coders import coder_impl
    44  from apache_beam.coders import coders
    45  from apache_beam.coders import row_coder
    46  from apache_beam.coders import typecoders
    47  from apache_beam.tools import utils
    48  from apache_beam.transforms import window
    49  from apache_beam.typehints import trivial_inference
    50  from apache_beam.typehints.pandas_type_compatibility import DataFrameBatchConverterDropIndex
    51  from apache_beam.utils import windowed_value
    52  
    53  
    54  def coder_benchmark_factory(coder, generate_fn):
    55    """Creates a benchmark that encodes and decodes a list of elements.
    56  
    57    Args:
    58      coder: coder to use to encode an element.
    59      generate_fn: a callable that generates an element.
    60    """
    61    class CoderBenchmark(object):
    62      def __init__(self, num_elements_per_benchmark):
    63        self._coder = coders.IterableCoder(coder)
    64        self._list = [generate_fn() for _ in range(num_elements_per_benchmark)]
    65  
    66      def __call__(self):
    67        # Calling coder operations on a single element at a time may incur
    68        # unrelevant overhead. To compensate, we use a list elements.
    69        _ = self._coder.decode(self._coder.encode(self._list))
    70  
    71    CoderBenchmark.__name__ = "%s, %s" % (generate_fn.__name__, str(coder))
    72  
    73    return CoderBenchmark
    74  
    75  
    76  def batch_row_coder_benchmark_factory(generate_fn, use_batch):
    77    """Creates a benchmark that encodes and decodes a list of elements.
    78  
    79    Args:
    80      coder: coder to use to encode an element.
    81      generate_fn: a callable that generates an element.
    82    """
    83    class CoderBenchmark(object):
    84      def __init__(self, num_elements_per_benchmark):
    85        self._use_batch = use_batch
    86        row_instance = generate_fn()
    87        row_type = trivial_inference.instance_to_type(row_instance)
    88        self._row_coder = get_row_coder(row_instance)
    89        self._batch_converter = DataFrameBatchConverterDropIndex(row_type)
    90        self._seq_coder = coders.IterableCoder(self._row_coder)
    91        self._data = self._batch_converter.produce_batch(
    92            [generate_fn() for _ in range(num_elements_per_benchmark)])
    93  
    94      def __call__(self):
    95        if self._use_batch:
    96          impl = self._row_coder.get_impl()
    97          columnar = {
    98              col: self._data[col].to_numpy()
    99              for col in self._data.columns
   100          }
   101          output_stream = coder_impl.create_OutputStream()
   102          impl.encode_batch_to_stream(columnar, output_stream)
   103          impl.decode_batch_from_stream(
   104              columnar, coder_impl.create_InputStream(output_stream.get()))
   105  
   106        else:
   107          # Calling coder operations on a single element at a time may incur
   108          # unrelevant overhead. To compensate, we use a list elements.
   109          self._batch_converter.produce_batch(
   110              self._seq_coder.decode(
   111                  self._seq_coder.encode(
   112                      self._batch_converter.explode_batch(self._data))))
   113  
   114    CoderBenchmark.__name__ = "%s, BatchRowCoder%s" % (
   115        generate_fn.__name__, use_batch)
   116  
   117    return CoderBenchmark
   118  
   119  
   120  def small_int():
   121    return random.randint(0, 127)
   122  
   123  
   124  def large_int():
   125    return random.randint(sys.maxsize >> 2, sys.maxsize)
   126  
   127  
   128  def random_string(length):
   129    return ''.join(
   130        random.choice(string.ascii_letters + string.digits)
   131        for _ in range(length))
   132  
   133  
   134  def small_string():
   135    return random_string(4)
   136  
   137  
   138  def large_string():
   139    return random_string(100)
   140  
   141  
   142  def list_int(size):
   143    return [small_int() for _ in range(size)]
   144  
   145  
   146  def dict_int_int(size):
   147    return {i: i for i in list_int(size)}
   148  
   149  
   150  def small_list():
   151    return list_int(10)
   152  
   153  
   154  def large_list():
   155    # Bool is the last item in FastPrimitiveCoders before pickle.
   156    return [bool(k) for k in list_int(1000)]
   157  
   158  
   159  def small_tuple():
   160    # Benchmark a common case of 2-element tuples.
   161    return tuple(list_int(2))
   162  
   163  
   164  def large_tuple():
   165    return tuple(large_list())
   166  
   167  
   168  def small_dict():
   169    return {i: i for i in small_list()}
   170  
   171  
   172  def large_dict():
   173    return {i: i for i in large_list()}
   174  
   175  
   176  def large_iterable():
   177    yield 'a' * coders.coder_impl.SequenceCoderImpl._DEFAULT_BUFFER_SIZE
   178    for k in range(1000):
   179      yield k
   180  
   181  
   182  def random_message_with_map(size):
   183    message = test_message.MessageWithMap()
   184    keys = list_int(size)
   185    random.shuffle(keys)
   186    for key in keys:
   187      message.field1[str(key)].field1 = small_string()
   188    return message
   189  
   190  
   191  def small_message_with_map():
   192    return random_message_with_map(5)
   193  
   194  
   195  def large_message_with_map():
   196    return random_message_with_map(20)
   197  
   198  
   199  def globally_windowed_value():
   200    return windowed_value.WindowedValue(
   201        value=small_int(), timestamp=12345678, windows=(window.GlobalWindow(), ))
   202  
   203  
   204  def random_windowed_value(num_windows):
   205    return windowed_value.WindowedValue(
   206        value=small_int(),
   207        timestamp=12345678,
   208        windows=tuple(
   209            window.IntervalWindow(i * 10, i * 10 + small_int())
   210            for i in range(num_windows)))
   211  
   212  
   213  def wv_with_one_window():
   214    return random_windowed_value(num_windows=1)
   215  
   216  
   217  def wv_with_multiple_windows():
   218    return random_windowed_value(num_windows=32)
   219  
   220  
   221  def tiny_row():
   222    return beam.Row(int_value=1)
   223  
   224  
   225  def large_row():
   226    return beam.Row(**{f'int_{ix}': ix for ix in range(20)})
   227  
   228  
   229  def nullable_row():
   230    return beam.Row(**{f'int_{ix}': ix if ix % 2 else None for ix in range(20)})
   231  
   232  
   233  def diverse_row():
   234    return beam.Row(
   235        int_value=1,
   236        float_value=3.14159,
   237        str_value='beam',
   238        row_value=beam.Row(int_value=2, float_value=2.718281828))
   239  
   240  
   241  def get_row_coder(row_instance):
   242    coder = typecoders.registry.get_coder(
   243        trivial_inference.instance_to_type(row_instance))
   244    assert isinstance(coder, row_coder.RowCoder)
   245    return coder
   246  
   247  
   248  def row_coder_benchmark_factory(generate_fn):
   249    return coder_benchmark_factory(get_row_coder(generate_fn()), generate_fn)
   250  
   251  
   252  def run_coder_benchmarks(
   253      num_runs, input_size, seed, verbose, filter_regex='.*'):
   254    random.seed(seed)
   255  
   256    # TODO(https://github.com/apache/beam/issues/18788): Pick coders using type
   257    # hints, for example:
   258    # tuple_coder = typecoders.registry.get_coder(typing.Tuple[int, ...])
   259    benchmarks = [
   260        coder_benchmark_factory(coders.FastPrimitivesCoder(), small_int),
   261        coder_benchmark_factory(coders.FastPrimitivesCoder(), large_int),
   262        coder_benchmark_factory(coders.FastPrimitivesCoder(), small_string),
   263        coder_benchmark_factory(coders.FastPrimitivesCoder(), large_string),
   264        coder_benchmark_factory(coders.FastPrimitivesCoder(), small_list),
   265        coder_benchmark_factory(
   266            coders.IterableCoder(coders.FastPrimitivesCoder()), small_list),
   267        coder_benchmark_factory(coders.FastPrimitivesCoder(), large_list),
   268        coder_benchmark_factory(
   269            coders.IterableCoder(coders.FastPrimitivesCoder()), large_list),
   270        coder_benchmark_factory(
   271            coders.IterableCoder(coders.FastPrimitivesCoder()), large_iterable),
   272        coder_benchmark_factory(coders.FastPrimitivesCoder(), small_tuple),
   273        coder_benchmark_factory(coders.FastPrimitivesCoder(), large_tuple),
   274        coder_benchmark_factory(coders.FastPrimitivesCoder(), small_dict),
   275        coder_benchmark_factory(coders.FastPrimitivesCoder(), large_dict),
   276        coder_benchmark_factory(
   277            coders.ProtoCoder(test_message.MessageWithMap),
   278            small_message_with_map),
   279        coder_benchmark_factory(
   280            coders.ProtoCoder(test_message.MessageWithMap),
   281            large_message_with_map),
   282        coder_benchmark_factory(
   283            coders.DeterministicProtoCoder(test_message.MessageWithMap),
   284            small_message_with_map),
   285        coder_benchmark_factory(
   286            coders.DeterministicProtoCoder(test_message.MessageWithMap),
   287            large_message_with_map),
   288        coder_benchmark_factory(
   289            coders.WindowedValueCoder(coders.FastPrimitivesCoder()),
   290            wv_with_one_window),
   291        coder_benchmark_factory(
   292            coders.WindowedValueCoder(
   293                coders.FastPrimitivesCoder(), coders.IntervalWindowCoder()),
   294            wv_with_multiple_windows),
   295        coder_benchmark_factory(
   296            coders.WindowedValueCoder(
   297                coders.FastPrimitivesCoder(), coders.GlobalWindowCoder()),
   298            globally_windowed_value),
   299        coder_benchmark_factory(
   300            coders.LengthPrefixCoder(coders.FastPrimitivesCoder()), small_int),
   301        row_coder_benchmark_factory(tiny_row),
   302        row_coder_benchmark_factory(large_row),
   303        row_coder_benchmark_factory(nullable_row),
   304        row_coder_benchmark_factory(diverse_row),
   305        batch_row_coder_benchmark_factory(tiny_row, False),
   306        batch_row_coder_benchmark_factory(tiny_row, True),
   307        batch_row_coder_benchmark_factory(large_row, False),
   308        batch_row_coder_benchmark_factory(large_row, True),
   309        batch_row_coder_benchmark_factory(nullable_row, False),
   310        batch_row_coder_benchmark_factory(nullable_row, True),
   311        batch_row_coder_benchmark_factory(diverse_row, False),
   312        batch_row_coder_benchmark_factory(diverse_row, True),
   313    ]
   314  
   315    suite = [
   316        utils.BenchmarkConfig(b, input_size, num_runs) for b in benchmarks
   317        if re.search(filter_regex, b.__name__, flags=re.I)
   318    ]
   319    utils.run_benchmarks(suite, verbose=verbose)
   320  
   321  
   322  if __name__ == "__main__":
   323    logging.basicConfig()
   324  
   325    parser = argparse.ArgumentParser()
   326    parser.add_argument('--filter', default='.*')
   327    parser.add_argument('--num_runs', default=20, type=int)
   328    parser.add_argument('--num_elements_per_benchmark', default=1000, type=int)
   329    parser.add_argument('--seed', default=42, type=int)
   330    options = parser.parse_args()
   331  
   332    utils.check_compiled("apache_beam.coders.coder_impl")
   333  
   334    num_runs = 20
   335    num_elements_per_benchmark = 1000
   336    seed = 42  # Fix the seed for better consistency
   337  
   338    run_coder_benchmarks(
   339        options.num_runs,
   340        options.num_elements_per_benchmark,
   341        options.seed,
   342        verbose=True,
   343        filter_regex=options.filter)