|
23 | 23 |
|
24 | 24 | This executes the same codepaths that are run on the Fn API (and Dataflow) |
25 | 25 | workers, but is generally easier to run (locally) and more stable. It does |
26 | | -not, on the other hand, excercise any non-trivial amount of IO (e.g. shuffle). |
| 26 | +not, on the other hand, exercise any non-trivial amount of IO (e.g. shuffle). |
27 | 27 |
|
28 | 28 | Run as |
29 | 29 |
|
|
32 | 32 |
|
33 | 33 | # pytype: skip-file |
34 | 34 |
|
| 35 | +import argparse |
35 | 36 | import logging |
36 | | -import time |
37 | | - |
38 | | -from scipy import stats |
39 | 37 |
|
40 | 38 | import apache_beam as beam |
41 | 39 | from apache_beam.tools import utils |
| 40 | +from apache_beam.transforms.window import FixedWindows |
42 | 41 |
|
43 | 42 |
|
44 | | -def run_benchmark(num_maps=100, num_runs=10, num_elements_step=1000): |
45 | | - timings = {} |
46 | | - for run in range(num_runs): |
47 | | - num_elements = num_elements_step * run + 1 |
48 | | - start = time.time() |
| 43 | +def map_pipeline(num_elements, num_maps=100): |
| 44 | + def _pipeline_runner(): |
49 | 45 | with beam.Pipeline() as p: |
50 | 46 | pc = p | beam.Create(list(range(num_elements))) |
51 | 47 | for ix in range(num_maps): |
52 | 48 | pc = pc | 'Map%d' % ix >> beam.FlatMap(lambda x: (None, )) |
53 | | - timings[num_elements] = time.time() - start |
54 | | - print( |
55 | | - "%6d element%s %g sec" % ( |
56 | | - num_elements, |
57 | | - " " if num_elements == 1 else "s", |
58 | | - timings[num_elements])) |
59 | | - |
60 | | - print() |
61 | | - # pylint: disable=unused-variable |
62 | | - gradient, intercept, r_value, p_value, std_err = stats.linregress( |
63 | | - *list(zip(*list(timings.items())))) |
64 | | - print("Fixed cost ", intercept) |
65 | | - print("Per-element ", gradient / num_maps) |
66 | | - print("R^2 ", r_value**2) |
| 49 | + |
| 50 | + return _pipeline_runner |
| 51 | + |
| 52 | + |
| 53 | +def map_with_global_side_input_pipeline(num_elements, num_maps=100): |
| 54 | + def add(element, side_input): |
| 55 | + return element + side_input |
| 56 | + |
| 57 | + def _pipeline_runner(): |
| 58 | + with beam.Pipeline() as p: |
| 59 | + side = p | 'CreateSide' >> beam.Create([1]) |
| 60 | + pc = p | 'CreateMain' >> beam.Create(list(range(num_elements))) |
| 61 | + for ix in range(num_maps): |
| 62 | + pc = pc | 'Map%d' % ix >> beam.Map(add, beam.pvalue.AsSingleton(side)) |
| 63 | + |
| 64 | + return _pipeline_runner |
| 65 | + |
| 66 | + |
| 67 | +def map_with_fixed_window_side_input_pipeline(num_elements, num_maps=100): |
| 68 | + def add(element, side_input): |
| 69 | + return element + side_input |
| 70 | + |
| 71 | + def _pipeline_runner(): |
| 72 | + with beam.Pipeline() as p: |
| 73 | + side = p | 'CreateSide' >> beam.Create( |
| 74 | + [1]) | 'WindowSide' >> beam.WindowInto(FixedWindows(1000)) |
| 75 | + pc = p | 'CreateMain' >> beam.Create(list(range( |
| 76 | + num_elements))) | 'WindowMain' >> beam.WindowInto(FixedWindows(1000)) |
| 77 | + for ix in range(num_maps): |
| 78 | + pc = pc | 'Map%d' % ix >> beam.Map(add, beam.pvalue.AsSingleton(side)) |
| 79 | + |
| 80 | + return _pipeline_runner |
| 81 | + |
| 82 | + |
| 83 | +def run_benchmark( |
| 84 | + starting_point=1, |
| 85 | + num_runs=10, |
| 86 | + num_elements_step=100, |
| 87 | + verbose=True, |
| 88 | + profile_filename_base=None, |
| 89 | +): |
| 90 | + suite = [ |
| 91 | + utils.BenchmarkConfig( |
| 92 | + map_with_fixed_window_side_input_pipeline, |
| 93 | + starting_point * 1000, |
| 94 | + num_runs, |
| 95 | + ), |
| 96 | + utils.LinearRegressionBenchmarkConfig( |
| 97 | + map_pipeline, starting_point, num_elements_step, num_runs), |
| 98 | + utils.BenchmarkConfig( |
| 99 | + map_with_global_side_input_pipeline, |
| 100 | + starting_point * 1000, |
| 101 | + num_runs, |
| 102 | + ), |
| 103 | + utils.BenchmarkConfig( |
| 104 | + map_with_global_side_input_pipeline_uncached, |
| 105 | + starting_point * 1000, |
| 106 | + num_runs, |
| 107 | + ), |
| 108 | + ] |
| 109 | + return utils.run_benchmarks( |
| 110 | + suite, verbose=verbose, profile_filename_base=profile_filename_base) |
67 | 111 |
|
68 | 112 |
|
69 | 113 | if __name__ == '__main__': |
70 | 114 | logging.basicConfig() |
71 | 115 | utils.check_compiled('apache_beam.runners.common') |
72 | | - run_benchmark() |
| 116 | + |
| 117 | + parser = argparse.ArgumentParser() |
| 118 | + parser.add_argument('--num_runs', default=10, type=int) |
| 119 | + parser.add_argument('--starting_point', default=1, type=int) |
| 120 | + parser.add_argument('--increment', default=100, type=int) |
| 121 | + parser.add_argument('--verbose', default=True, type=bool) |
| 122 | + parser.add_argument('--profile_filename_base', default=None, type=str) |
| 123 | + options = parser.parse_args() |
| 124 | + |
| 125 | + run_benchmark( |
| 126 | + options.starting_point, |
| 127 | + options.num_runs, |
| 128 | + options.increment, |
| 129 | + options.verbose, |
| 130 | + options.profile_filename_base, |
| 131 | + ) |
0 commit comments