lindong28 commented on code in PR #87:
URL: https://github.com/apache/flink-ml/pull/87#discussion_r866599491
##########
flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/Benchmark.java:
##########
@@ -31,34 +31,33 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.util.ArrayList;
import java.util.Arrays;
-import java.util.List;
+import java.util.Collections;
import java.util.Map;
/** Entry class for benchmark execution. */
public class Benchmark {
- private static final Logger LOG = LoggerFactory.getLogger(Benchmark.class);
-
static final String VERSION_KEY = "version";
- static final Option HELP_OPTION =
+ private static final Logger LOG = LoggerFactory.getLogger(Benchmark.class);
+
+ private static final Option HELP_OPTION =
Option.builder("h")
.longOpt("help")
.desc("Show the help message for the command line
interface.")
.build();
- static final Option OUTPUT_FILE_OPTION =
+ private static final Option OUTPUT_FILE_OPTION =
Option.builder()
.longOpt("output-file")
.desc("The output file name to save benchmark results.")
.hasArg()
.build();
- static final Options OPTIONS =
+ private static final Options OPTIONS =
new Options().addOption(HELP_OPTION).addOption(OUTPUT_FILE_OPTION);
- public static void printHelp() {
+ private static void printHelp() {
HelpFormatter formatter = new HelpFormatter();
formatter.setLeftPadding(5);
formatter.setWidth(80);
Review Comment:
Should we update the line below to use the new script name?
And we can simplify the code a bit by using `\n` in `"./benchmark-run.sh
<config-file-path> [OPTIONS]\n"` instead of using `System.out.println()`.
##########
flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/Benchmark.java:
##########
@@ -71,46 +70,59 @@ public static void printHelp() {
System.out.println();
}
- @SuppressWarnings("unchecked")
- public static void executeBenchmarks(CommandLine commandLine) throws
Exception {
+ private static void executeBenchmarks(CommandLine commandLine) throws
Exception {
String configFile = commandLine.getArgs()[0];
- Map<String, ?> benchmarks = BenchmarkUtils.parseJsonFile(configFile);
- System.out.println("Found benchmarks " + benchmarks.keySet());
+ Map<String, Map<String, Map<String, ?>>> benchmarks =
+ BenchmarkUtils.parseJsonFile(configFile);
+ System.out.println("Found " + benchmarks.keySet().size() + "
benchmarks.");
+ String saveFile =
commandLine.getOptionValue(OUTPUT_FILE_OPTION.getLongOpt());
StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
- List<BenchmarkResult> results = new ArrayList<>();
- for (Map.Entry<String, ?> benchmark : benchmarks.entrySet()) {
- LOG.info("Running benchmark " + benchmark.getKey() + ".");
-
- BenchmarkResult result =
- BenchmarkUtils.runBenchmark(
- tEnv, benchmark.getKey(), (Map<String, ?>)
benchmark.getValue());
-
- results.add(result);
- LOG.info(BenchmarkUtils.getResultsMapAsJson(result));
+ int index = 0;
+ for (Map.Entry<String, Map<String, Map<String, ?>>> entry :
benchmarks.entrySet()) {
+ String benchmarkName = entry.getKey();
+ Map<String, Map<String, ?>> benchmarkMap = entry.getValue();
+
+ LOG.info(
+ String.format(
+ "Running benchmark %d/%d: %s",
+ index++, benchmarks.keySet().size(),
benchmarkName));
+
+ try {
+ BenchmarkResult result =
+ BenchmarkUtils.runBenchmark(tEnv, benchmarkName,
benchmarkMap);
+ benchmarkMap.put("results", result.toMap());
+ LOG.info(String.format("Benchmark %s finished.\n%s",
benchmarkName, benchmarkMap));
+ } catch (Exception e) {
+ benchmarkMap.put(
+ "results",
+ Collections.singletonMap(
+ "exception",
+ String.format(
+ "%s(%s:%s)",
+ e,
+ e.getStackTrace()[0].getFileName(),
+
e.getStackTrace()[0].getLineNumber())));
+ LOG.info(String.format("Benchmark %s failed.\n%s",
benchmarkName, e));
Review Comment:
Should this be `LOG.error(...)`?
##########
flink-ml-dist/src/main/flink-ml-bin/bin/benchmark-results-visualize.py:
##########
@@ -0,0 +1,68 @@
+################################################################################
+# 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.
+################################################################################
+import argparse
+import json
+import matplotlib.pyplot as plt
+import re
+
+
+def get_nested_field_value(nested_fields, field_names):
+ for field_name in field_names:
+ nested_fields = nested_fields[field_name]
+ return nested_fields
+
+
+def benchmark_results_visualize(file_name, name_pattern, x_field, y_field):
Review Comment:
Would it be a bit more intuitive to name this method as
`visualize_benchmark_results(...)`?
##########
flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/Benchmark.java:
##########
@@ -71,46 +70,59 @@ public static void printHelp() {
System.out.println();
}
- @SuppressWarnings("unchecked")
- public static void executeBenchmarks(CommandLine commandLine) throws
Exception {
+ private static void executeBenchmarks(CommandLine commandLine) throws
Exception {
String configFile = commandLine.getArgs()[0];
- Map<String, ?> benchmarks = BenchmarkUtils.parseJsonFile(configFile);
- System.out.println("Found benchmarks " + benchmarks.keySet());
+ Map<String, Map<String, Map<String, ?>>> benchmarks =
+ BenchmarkUtils.parseJsonFile(configFile);
+ System.out.println("Found " + benchmarks.keySet().size() + "
benchmarks.");
+ String saveFile =
commandLine.getOptionValue(OUTPUT_FILE_OPTION.getLongOpt());
StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
- List<BenchmarkResult> results = new ArrayList<>();
- for (Map.Entry<String, ?> benchmark : benchmarks.entrySet()) {
- LOG.info("Running benchmark " + benchmark.getKey() + ".");
-
- BenchmarkResult result =
- BenchmarkUtils.runBenchmark(
- tEnv, benchmark.getKey(), (Map<String, ?>)
benchmark.getValue());
-
- results.add(result);
- LOG.info(BenchmarkUtils.getResultsMapAsJson(result));
+ int index = 0;
+ for (Map.Entry<String, Map<String, Map<String, ?>>> entry :
benchmarks.entrySet()) {
+ String benchmarkName = entry.getKey();
+ Map<String, Map<String, ?>> benchmarkMap = entry.getValue();
+
+ LOG.info(
+ String.format(
+ "Running benchmark %d/%d: %s",
+ index++, benchmarks.keySet().size(),
benchmarkName));
+
+ try {
+ BenchmarkResult result =
+ BenchmarkUtils.runBenchmark(tEnv, benchmarkName,
benchmarkMap);
+ benchmarkMap.put("results", result.toMap());
+ LOG.info(String.format("Benchmark %s finished.\n%s",
benchmarkName, benchmarkMap));
+ } catch (Exception e) {
+ benchmarkMap.put(
+ "results",
+ Collections.singletonMap(
+ "exception",
+ String.format(
+ "%s(%s:%s)",
+ e,
+ e.getStackTrace()[0].getFileName(),
+
e.getStackTrace()[0].getLineNumber())));
+ LOG.info(String.format("Benchmark %s failed.\n%s",
benchmarkName, e));
+ }
}
-
- String benchmarkResultsJson =
- BenchmarkUtils.getResultsMapAsJson(results.toArray(new
BenchmarkResult[0]));
System.out.println("Benchmarks execution completed.");
- System.out.println("Benchmark results summary:");
- System.out.println(benchmarkResultsJson);
+ String benchmarkResultsJson =
+ ReadWriteUtils.OBJECT_MAPPER
+ .writerWithDefaultPrettyPrinter()
+ .writeValueAsString(benchmarks);
if (commandLine.hasOption(OUTPUT_FILE_OPTION.getLongOpt())) {
- String saveFile =
commandLine.getOptionValue(OUTPUT_FILE_OPTION.getLongOpt());
ReadWriteUtils.saveToFile(saveFile, benchmarkResultsJson, true);
System.out.println("Benchmark results saved as json in " +
saveFile + ".");
+ } else {
+ System.out.println("Benchmark results summary:");
Review Comment:
Would it be simpler and more consistent with the `LOG.info(...)` above by
using the following code:
`System.out.println(String.format("Benchmark results summary:\n%s",
benchmarkResultsJson))`
##########
flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/Benchmark.java:
##########
@@ -120,7 +132,10 @@ public static void main(String[] args) throws Exception {
} else if (commandLine.getArgs().length == 1) {
executeBenchmarks(commandLine);
} else {
- printInvalidError(args);
+ System.out.println("Invalid command line arguments " +
Arrays.toString(args));
Review Comment:
Would it be simpler and more consistent with the LOG.info(...) above by
using the following code:
`System.out.println(String.format("Invalid command line arguments %s \n",
Arrays.toString(args)))`
##########
flink-ml-benchmark/README.md:
##########
@@ -63,59 +63,89 @@ Then in Flink ML's binary distribution's folder, execute
the following command
to run the default benchmarks.
```bash
-./bin/flink-ml-benchmark.sh ./conf/benchmark-conf.json --output-file
results.json
+./bin/benchmark-run.sh ./conf/benchmark-conf.json --output-file results.json
Review Comment:
Would it be simpler and more readable to use the following command
```
./bin/benchmark-run.sh conf/benchmark-conf.json --output-file results.json
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]