forked from apache/beam
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Main.java
397 lines (366 loc) · 15.2 KB
/
Main.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.beam.sdk.nexmark;
import com.google.api.services.bigquery.model.TableFieldSchema;
import com.google.api.services.bigquery.model.TableRow;
import com.google.api.services.bigquery.model.TableSchema;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableList;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Paths;
import java.nio.file.StandardOpenOption;
import java.util.ArrayList;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import javax.annotation.Nullable;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.coders.CoderException;
import org.apache.beam.sdk.coders.CustomCoder;
import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.coders.SerializableCoder;
import org.apache.beam.sdk.coders.StringUtf8Coder;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices;
import org.apache.beam.sdk.io.gcp.bigquery.TableDestination;
import org.apache.beam.sdk.nexmark.model.Auction;
import org.apache.beam.sdk.nexmark.model.Bid;
import org.apache.beam.sdk.nexmark.model.Person;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.transforms.SerializableFunction;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.ValueInSingleWindow;
import org.joda.time.Duration;
import org.joda.time.Instant;
/**
* An implementation of the 'NEXMark queries' for Beam.
* These are multiple queries over a three table schema representing an online auction system:
* <ul>
* <li>{@link Person} represents a person submitting an item for auction and/or making a bid
* on an auction.
* <li>{@link Auction} represents an item under auction.
* <li>{@link Bid} represents a bid for an item under auction.
* </ul>
* The queries exercise many aspects of the Beam model.
*
* <p>We synthesize the creation of people, auctions and bids in real-time. The data is not
* particularly sensible.
*
* <p>See <a href="http://datalab.cs.pdx.edu/niagaraST/NEXMark/">
* http://datalab.cs.pdx.edu/niagaraST/NEXMark/</a>
*/
public class Main<OptionT extends NexmarkOptions> {
/**
* Entry point.
*/
void runAll(OptionT options, NexmarkLauncher nexmarkLauncher) throws IOException {
Instant start = Instant.now();
Map<NexmarkConfiguration, NexmarkPerf> baseline = loadBaseline(options.getBaselineFilename());
Map<NexmarkConfiguration, NexmarkPerf> actual = new LinkedHashMap<>();
Iterable<NexmarkConfiguration> configurations = options.getSuite().getConfigurations(options);
boolean successful = true;
try {
// Run all the configurations.
for (NexmarkConfiguration configuration : configurations) {
NexmarkPerf perf = nexmarkLauncher.run(configuration);
if (perf != null) {
if (perf.errors == null || perf.errors.size() > 0) {
successful = false;
}
appendPerf(options.getPerfFilename(), configuration, perf);
actual.put(configuration, perf);
// Summarize what we've run so far.
saveSummary(null, configurations, actual, baseline, start, options);
}
}
if (options.getExportSummaryToBigQuery()){
savePerfsToBigQuery(options, actual, null);
}
} finally {
if (options.getMonitorJobs()) {
// Report overall performance.
saveSummary(options.getSummaryFilename(), configurations, actual, baseline, start, options);
saveJavascript(options.getJavascriptFilename(), configurations, actual, baseline, start);
}
}
if (!successful) {
throw new RuntimeException("Execution was not successful");
}
}
@VisibleForTesting
static void savePerfsToBigQuery(
NexmarkOptions options,
Map<NexmarkConfiguration, NexmarkPerf> perfs,
@Nullable BigQueryServices testBigQueryServices) {
Pipeline pipeline = Pipeline.create(options);
PCollection<KV<NexmarkConfiguration, NexmarkPerf>> perfsPCollection =
pipeline.apply(
Create.of(perfs)
.withCoder(
KvCoder.of(
SerializableCoder.of(NexmarkConfiguration.class),
new CustomCoder<NexmarkPerf>() {
@Override
public void encode(NexmarkPerf value, OutputStream outStream)
throws CoderException, IOException {
StringUtf8Coder.of().encode(value.toString(), outStream);
}
@Override
public NexmarkPerf decode(InputStream inStream)
throws CoderException, IOException {
String perf = StringUtf8Coder.of().decode(inStream);
return NexmarkPerf.fromString(perf);
}
})));
TableSchema tableSchema =
new TableSchema()
.setFields(
ImmutableList.of(
new TableFieldSchema().setName("runtimeSec").setType("FLOAT"),
new TableFieldSchema().setName("eventsPerSec").setType("FLOAT"),
new TableFieldSchema()
.setName("numResults")
.setType("INTEGER")));
String tableSpec =
NexmarkUtils.tableSpec(options, "{query}", 0L, null);
SerializableFunction<
ValueInSingleWindow<KV<NexmarkConfiguration, NexmarkPerf>>, TableDestination>
tableFunction =
input -> new TableDestination(
tableSpec.replace("{query}", String.valueOf(input.getValue().getKey().query)),
"perfkit queries");
SerializableFunction<KV<NexmarkConfiguration, NexmarkPerf>, TableRow> rowFunction =
input -> {
NexmarkPerf nexmarkPerf = input.getValue();
TableRow row = new TableRow()
.set("runtimeSec", nexmarkPerf.runtimeSec)
.set("eventsPerSec", nexmarkPerf.eventsPerSec)
.set("numResults", nexmarkPerf.numResults);
return row;
};
BigQueryIO.Write io =
BigQueryIO.<KV<NexmarkConfiguration, NexmarkPerf>>write()
.to(tableFunction)
.withSchema(tableSchema)
.withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED)
.withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_APPEND)
.withFormatFunction(rowFunction);
if (testBigQueryServices != null){
io = io.withTestServices(testBigQueryServices);
}
perfsPCollection.apply("savePerfsToBigQuery", io);
pipeline.run();
}
/**
* Append the pair of {@code configuration} and {@code perf} to perf file.
*/
private void appendPerf(
@Nullable String perfFilename, NexmarkConfiguration configuration,
NexmarkPerf perf) {
if (perfFilename == null) {
return;
}
List<String> lines = new ArrayList<>();
lines.add("");
lines.add(String.format("# %s", Instant.now()));
lines.add(String.format("# %s", configuration.toShortString()));
lines.add(configuration.toString());
lines.add(perf.toString());
try {
Files.write(Paths.get(perfFilename), lines, StandardCharsets.UTF_8, StandardOpenOption.CREATE,
StandardOpenOption.APPEND);
} catch (IOException e) {
throw new RuntimeException("Unable to write perf file: ", e);
}
NexmarkUtils.console("appended results to perf file %s.", perfFilename);
}
/**
* Load the baseline perf.
*/
@Nullable
private static Map<NexmarkConfiguration, NexmarkPerf> loadBaseline(
@Nullable String baselineFilename) {
if (baselineFilename == null) {
return null;
}
Map<NexmarkConfiguration, NexmarkPerf> baseline = new LinkedHashMap<>();
List<String> lines;
try {
lines = Files.readAllLines(Paths.get(baselineFilename), StandardCharsets.UTF_8);
} catch (IOException e) {
throw new RuntimeException("Unable to read baseline perf file: ", e);
}
for (int i = 0; i < lines.size(); i++) {
if (lines.get(i).startsWith("#") || lines.get(i).trim().isEmpty()) {
continue;
}
NexmarkConfiguration configuration = NexmarkConfiguration.fromString(lines.get(i++));
NexmarkPerf perf = NexmarkPerf.fromString(lines.get(i));
baseline.put(configuration, perf);
}
NexmarkUtils.console("loaded %d entries from baseline file %s.", baseline.size(),
baselineFilename);
return baseline;
}
private static final String LINE =
"==========================================================================================";
/** Print summary of {@code actual} vs (if non-null) {@code baseline}. */
private static void saveSummary(
@Nullable String summaryFilename,
Iterable<NexmarkConfiguration> configurations,
Map<NexmarkConfiguration, NexmarkPerf> actual,
@Nullable Map<NexmarkConfiguration, NexmarkPerf> baseline,
Instant start,
NexmarkOptions options) {
List<String> lines = new ArrayList<>();
lines.add("");
lines.add(LINE);
lines.add(
String.format("Run started %s and ran for %s", start, new Duration(start, Instant.now())));
lines.add("");
lines.add("Default configuration:");
lines.add(NexmarkConfiguration.DEFAULT.toString());
lines.add("");
lines.add("Configurations:");
lines.add(" Conf Description");
int conf = 0;
for (NexmarkConfiguration configuration : configurations) {
lines.add(String.format(" %04d %s", conf++, configuration.toShortString()));
NexmarkPerf actualPerf = actual.get(configuration);
if (actualPerf != null && actualPerf.jobId != null) {
lines.add(String.format(" %4s [Ran as job %s]", "", actualPerf.jobId));
}
}
lines.add("");
lines.add("Performance:");
lines.add(String.format(" %4s %12s %12s %12s %12s %12s %12s", "Conf", "Runtime(sec)",
"(Baseline)", "Events(/sec)", "(Baseline)", "Results", "(Baseline)"));
conf = 0;
for (NexmarkConfiguration configuration : configurations) {
String line = String.format(" %04d ", conf++);
NexmarkPerf actualPerf = actual.get(configuration);
if (actualPerf == null) {
line += "*** not run ***";
} else {
NexmarkPerf baselinePerf = baseline == null ? null : baseline.get(configuration);
double runtimeSec = actualPerf.runtimeSec;
line += String.format("%12.1f ", runtimeSec);
if (baselinePerf == null) {
line += String.format("%12s ", "");
} else {
double baselineRuntimeSec = baselinePerf.runtimeSec;
double diff = ((runtimeSec - baselineRuntimeSec) / baselineRuntimeSec) * 100.0;
line += String.format("%+11.2f%% ", diff);
}
double eventsPerSec = actualPerf.eventsPerSec;
line += String.format("%12.1f ", eventsPerSec);
if (baselinePerf == null) {
line += String.format("%12s ", "");
} else {
double baselineEventsPerSec = baselinePerf.eventsPerSec;
double diff = ((eventsPerSec - baselineEventsPerSec) / baselineEventsPerSec) * 100.0;
line += String.format("%+11.2f%% ", diff);
}
long numResults = actualPerf.numResults;
line += String.format("%12d ", numResults);
if (baselinePerf == null) {
line += String.format("%12s", "");
} else {
long baselineNumResults = baselinePerf.numResults;
long diff = numResults - baselineNumResults;
line += String.format("%+12d", diff);
}
}
lines.add(line);
if (actualPerf != null) {
List<String> errors = actualPerf.errors;
if (errors == null) {
errors = new ArrayList<>();
errors.add("NexmarkGoogleRunner returned null errors list");
}
for (String error : errors) {
lines.add(String.format(" %4s *** %s ***", "", error));
}
}
}
lines.add(LINE);
lines.add("");
for (String line : lines) {
System.out.println(line);
}
if (summaryFilename != null) {
try {
Files.write(Paths.get(summaryFilename), lines, StandardCharsets.UTF_8,
StandardOpenOption.CREATE, StandardOpenOption.APPEND);
} catch (IOException e) {
throw new RuntimeException("Unable to save summary file: ", e);
}
NexmarkUtils.console("appended summary to summary file %s.", summaryFilename);
}
}
/**
* Write all perf data and any baselines to a javascript file which can be used by
* graphing page etc.
*/
private static void saveJavascript(
@Nullable String javascriptFilename,
Iterable<NexmarkConfiguration> configurations, Map<NexmarkConfiguration, NexmarkPerf> actual,
@Nullable Map<NexmarkConfiguration, NexmarkPerf> baseline, Instant start) {
if (javascriptFilename == null) {
return;
}
List<String> lines = new ArrayList<>();
lines.add(String.format(
"// Run started %s and ran for %s", start, new Duration(start, Instant.now())));
lines.add("var all = [");
for (NexmarkConfiguration configuration : configurations) {
lines.add(" {");
lines.add(String.format(" config: %s", configuration));
NexmarkPerf actualPerf = actual.get(configuration);
if (actualPerf != null) {
lines.add(String.format(" ,perf: %s", actualPerf));
}
NexmarkPerf baselinePerf = baseline == null ? null : baseline.get(configuration);
if (baselinePerf != null) {
lines.add(String.format(" ,baseline: %s", baselinePerf));
}
lines.add(" },");
}
lines.add("];");
try {
Files.write(Paths.get(javascriptFilename), lines, StandardCharsets.UTF_8,
StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING);
} catch (IOException e) {
throw new RuntimeException("Unable to save javascript file: ", e);
}
NexmarkUtils.console("saved javascript to file %s.", javascriptFilename);
}
public static void main(String[] args) throws IOException {
NexmarkOptions options = PipelineOptionsFactory.fromArgs(args)
.withValidation()
.as(NexmarkOptions.class);
NexmarkLauncher<NexmarkOptions> nexmarkLauncher = new NexmarkLauncher<>(options);
new Main<>().runAll(options, nexmarkLauncher);
}
}