You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/03/17 06:55:23 UTC

[GitHub] [flink-ml] yunfengzhou-hub opened a new pull request #71: [Flink 26443] Add benchmark framework for Flink ML

yunfengzhou-hub opened a new pull request #71:
URL: https://github.com/apache/flink-ml/pull/71


   ## What is the purpose of the change
   This PR adds a framework to benchmark Flink ML algorithms.
   
   ## Brief change log
   - Adds `flink-ml-benchmark` module to hold codes for executing benchmarks.
       - Adds `Benchmark`, `BenchmarkResult` and `BenchmarkUtils` class as benchmark's basic infrastructure.
       - Adds `GeneratorParams`, `DataGenerator` and `GeneratorUtils` as infrastructure to generate random test data for benchmarks.
       - Adds `KMeansInputsGenerator`, `KMeansModelDataGenerator` and `KMeansModelInputsGenerator` as an example implementation of the benchmark API.
       - Adds `benchmark-example-conf.json` and `BenchmarkTest` to test benchmark framework's correctness.
   - Adds `flink-ml-dist` module to generate binary distribution of Flink ML's source code.
       - Adds `flink-ml-benchmark.sh` to provide CLI to run benchmark jobs.
   - Updates `README.md` to record the usage of benchmark java/CLI interface.
   
   ## Does this pull request potentially affect one of the following parts:
   - Dependencies (does it add or upgrade a dependency): (no)
   - The public API, i.e., is any changed class annotated with @public(Evolving): (no)
   - Does this pull request introduce a new feature? (yes)
   - If yes, how is the feature documented? (Java doc)


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r837430785



##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,261 @@
+# Flink ML Benchmark Guideline
+
+This document provides instructions about how to run benchmarks on Flink ML's
+stages.
+
+## Write Benchmark Programs
+
+### Add Maven Dependencies
+
+In order to write Flink ML's java benchmark programs, first make sure that the
+following dependencies have been added to your maven project's `pom.xml`.
+
+```xml
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-core_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-iteration_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-lib_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-benchmark_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>statefun-flink-core</artifactId>
+  <version>3.1.0</version>
+  <exclusions>
+    <exclusion>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-streaming-java_2.12</artifactId>
+    </exclusion>
+  </exclusions>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-table-api-java-bridge_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-table-planner_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-clients_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+```
+
+### Write Java Program
+
+Then you can write a program as follows to run benchmark on Flink ML stages. The
+example code below tests the performance of Flink ML's KMeans algorithm, with
+the default configuration parameters used.
+
+```java
+public class Main {
+    public static void main(String[] args) throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        KMeans kMeans = new KMeans();
+        KMeansInputsGenerator inputsGenerator = new KMeansInputsGenerator();
+
+        BenchmarkResult result =
+                BenchmarkUtils.runBenchmark("exampleBenchmark", tEnv, kMeans, inputsGenerator);
+
+        BenchmarkUtils.printResult(result);
+    }
+}
+```
+
+### Execute Benchmark Program
+
+After executing the `main()` method above, you will see benchmark results
+printed out in your terminal. An example of the printed content is as follows.
+
+```
+Benchmark Name: exampleBenchmark
+Total Execution Time(ms): 828.0
+```
+
+### Configure Benchmark Parameters
+
+If you want to run benchmark on customed configuration parameters, you can set
+them with Flink ML's `WithParams` API as follows.
+
+```java
+KMeans kMeans = new KMeans()
+  .setK(5)
+  .setMaxIter(50);
+KMeansInputsGenerator inputsGenerator = new KMeansInputsGenerator()
+  .setDims(3)
+  .setDataSize(10000);
+```
+
+## Execute Benchmark through Command-Line Interface (CLI)
+
+You can also configure and execute benchmarks through Command-Line Interface
+(CLI) without writing java programs.
+
+### Prerequisites
+
+Before using Flink ML's CLI, make sure you have installed Flink 1.14 in your
+local environment, and that you have started a Flink cluster locally. If not,
+you can start a standalone session with the following command.
+
+```bash
+$ start-cluster
+```
+
+In order to use Flink ML's CLI you need to have the latest binary distribution
+of Flink ML. You can acquire the distribution by building Flink ML's source code
+locally, which means to execute the following command in Flink ML repository's
+root directory.
+
+```bash
+$ mvn clean package -DskipTests
+```
+
+After executing the command above, you will be able to find the binary
+distribution under
+`./flink-ml-dist/target/flink-ml-<version>-bin/flink-ml-<version>/`.
+
+### Run Benchmark CLI
+
+In the binary distribution's folder, execute the following command to run an
+example benchmark.
+
+```bash
+$ ./bin/flink-ml-benchmark.sh ./examples/benchmark-example-conf.json
+```
+
+You will notice that some Flink job is submitted to your Flink cluster, and the
+following information is printed out in your terminal. This means that you have
+successfully executed a benchmark on `KMeansModel`.
+
+```
+Job has been submitted with JobID 85b4a33df5c00a315e0d1142e1d743be
+Program execution finished
+Job with JobID 85b4a33df5c00a315e0d1142e1d743be has finished.
+Job Runtime: 828 ms
+
+Benchmark Name: KMeansModel-1
+Total Execution Time(ms): 828.0
+
+```
+
+### Save Benchmark Result to File
+
+`flink-ml-benchmark.sh` has redirected all warnings and process logs to stderr,
+and the benchmark results to stdout. So if you write the command in the
+following way
+
+```bash
+$ ./bin/flink-ml-benchmark.sh ./examples/benchmark-example-conf.json > output.txt
+```
+
+You will get a clean benchmark result saved in `output.txt` as follows.
+
+```
+Benchmark Name: KMeansModel-1
+Total Execution Time(ms): 828.0
+
+```
+
+### Configuration File Format
+
+The benchmark CLI creates stages and test data according to the input
+configuration file. The configuration file should contain a JSON object in the
+following format.
+
+First of all, the file should contain the following configurations as the
+metadata of the JSON object.
+
+- `"_version"`: The version of the json format. Currently its value must be 1.
+
+Then, each benchmark should be specified through key-object pairs.
+
+The key for each benchmark JSON object will be regarded as the name of the
+benchmark. The benchmark name can contain English letters, numbers, hyphens(-)
+and underscores(_), but should not start with a hyphen or underscore.
+
+The value of each benchmark name should be a JSON object containing at least
+`"stage"` and `"inputs"`. If the stage to be tested is a `Model` and its model
+data needs to be explicitly set, the JSON object should also contain
+`"modelData"`.
+
+The value of `"stage"`, `"inputs"` or `"modelData"` should be a JSON object
+containing `"className"` and `paramMap`.
+
+- `"className"`'s value should be the full classpath of a `WithParams` subclass.
+  For `"stage"`, the class should be a subclass of `Stage`. For `"inputs"` or
+  `"modelData"`, the class should be a subclass of `DataGenerator`.
+
+- `"paramMap"`'s value should be a JSON object containing the parameters related
+  to the specific `Stage` or `DataGenerator`. Note that all parameter values
+  should be wrapped as strings.
+
+Combining the format requirements above, an example configuration file is as
+follows.
+
+```json
+{
+  "_version": 1,
+  "KMeansModel-1": {
+    "stage": {
+      "className": "org.apache.flink.ml.clustering.kmeans.KMeansModel",
+      "paramMap": {
+        "featuresCol": "\"features\"",

Review comment:
       According to offline discussion, I'll modify the implementations in `ReadWriteUtils` and `Param` so that all saved stage metadata and loaded benchmark configurations can specify `"featuresCol": "features"`.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r840332199



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/Benchmark.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** Entry class for benchmark execution. */
+public class Benchmark {
+    private static final Logger LOG = LoggerFactory.getLogger(Benchmark.class);
+
+    @SuppressWarnings("unchecked")
+    public static void main(String[] args) throws Exception {

Review comment:
       OK. I'll add the `--help` option.
   
   When `flink` receives invalid arguments, it would print out information like follows, instead of display help messages:
   ```
   $ flink xxx
   "xxx" is not a valid action.
   
   Valid actions are "run", "run-application", "list", "info", "savepoint", "stop", or "cancel".
   
   Specify the version option (-v or --version) to print Flink version.
   
   Specify the help option (-h or --help) to get help on the command.
   ```
   
   Thus I think we can also print a message like above. With that said, our help message is quite simple, so it is also OK with me if it is better to print out the help message directly.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#issuecomment-1086595525


   Thanks for the comments. I have updated the PR according to the comments.


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r840253077



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/BenchmarkUtils.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.ml.api.AlgoOperator;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.api.Model;
+import org.apache.flink.ml.api.Stage;
+import org.apache.flink.ml.benchmark.generator.DataGenerator;
+import org.apache.flink.ml.common.datastream.TableUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/** Utility methods for benchmarks. */
+public class BenchmarkUtils {
+    /**
+     * Instantiates a benchmark from its parameter map and executes the benchmark in the provided
+     * environment.
+     *
+     * @return Results of the executed benchmark.
+     */
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    public static BenchmarkResult runBenchmark(
+            StreamTableEnvironment tEnv, String name, Map<String, ?> params) throws Exception {
+        Stage stage = ReadWriteUtils.instantiateWithParams((Map<String, ?>) params.get("stage"));
+        DataGenerator inputsGenerator =
+                ReadWriteUtils.instantiateWithParams((Map<String, ?>) params.get("inputs"));
+        DataGenerator modelDataGenerator = null;
+        if (params.containsKey("modelData")) {
+            modelDataGenerator =
+                    ReadWriteUtils.instantiateWithParams((Map<String, ?>) params.get("modelData"));
+        }
+
+        return runBenchmark(tEnv, name, stage, inputsGenerator, modelDataGenerator);
+    }
+
+    /**
+     * Executes a benchmark from a stage with its inputsGenerator in the provided environment.
+     *
+     * @return Results of the executed benchmark.
+     */
+    public static BenchmarkResult runBenchmark(
+            StreamTableEnvironment tEnv,
+            String name,
+            Stage<?> stage,
+            DataGenerator<?> inputsGenerator)
+            throws Exception {
+        return runBenchmark(tEnv, name, stage, inputsGenerator, null);
+    }
+
+    /**
+     * Executes a benchmark from a stage with its inputsGenerator and modelDataGenerator in the
+     * provided environment.
+     *
+     * @return Results of the executed benchmark.
+     */
+    public static BenchmarkResult runBenchmark(
+            StreamTableEnvironment tEnv,
+            String name,
+            Stage<?> stage,
+            DataGenerator<?> inputsGenerator,
+            DataGenerator<?> modelDataGenerator)
+            throws Exception {
+        StreamExecutionEnvironment env = TableUtils.getExecutionEnvironment(tEnv);
+
+        Table[] inputTables = inputsGenerator.getData(tEnv);
+        if (modelDataGenerator != null) {
+            ((Model<?>) stage).setModelData(modelDataGenerator.getData(tEnv));
+        }
+
+        Table[] outputTables;
+        if (stage instanceof Estimator) {
+            outputTables = ((Estimator<?, ?>) stage).fit(inputTables).getModelData();
+        } else if (stage instanceof AlgoOperator) {
+            outputTables = ((AlgoOperator<?>) stage).transform(inputTables);
+        } else {
+            throw new IllegalArgumentException("Unsupported Stage class " + stage.getClass());
+        }
+
+        for (Table table : outputTables) {
+            tEnv.toDataStream(table).addSink(new CountingAndDiscardingSink<>());
+        }
+
+        JobExecutionResult executionResult = env.execute();
+
+        BenchmarkResult result = new BenchmarkResult();
+        result.name = name;
+        result.totalTimeMs = (double) executionResult.getNetRuntime(TimeUnit.MILLISECONDS);
+        result.inputRecordNum = inputsGenerator.getNumData();
+        result.inputThroughput = result.inputRecordNum * 1000.0 / result.totalTimeMs;
+        result.outputRecordNum =
+                executionResult.getAccumulatorResult(CountingAndDiscardingSink.COUNTER_NAME);
+        result.outputThroughput = result.outputRecordNum * 1000.0 / result.totalTimeMs;
+
+        return result;
+    }
+
+    /** Prints out the provided benchmark result. */
+    public static void printResult(BenchmarkResult result) {

Review comment:
       I think we still need to print out the benchmark results in the terminal. I'll rewrite `printResult` so that it prints in json format.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r840347725



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/generator/GeneratorParams.java
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.flink.ml.benchmark.generator;
+
+import org.apache.flink.ml.common.param.HasSeed;
+import org.apache.flink.ml.param.IntParam;
+import org.apache.flink.ml.param.LongParam;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+
+/** Interface for the generator params. */
+public interface GeneratorParams<T> extends HasSeed<T> {
+    Param<Long> NUM_DATA =
+            new LongParam("numData", "Number of data to be generated.", 10L, ParamValidators.gt(0));
+
+    Param<Integer> DIMS =
+            new IntParam(
+                    "dims",
+                    "Dimension of vector-typed data to be generated.",
+                    1,
+                    ParamValidators.gt(0));
+
+    default long getNumData() {
+        return get(NUM_DATA);
+    }
+
+    default T setNumData(long value) {
+        return set(NUM_DATA, value);
+    }
+
+    default int getDims() {

Review comment:
       OK. I'll move them to `DenseVectorGeneratorParams` and `DenseVectorArrayGeneratorParams`.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
lindong28 commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r841071624



##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,172 @@
+# Flink ML Benchmark Getting Started
+
+This document provides instructions on how to run benchmarks on Flink ML's
+stages in a Linux/MacOS environment.
+
+## Prerequisites
+
+### Install Flink
+
+Please make sure Flink 1.14 or higher version has been installed in your local
+environment. You can refer to the [local
+installation](https://nightlies.apache.org/flink/flink-docs-master/docs/try-flink/local_installation/)
+instruction on Flink's document website for how to achieve this.
+
+### Set Up Flink Environment Variables
+
+After having installed Flink, please register `$FLINK_HOME` as an environment
+variable into your local environment. For example, suppose you have downloaded
+Flink 1.14.0 and placed Flink's binary folder under `/usr/local/`, then you need
+to run the following command:
+
+```bash
+export FLINK_HOME=`/usr/local/flink-1.14.0`
+```
+
+Then please run the following command. If this command returns 1.14.0 or a
+higher version, then it means that the required Flink environment has been
+successfully installed and registered in your local environment.
+
+```bash
+$FLINK_HOME/bin/flink --version
+```
+
+### Acquire Flink ML Binary Distribution
+
+In order to use Flink ML's CLI you need to have the latest binary distribution
+of Flink ML. You can acquire the distribution by building Flink ML's source code
+locally, which means to execute the following command in Flink ML repository's
+root directory.
+
+```bash
+mvn clean package -DskipTests
+cd ./flink-ml-dist/target/flink-ml-*-bin/flink-ml*/
+```
+
+### Start Flink Cluster
+
+Please start a Flink standalone session in your local environment with the
+following command.
+
+```bash
+$FLINK_HOME/bin/start-cluster.sh
+```
+
+You should be able to navigate to the web UI at
+[localhost:8081](http://localhost:8081/) to view the Flink dashboard and see
+that the cluster is up and running.
+
+## Run Benchmark Example
+
+In Flink ML's binary distribution's folder, execute the following command to run
+an example benchmark.
+
+```bash
+./bin/flink-ml-benchmark.sh ./examples/benchmark-example-conf.json --output-file ./output/results.json
+```
+
+You will notice that some Flink job is submitted to your Flink cluster, and the
+following information is printed out in your terminal. This means that you have
+successfully executed a benchmark on `KMeansModel`.
+
+```
+Creating fat jar containing all flink ml dependencies to be submitted.
+Job has been submitted with JobID a5d8868d808eecfb357eb904c961c3bf
+Program execution finished
+Job with JobID a5d8868d808eecfb357eb904c961c3bf has finished.
+Job Runtime: 897 ms
+Accumulator Results: 
+- numElements (java.lang.Long): 10000
+
+
+{
+  "name" : "KMeansModel-1",
+  "totalTimeMs" : 897.0,
+  "inputRecordNum" : 10000,
+  "inputThroughput" : 11148.272017837235,
+  "outputRecordNum" : 10000,
+  "outputThroughput" : 11148.272017837235
+}
+```
+
+The command above would save the results into `./output/results.json` as below.
+
+```json
+[ {
+  "name" : "KMeansModel-1",
+  "totalTimeMs" : 897.0,
+  "inputRecordNum" : 10000,
+  "inputThroughput" : 11148.272017837235,
+  "outputRecordNum" : 10000,
+  "outputThroughput" : 11148.272017837235
+} ]
+```
+
+## Customize Benchmark Configuration
+
+`flink-ml-benchmark.sh` parses benchmarks to be executed according to the input
+configuration file, like `./examples/benchmark-example-conf.json`. It can also

Review comment:
       Could we move the file to the folder `conf` instead of `examples`. By doing this, the directory structure would be simpler and also more consistent with Flink.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
lindong28 commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r841004650



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/BenchmarkUtils.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.ml.api.AlgoOperator;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.api.Model;
+import org.apache.flink.ml.api.Stage;
+import org.apache.flink.ml.benchmark.generator.DataGenerator;
+import org.apache.flink.ml.common.datastream.TableUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/** Utility methods for benchmarks. */
+public class BenchmarkUtils {
+    /**
+     * Instantiates a benchmark from its parameter map and executes the benchmark in the provided
+     * environment.
+     *
+     * @return Results of the executed benchmark.
+     */
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    public static BenchmarkResult runBenchmark(
+            StreamTableEnvironment tEnv, String name, Map<String, ?> params) throws Exception {
+        Stage stage = ReadWriteUtils.instantiateWithParams((Map<String, ?>) params.get("stage"));

Review comment:
       Sounds good.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r840197583



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/generator/GeneratorUtils.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.flink.ml.benchmark.generator;
+
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.linalg.Vectors;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.util.NumberSequenceIterator;
+
+import java.util.Random;
+
+/** Utility methods to generate data for benchmarks. */
+public class GeneratorUtils {
+    /**
+     * Generates random continuous vectors.
+     *
+     * @param env The stream execution environment.
+     * @param numData Number of examples to generate in total.
+     * @param seed The seed to generate seed on each partition.
+     * @param dims Dimension of the vectors to be generated.
+     * @return The generated vector stream.
+     */
+    public static DataStream<DenseVector> generateRandomContinuousVectorStream(

Review comment:
       We might have `discreteVectorStream` in future that only contains 0 and 1. But I agree that we can remove the word `continuous` from the method name, and maybe in future we can add an option on this method or class for users to specify how they would like the values to distribute in the generated vectors.
   
   Given that we are introducing classes like `DenseVectorArrayGenerator`, `GeneratorUtils` would be removed so the rest of this comment is not applicable now.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r840328069



##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,169 @@
+# Flink ML Benchmark Getting Started
+
+This document provides instructions about how to run benchmarks on Flink ML's
+stages in a Linux/MacOS environment.
+
+## Prerequisites
+
+### Installing Flink
+
+Please make sure Flink 1.14 or higher version has been installed in your local
+environment. You can refer to the [local
+installation](https://nightlies.apache.org/flink/flink-docs-master/docs/try-flink/local_installation/)
+instruction on Flink's document website for how to achieve this.
+
+### Setting Up Flink Environment Variables
+
+After having installed Flink, please register `$FLINK_HOME` as an environment
+variable into your local environment, and add `$FLINK_HOME` into your `$PATH`
+variable. This can be completed by running the following commands in the Flink's
+folder.
+
+```bash
+export FLINK_HOME=`pwd`
+export PATH=$FLINK_HOME/bin:$PATH
+```
+
+Then please run the following command. If this command returns 1.14.0 or a
+higher version, then it means that the required Flink environment has been
+successfully installed and registered in your local environment.
+
+```bash
+flink --version
+```
+
+### Acquiring Flink ML Binary Distribution
+
+In order to use Flink ML's CLI you need to have the latest binary distribution
+of Flink ML. You can acquire the distribution by building Flink ML's source code
+locally, which means to execute the following command in Flink ML repository's
+root directory.
+
+```bash
+$ mvn clean package -DskipTests
+```
+
+After executing the command above, you will be able to find the binary
+distribution under
+`./flink-ml-dist/target/flink-ml-<version>-bin/flink-ml-<version>/`.
+
+### Starting Flink Cluster
+
+Please start a Flink standalone session in your local environment with the
+following command.
+
+```bash
+start-cluster.sh
+```
+
+You should be able to navigate to the web UI at
+[localhost:8081](http://localhost:8081/) to view the Flink dashboard and see
+that the cluster is up and running.
+
+## Run Benchmark Example
+
+In Flink ML's binary distribution's folder, execute the following command to run
+an example benchmark.
+
+```bash
+$ ./bin/flink-ml-benchmark.sh ./examples/benchmark-example-conf.json ./output/results.json
+```
+
+You will notice that some Flink job is submitted to your Flink cluster, and the
+following information is printed out in your terminal. This means that you have
+successfully executed a benchmark on `KMeansModel`.
+
+```
+Creating fat jar containing all flink ml dependencies to be submitted.
+Job has been submitted with JobID bdaa54b065adf2c813619113a00337de
+Program execution finished
+Job with JobID bdaa54b065adf2c813619113a00337de has finished.
+Job Runtime: 215 ms
+Accumulator Results: 
+- numElements (java.lang.Long): 10000
+
+
+Benchmark Name: KMeansModel-1
+Total Execution Time: 215.0 ms
+Total Input Record Number: 10000
+Average Input Throughput: 46511.62790697674 events per second

Review comment:
       As we are adopting json format according to other comments, this comment is not applicable now.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#issuecomment-1082653503


   Thanks for the comments. I have updated the PR according to the comments.


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
lindong28 commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r840999157



##########
File path: flink-ml-benchmark/src/test/java/org/apache/flink/ml/benchmark/BenchmarkTest.java
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.ml.benchmark.clustering.kmeans.KMeansInputsGenerator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+import org.apache.flink.ml.clustering.kmeans.KMeansModel;
+import org.apache.flink.ml.param.WithParams;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.test.util.AbstractTestBase;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.InputStream;
+import java.io.PrintStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.ml.util.ReadWriteUtils.OBJECT_MAPPER;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/** Tests benchmarks. */
+@SuppressWarnings("unchecked")
+public class BenchmarkTest extends AbstractTestBase {
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+
+    private final ByteArrayOutputStream outContent = new ByteArrayOutputStream();
+    private final PrintStream originalOut = System.out;
+
+    @Before
+    public void before() {
+        System.setOut(new PrintStream(outContent));
+    }
+
+    @After
+    public void after() {
+        System.setOut(originalOut);

Review comment:
       Thanks for the explanation.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
lindong28 commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r840998630



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/Benchmark.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** Entry class for benchmark execution. */
+public class Benchmark {
+    private static final Logger LOG = LoggerFactory.getLogger(Benchmark.class);
+
+    @SuppressWarnings("unchecked")
+    public static void main(String[] args) throws Exception {

Review comment:
       Thanks for the improvement. The current behavior of the script looks good.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
lindong28 commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r841001187



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/data/DataGenerator.java
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.flink.ml.benchmark.data;
+
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+
+/** Interface for generating data as table arrays. */
+public interface DataGenerator<T extends DataGenerator<T>> extends CommonDataGeneratorParams<T> {
+    /**
+     * Gets an array of Tables containing the data to be generated in the provided stream table

Review comment:
       nits: it does not sound right to say `containing the data to be generated ...`.
   
   How about changing it to `containing the data generated ...`?

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/clustering/kmeans/KMeansModelDataGenerator.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.flink.ml.benchmark.clustering.kmeans;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.ml.benchmark.data.DataGenerator;
+import org.apache.flink.ml.benchmark.data.DenseVectorArrayGenerator;
+import org.apache.flink.ml.benchmark.data.DenseVectorArrayGeneratorParams;
+import org.apache.flink.ml.clustering.kmeans.KMeansModelData;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Class that generates table arrays containing model data for {@link
+ * org.apache.flink.ml.clustering.kmeans.KMeansModel}.
+ */
+public class KMeansModelDataGenerator
+        implements DataGenerator<KMeansModelDataGenerator>,
+                DenseVectorArrayGeneratorParams<KMeansModelDataGenerator> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+
+    public KMeansModelDataGenerator() {
+        ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+    }
+
+    @Override
+    public Table[] getData(StreamTableEnvironment tEnv) {
+        DataGenerator<?> vectorArrayGenerator = new DenseVectorArrayGenerator();

Review comment:
       nits: The code looks busy. Could we add line break in the function, e.g.:
   
   ```
       DataGenerator<?> vectorArrayGenerator = new DenseVectorArrayGenerator();
       ReadWriteUtils.updateExistingParams(vectorArrayGenerator, paramMap);
   
       Table vectorArrayTable = vectorArrayGenerator.getData(tEnv)[0];
       DataStream<KMeansModelData> modelDataStream = tEnv
               .toDataStream(vectorArrayTable, DenseVector[].class)
               .map(new GenerateKMeansModelDataFunction());
   
       return new Table[] {tEnv.fromDataStream(modelDataStream)};
   ```

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/data/DenseVectorArrayGeneratorParams.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.flink.ml.benchmark.data;
+
+import org.apache.flink.ml.param.IntParam;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+
+/** Interface for the vector array generator params. */
+public interface DenseVectorArrayGeneratorParams<T> extends DenseVectorGeneratorParams<T> {

Review comment:
       It seems a bit weird that `KMeansModelDataGenerator` implements a class named `DenseVectorArrayGenerator*`.
   
   Would it be more intuitive to renaming this class as `HasArraySize`?
   
   And would it be more readable to this param class to the folder `benchmark/param`? 
   
   Same for similar classes.

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/data/CommonDataGeneratorParams.java
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.flink.ml.benchmark.data;
+
+import org.apache.flink.ml.common.param.HasOutputCols;
+import org.apache.flink.ml.common.param.HasSeed;
+import org.apache.flink.ml.param.LongParam;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+
+/** Interface for the common generator params. */
+public interface CommonDataGeneratorParams<T> extends HasSeed<T>, HasOutputCols<T> {

Review comment:
       `HasOutputCols` represents a list of column names. But what we need here is a list of list of column names for all output data tables.
   
   How about define a parameter named `columnNames` like below
   
   ```
   // When not specified, the default field names (e.g. f0, f1) are used in the output data tables.
   Param<String[]> COLUMN_NAMES =
           new StringArrayParam(
                   "columnNames", "an array of common-separated strings representing field names of data tables", null, ParamValidators.alwaysTrue());
   ```

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/Benchmark.java
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.DefaultParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** 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 String SHELL_SCRIPT = "flink-ml-benchmark.sh";

Review comment:
       Instead of explicitly defining this parameter and keeping it in-sync with the script name, would it be simpler to just get the script name as `args[0]`, where `args` is the argument to `main()`?
   
   Note that it is common practice to set args[0] to the string name of the command line tool.

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/Benchmark.java
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.DefaultParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** 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 String SHELL_SCRIPT = "flink-ml-benchmark.sh";
+
+    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 =
+            Option.builder()
+                    .longOpt("output-file")
+                    .desc("The output file name to save benchmark results.")
+                    .hasArg()
+                    .build();
+
+    static final Options OPTIONS =
+            new Options().addOption(HELP_OPTION).addOption(OUTPUT_FILE_OPTION);
+
+    public static void printHelp() {
+        HelpFormatter formatter = new HelpFormatter();
+        formatter.setLeftPadding(5);
+        formatter.setWidth(80);
+
+        System.out.println("./" + SHELL_SCRIPT + " <config-file-path> [OPTIONS]");
+        System.out.println();
+        formatter.setSyntaxPrefix("The following options are available:");
+        formatter.printHelp(" ", OPTIONS);
+
+        System.out.println();
+    }
+
+    @SuppressWarnings("unchecked")
+    public static void executeBenchmarks(CommandLine commandLine) throws Exception {
+        String configFile = commandLine.getArgs()[0];
+
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        InputStream inputStream = new FileInputStream(configFile);
+        Map<String, ?> jsonMap = ReadWriteUtils.OBJECT_MAPPER.readValue(inputStream, Map.class);
+        Preconditions.checkArgument(
+                jsonMap.containsKey(VERSION_KEY) && jsonMap.get(VERSION_KEY).equals(1));
+
+        List<String> benchmarkNames =
+                jsonMap.keySet().stream()
+                        .filter(x -> !x.equals(VERSION_KEY))
+                        .collect(Collectors.toList());
+        LOG.info("Found benchmarks " + benchmarkNames);
+
+        List<BenchmarkResult> results = new ArrayList<>();
+
+        for (String benchmarkName : benchmarkNames) {
+            LOG.info("Running benchmark " + benchmarkName + ".");
+
+            BenchmarkResult result =
+                    BenchmarkUtils.runBenchmark(
+                            tEnv, benchmarkName, (Map<String, ?>) jsonMap.get(benchmarkName));
+
+            results.add(result);
+            BenchmarkUtils.printResults(result);
+        }
+
+        if (commandLine.hasOption(OUTPUT_FILE_OPTION.getLongOpt())) {

Review comment:
       It would provide user better experience by also printing benchmark result summary to the stdout when the `output-file` is not specified?

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/data/DenseVectorArrayGenerator.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.flink.ml.benchmark.data;
+
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.ml.common.datastream.TableUtils;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.util.NumberSequenceIterator;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+
+/** Class that generates a table containing arrays of dense vectors. */

Review comment:
       It is not obvious from the Java doc whether the element type is `arrays of dense vectors` of `array of dense vectors`.
   
   Also note that the Java doc for `Model` typically starts with `A Mode ...`.
   
   How about changing the doc to `A DataGenerator which creates a table of DenseVector array`?
   
   Same for other data generators.

##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/common/param/HasOutputCols.java
##########
@@ -19,15 +19,13 @@
 package org.apache.flink.ml.common.param;
 
 import org.apache.flink.ml.param.Param;
-import org.apache.flink.ml.param.ParamValidators;
 import org.apache.flink.ml.param.StringArrayParam;
 import org.apache.flink.ml.param.WithParams;
 
 /** Interface for the shared outputCols param. */
 public interface HasOutputCols<T> extends WithParams<T> {
     Param<String[]> OUTPUT_COLS =
-            new StringArrayParam(
-                    "outputCols", "Output column names.", null, ParamValidators.nonEmptyArray());
+            new StringArrayParam("outputCols", "Output column names.", new String[0]);

Review comment:
       Why do we make this change? Is the semantic still intuitive for all classes that use this parameter?

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/clustering/kmeans/KMeansModelDataGenerator.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.flink.ml.benchmark.clustering.kmeans;

Review comment:
       Since this is also data generator, would it be better to move this file to be under the directory `benchmark/data`.
   
   Now that since we expect 0-2 algorithm-specific data generator for each algorithm, we probably don't need the `kmeans` on the file path.
   
   How about the following directory structure:
   - Put re-usable data generator under `benchmark/data/common`
   - Put algorithm-specific data generator under `benchmark/data`
   - I am good with further divide the algorithm-specific data generator into sub-folders according to `clustering`, `classification` etc. to reduce the number of files in a folder.

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/CountingAndDiscardingSink.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.api.common.accumulators.LongCounter;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+
+/**
+ * A stream sink that counts the number of all elements. The counting result is stored in an {@link
+ * org.apache.flink.api.common.accumulators.Accumulator} specified by {@link
+ * CountingAndDiscardingSink#COUNTER_NAME} and can be acquired by {@link
+ * org.apache.flink.api.common.JobExecutionResult#getAccumulatorResult(String)}.
+ *
+ * @param <T> The type of elements received by the sink.
+ */
+class CountingAndDiscardingSink<T> extends RichSinkFunction<T> {

Review comment:
       Since this class is only used in `BenchmarkUtils`, would it be simpler to keep it as a private static class in `BenchmarkUtils`?

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/data/DataGenerator.java
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.flink.ml.benchmark.data;
+
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+
+/** Interface for generating data as table arrays. */
+public interface DataGenerator<T extends DataGenerator<T>> extends CommonDataGeneratorParams<T> {

Review comment:
       Model data generators typically won't need `numValues` and `columnNames`. It might be confusing to have `numValues` for model generator.
   
   Maybe we should create `InputDataGenerator` as a subclass of `DataGenerator`. And we can define `numValues` and `columnNames` parameters directly in the `InputDataGenerator`.
   
   What do you think?




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r840329681



##########
File path: flink-ml-core/src/main/java/org/apache/flink/ml/util/ReadWriteUtils.java
##########
@@ -94,25 +95,29 @@
      */
     public static void saveMetadata(Stage<?> stage, String path, Map<String, ?> extraMetadata)
             throws IOException {
-        // Creates parent directories if not already created.
-        FileSystem fs = mkdirs(path);
-
         Map<String, Object> metadata = new HashMap<>(extraMetadata);
         metadata.put("className", stage.getClass().getName());
         metadata.put("timestamp", System.currentTimeMillis());
         metadata.put("paramMap", jsonEncode(stage.getParamMap()));
         // TODO: add version in the metadata.
         String metadataStr = OBJECT_MAPPER.writeValueAsString(metadata);
 
-        Path metadataPath = new Path(path, "metadata");
-        if (fs.exists(metadataPath)) {
-            throw new IOException("File " + metadataPath + " already exists.");
+        saveToFile(new Path(path, "metadata"), metadataStr);
+    }
+
+    /** Saves a given string to the specified file. */
+    public static void saveToFile(Path path, String content) throws IOException {
+        // Creates parent directories if not already created.
+        FileSystem fs = mkdirs(path.getParent().toString());
+
+        if (fs.exists(path)) {
+            throw new IOException("File " + path + " already exists.");

Review comment:
       Shall we just overwrite the file when saving the benchmark results, or also overwrite the files when saving Stages?




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
lindong28 commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r841071352



##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,172 @@
+# Flink ML Benchmark Getting Started
+
+This document provides instructions on how to run benchmarks on Flink ML's
+stages in a Linux/MacOS environment.
+
+## Prerequisites
+
+### Install Flink
+
+Please make sure Flink 1.14 or higher version has been installed in your local
+environment. You can refer to the [local
+installation](https://nightlies.apache.org/flink/flink-docs-master/docs/try-flink/local_installation/)
+instruction on Flink's document website for how to achieve this.
+
+### Set Up Flink Environment Variables
+
+After having installed Flink, please register `$FLINK_HOME` as an environment
+variable into your local environment. For example, suppose you have downloaded
+Flink 1.14.0 and placed Flink's binary folder under `/usr/local/`, then you need
+to run the following command:
+
+```bash
+export FLINK_HOME=`/usr/local/flink-1.14.0`
+```
+
+Then please run the following command. If this command returns 1.14.0 or a
+higher version, then it means that the required Flink environment has been
+successfully installed and registered in your local environment.
+
+```bash
+$FLINK_HOME/bin/flink --version
+```
+
+### Acquire Flink ML Binary Distribution

Review comment:
       Sounds good.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r840189932



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/generator/GeneratorUtils.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.flink.ml.benchmark.generator;
+
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.linalg.Vectors;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.util.NumberSequenceIterator;
+
+import java.util.Random;
+
+/** Utility methods to generate data for benchmarks. */
+public class GeneratorUtils {
+    /**
+     * Generates random continuous vectors.
+     *
+     * @param env The stream execution environment.
+     * @param numData Number of examples to generate in total.
+     * @param seed The seed to generate seed on each partition.
+     * @param dims Dimension of the vectors to be generated.
+     * @return The generated vector stream.
+     */
+    public static DataStream<DenseVector> generateRandomContinuousVectorStream(
+            StreamExecutionEnvironment env, long numData, long seed, int dims) {
+        return env.fromParallelCollection(
+                        new NumberSequenceIterator(1L, numData), BasicTypeInfo.LONG_TYPE_INFO)
+                .map(new GenerateRandomContinuousVectorFunction(seed, dims));
+    }
+
+    private static class GenerateRandomContinuousVectorFunction
+            extends RichMapFunction<Long, DenseVector> {
+        private final int dims;
+        private final long initSeed;
+        private Random random;
+
+        private GenerateRandomContinuousVectorFunction(long initSeed, int dims) {
+            this.dims = dims;
+            this.initSeed = initSeed;
+        }
+
+        @Override
+        public void open(Configuration parameters) throws Exception {
+            super.open(parameters);
+            int index = getRuntimeContext().getIndexOfThisSubtask();
+            random = new Random(Tuple2.of(initSeed, index).hashCode());
+        }
+
+        @Override
+        public DenseVector map(Long value) {
+            double[] values = new double[dims];
+            for (int i = 0; i < dims; i++) {
+                values[i] = random.nextDouble();
+            }
+            return Vectors.dense(values);
+        }
+    }
+
+    /**
+     * Generates random continuous vector arrays.
+     *
+     * @param env The stream execution environment.
+     * @param numData Number of examples to generate in total.
+     * @param arraySize Size of the vector array.
+     * @param seed The seed to generate seed on each partition.
+     * @param dims Dimension of the vectors to be generated.
+     * @return The generated vector stream.
+     */
+    public static DataStream<DenseVector[]> generateRandomContinuousVectorArrayStream(

Review comment:
       Given that we are introducing classes like `DenseVectorArrayGenerator`, `GeneratorUtils` would be removed so this comment is not applicable now.

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/generator/GeneratorUtils.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.flink.ml.benchmark.generator;
+
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.linalg.Vectors;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.util.NumberSequenceIterator;
+
+import java.util.Random;
+
+/** Utility methods to generate data for benchmarks. */
+public class GeneratorUtils {

Review comment:
       Given that we are introducing classes like `DenseVectorArrayGenerator`, `GeneratorUtils` would be removed so this comment is not applicable now.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r837426927



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/BenchmarkUtils.java
##########
@@ -0,0 +1,233 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.ml.api.AlgoOperator;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.api.Model;
+import org.apache.flink.ml.api.Stage;
+import org.apache.flink.ml.benchmark.generator.DataGenerator;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.WithParams;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.bridge.java.internal.StreamTableEnvironmentImpl;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.Preconditions;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+/** Utility methods for benchmarks. */
+@SuppressWarnings("unchecked")
+public class BenchmarkUtils {
+    private static final String benchmarkNamePattern = "^[A-Za-z0-9][A-Za-z0-9_\\-]*$";
+
+    /**
+     * Loads benchmark paramMaps from the provided json map.
+     *
+     * @return A map whose key is the names of the loaded benchmarks, value is the parameters of the
+     *     benchmarks.
+     */
+    public static Map<String, Map<String, ?>> parseBenchmarkParams(Map<String, ?> jsonMap) {
+        Preconditions.checkArgument(
+                jsonMap.containsKey("_version") && jsonMap.get("_version").equals(1));
+
+        Map<String, Map<String, ?>> result = new HashMap<>();
+        for (String key : jsonMap.keySet()) {
+            if (!isValidBenchmarkName(key)) {
+                continue;
+            }
+            result.put(key, (Map<String, ?>) jsonMap.get(key));
+        }
+        return result;
+    }
+
+    /**
+     * Checks whether a string is a valid benchmark name.
+     *
+     * <p>A valid benchmark name should only contain English letters, numbers, hyphens (-) and
+     * underscores (_). The name should not start with a hyphen or underscore.
+     */
+    public static boolean isValidBenchmarkName(String name) {
+        return Pattern.matches(benchmarkNamePattern, name);
+    }
+
+    /**
+     * Instantiates a benchmark from its parameter map and executes the benchmark in the provided
+     * environment.
+     *
+     * @return Results of the executed benchmark.
+     */
+    public static BenchmarkResult runBenchmark(
+            String name, StreamTableEnvironment tEnv, Map<String, ?> benchmarkParamsMap)
+            throws Exception {
+        Stage<?> stage =
+                (Stage<?>) instantiateWithParams((Map<String, ?>) benchmarkParamsMap.get("stage"));
+
+        BenchmarkResult result;
+        if (benchmarkParamsMap.size() == 2) {
+            DataGenerator<?> inputsGenerator =
+                    (DataGenerator<?>)
+                            instantiateWithParams(
+                                    (Map<String, ?>) benchmarkParamsMap.get("inputs"));
+            result = runBenchmark(name, tEnv, stage, inputsGenerator);
+        } else if (benchmarkParamsMap.size() == 3 && stage instanceof Model) {
+            DataGenerator<?> inputsGenerator =
+                    (DataGenerator<?>)
+                            instantiateWithParams(
+                                    (Map<String, ?>) benchmarkParamsMap.get("inputs"));
+            DataGenerator<?> modelDataGenerator =
+                    (DataGenerator<?>)
+                            instantiateWithParams(
+                                    (Map<String, ?>) benchmarkParamsMap.get("modelData"));
+            result =
+                    runBenchmark(name, tEnv, (Model<?>) stage, modelDataGenerator, inputsGenerator);
+        } else {
+            throw new IllegalArgumentException(
+                    "Unsupported json map with keys " + benchmarkParamsMap.keySet());
+        }
+
+        return result;
+    }
+
+    /**
+     * Executes a benchmark from a stage and an inputsGenerator in the provided environment.
+     *
+     * @return Results of the executed benchmark.
+     */
+    public static BenchmarkResult runBenchmark(
+            String name,
+            StreamTableEnvironment tEnv,
+            Stage<?> stage,
+            DataGenerator<?> inputsGenerator)
+            throws Exception {
+        Table[] inputTables = inputsGenerator.getData(tEnv);
+
+        Table[] outputTables;
+        if (stage instanceof Estimator) {
+            outputTables = ((Estimator<?, ?>) stage).fit(inputTables).getModelData();
+        } else if (stage instanceof AlgoOperator) {
+            outputTables = ((AlgoOperator<?>) stage).transform(inputTables);
+        } else {
+            throw new IllegalArgumentException("Unsupported Stage class " + stage.getClass());
+        }
+
+        for (Table table : outputTables) {
+            tEnv.toDataStream(table).addSink(new DiscardingSink<>());
+        }
+
+        StreamExecutionEnvironment env = ((StreamTableEnvironmentImpl) tEnv).execEnv();
+        JobExecutionResult executionResult = env.execute();
+
+        BenchmarkResult result = new BenchmarkResult();
+        result.name = name;
+        result.executionTimeMillis = (double) executionResult.getNetRuntime(TimeUnit.MILLISECONDS);

Review comment:
       I can make `DataGenerator implements GeneratorParams`, which means each generator must have a `getNumData()` method. That should be a simpler solution.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r838146133



##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,261 @@
+# Flink ML Benchmark Guideline
+
+This document provides instructions about how to run benchmarks on Flink ML's
+stages.
+
+## Write Benchmark Programs
+
+### Add Maven Dependencies
+
+In order to write Flink ML's java benchmark programs, first make sure that the
+following dependencies have been added to your maven project's `pom.xml`.
+
+```xml
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-core_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-iteration_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-lib_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-benchmark_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>statefun-flink-core</artifactId>
+  <version>3.1.0</version>
+  <exclusions>
+    <exclusion>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-streaming-java_2.12</artifactId>
+    </exclusion>
+  </exclusions>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-table-api-java-bridge_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-table-planner_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-clients_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+```
+
+### Write Java Program
+
+Then you can write a program as follows to run benchmark on Flink ML stages. The
+example code below tests the performance of Flink ML's KMeans algorithm, with
+the default configuration parameters used.
+
+```java
+public class Main {
+    public static void main(String[] args) throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        KMeans kMeans = new KMeans();
+        KMeansInputsGenerator inputsGenerator = new KMeansInputsGenerator();
+
+        BenchmarkResult result =
+                BenchmarkUtils.runBenchmark("exampleBenchmark", tEnv, kMeans, inputsGenerator);
+
+        BenchmarkUtils.printResult(result);
+    }
+}
+```
+
+### Execute Benchmark Program
+
+After executing the `main()` method above, you will see benchmark results
+printed out in your terminal. An example of the printed content is as follows.
+
+```
+Benchmark Name: exampleBenchmark
+Total Execution Time(ms): 828.0
+```
+
+### Configure Benchmark Parameters
+
+If you want to run benchmark on customed configuration parameters, you can set
+them with Flink ML's `WithParams` API as follows.
+
+```java
+KMeans kMeans = new KMeans()
+  .setK(5)
+  .setMaxIter(50);
+KMeansInputsGenerator inputsGenerator = new KMeansInputsGenerator()
+  .setDims(3)
+  .setDataSize(10000);
+```
+
+## Execute Benchmark through Command-Line Interface (CLI)
+
+You can also configure and execute benchmarks through Command-Line Interface
+(CLI) without writing java programs.
+
+### Prerequisites
+
+Before using Flink ML's CLI, make sure you have installed Flink 1.14 in your
+local environment, and that you have started a Flink cluster locally. If not,
+you can start a standalone session with the following command.
+
+```bash
+$ start-cluster

Review comment:
       OK. I'll add the instructions.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r840405050



##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,169 @@
+# Flink ML Benchmark Getting Started
+
+This document provides instructions about how to run benchmarks on Flink ML's
+stages in a Linux/MacOS environment.
+
+## Prerequisites
+
+### Installing Flink
+
+Please make sure Flink 1.14 or higher version has been installed in your local
+environment. You can refer to the [local
+installation](https://nightlies.apache.org/flink/flink-docs-master/docs/try-flink/local_installation/)
+instruction on Flink's document website for how to achieve this.
+
+### Setting Up Flink Environment Variables
+
+After having installed Flink, please register `$FLINK_HOME` as an environment
+variable into your local environment, and add `$FLINK_HOME` into your `$PATH`
+variable. This can be completed by running the following commands in the Flink's
+folder.
+
+```bash
+export FLINK_HOME=`pwd`

Review comment:
       It is difficult as we don't know user's environment and the Flink version users have installed. I tried using an example cmd, but it still means that users cannot directly copy and paste the commands without reading the instruction carefully. Please check if this is a proper implementation.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
lindong28 commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r840990551



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/BenchmarkResult.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/** The result of executing a benchmark. */
+public class BenchmarkResult {
+    /** The name of the benchmark. */
+    public String name;

Review comment:
       hmm... in the future when we add `latencyP99Ms`, we can just add a new constructor and update the existing constructors to provide a default value for `latencyP99Ms`. We want need to change all existing usages, right?
   
   Note that if we need to change existing usages, it should be because we want to provide a valid non-default value for `latencyP99Ms`. We will also need to do these changes if we use `Builder` in this case.
   
   The usages of multiple constructors is a common pattern across Flink to handle scenarios like this.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
lindong28 commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r840998428



##########
File path: flink-ml-core/src/main/java/org/apache/flink/ml/util/ReadWriteUtils.java
##########
@@ -94,25 +95,29 @@
      */
     public static void saveMetadata(Stage<?> stage, String path, Map<String, ?> extraMetadata)
             throws IOException {
-        // Creates parent directories if not already created.
-        FileSystem fs = mkdirs(path);
-
         Map<String, Object> metadata = new HashMap<>(extraMetadata);
         metadata.put("className", stage.getClass().getName());
         metadata.put("timestamp", System.currentTimeMillis());
         metadata.put("paramMap", jsonEncode(stage.getParamMap()));
         // TODO: add version in the metadata.
         String metadataStr = OBJECT_MAPPER.writeValueAsString(metadata);
 
-        Path metadataPath = new Path(path, "metadata");
-        if (fs.exists(metadataPath)) {
-            throw new IOException("File " + metadataPath + " already exists.");
+        saveToFile(new Path(path, "metadata"), metadataStr);
+    }
+
+    /** Saves a given string to the specified file. */
+    public static void saveToFile(Path path, String content) throws IOException {
+        // Creates parent directories if not already created.
+        FileSystem fs = mkdirs(path.getParent().toString());
+
+        if (fs.exists(path)) {
+            throw new IOException("File " + path + " already exists.");

Review comment:
       Spark ML don't overwrite model data files by default. I think we can follow the same practice for model data files.
   
   Based on my past experience, I believe we typically allow overwriting output files specified on the command line tool. But I don't have an example (or counter-example) currently.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r841037610



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/data/DataGenerator.java
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.flink.ml.benchmark.data;
+
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+
+/** Interface for generating data as table arrays. */
+public interface DataGenerator<T extends DataGenerator<T>> extends CommonDataGeneratorParams<T> {

Review comment:
       I think it is a good idea to introduce `InputDataGenerator`, but I think we might still need `InputDataGeneratorParams` for `numValues` and `colNames`, as it could make inheritance relationship clearer. Similarly, I'll also add interfaces like `DenseVectorGeneratorParams` and `KMeansModelDataGeneratorParams`.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r841038991



##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,172 @@
+# Flink ML Benchmark Getting Started
+
+This document provides instructions on how to run benchmarks on Flink ML's
+stages in a Linux/MacOS environment.
+
+## Prerequisites
+
+### Install Flink
+
+Please make sure Flink 1.14 or higher version has been installed in your local
+environment. You can refer to the [local
+installation](https://nightlies.apache.org/flink/flink-docs-master/docs/try-flink/local_installation/)
+instruction on Flink's document website for how to achieve this.
+
+### Set Up Flink Environment Variables
+
+After having installed Flink, please register `$FLINK_HOME` as an environment
+variable into your local environment. For example, suppose you have downloaded
+Flink 1.14.0 and placed Flink's binary folder under `/usr/local/`, then you need
+to run the following command:
+
+```bash
+export FLINK_HOME=`/usr/local/flink-1.14.0`
+```
+
+Then please run the following command. If this command returns 1.14.0 or a
+higher version, then it means that the required Flink environment has been
+successfully installed and registered in your local environment.
+
+```bash
+$FLINK_HOME/bin/flink --version
+```
+
+### Acquire Flink ML Binary Distribution

Review comment:
       Yes it is a better name. I had used `Acquire Flink ML Binary Distribution` because in the long run the more common practice would be to download a binary release instead of building the binary locally. I'll add a TODO here to remind us to add instructions about where to download the binary package.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
lindong28 commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r841073165



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/Benchmark.java
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.DefaultParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** 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 String SHELL_SCRIPT = "flink-ml-benchmark.sh";

Review comment:
       Since this variable is only used once in `printHelp()`. Would it be simpler to remove this variable and simply do the following in  `printHelp()`?
   
   ```
   System.out.println("./flink-ml-benchmark.sh <config-file-path> [OPTIONS]\n");
   ```

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/data/DataGenerator.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.flink.ml.benchmark.data;
+
+import org.apache.flink.ml.common.param.HasSeed;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+
+/** Interface for generating data as table arrays. */
+public interface DataGenerator<T extends DataGenerator<T>> extends HasSeed<T> {

Review comment:
       Should this interface extends `DataGeneratorParams`, similar to how `InputDataGenerator` extends `InputDataGeneratorParams`?




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
lindong28 commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r841078416



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/data/DataGenerator.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.flink.ml.benchmark.data;
+
+import org.apache.flink.ml.common.param.HasSeed;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+
+/** Interface for generating data as table arrays. */
+public interface DataGenerator<T extends DataGenerator<T>> extends HasSeed<T> {

Review comment:
       After thinking about this more, I think it might be simpler to just remove `DataGeneratorParams` and define parameters directly in `DataGenerator`.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
lindong28 commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r841074926



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/data/InputDataGeneratorParams.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.flink.ml.benchmark.data;
+
+import org.apache.flink.ml.param.LongParam;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+import org.apache.flink.ml.param.StringArrayParam;
+
+/** Interface for the input data generator params. */
+public interface InputDataGeneratorParams<T> extends DataGeneratorParams<T> {

Review comment:
       Would it be simpler to remove this class and define parameters directly in `InputDataGenerator`?
   
   By doing this, all subclasses of `InputDataGenerator`, such as `DenseVectorArrayGenerator` and `DenseVectorGenerator`, would inherit those parameters.
   
   Note that we have dedicated parameter classes for algorithms because an algorithm typically have a pair of estimator and model, where estimator and model share some but not call parameters. Data generators do not have have this problem.
   
   Similarly, it might be simpler to remove the generator-specific parameter classes such as `DenseVectorArrayGeneratorParams` and `DenseVectorGeneratorParams`. 
   
   We can still define non-generator-specific parameters in dedicated classes such as `HasArraySize`.
   

##########
File path: flink-ml-benchmark/src/test/java/org/apache/flink/ml/benchmark/BenchmarkTest.java
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.ml.benchmark.data.InputDataGenerator;
+import org.apache.flink.ml.benchmark.data.common.DenseVectorGenerator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+import org.apache.flink.ml.clustering.kmeans.KMeansModel;
+import org.apache.flink.ml.param.WithParams;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.test.util.AbstractTestBase;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.InputStream;
+import java.io.PrintStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.ml.util.ReadWriteUtils.OBJECT_MAPPER;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/** Tests benchmarks. */
+@SuppressWarnings("unchecked")
+public class BenchmarkTest extends AbstractTestBase {
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+
+    private static final String exampleConfigFile = "kmeansmodel-benchmark.json";
+    private static final String expectedBenchmarkName = "KMeansModel-1";
+
+    private final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+    private final PrintStream originalOut = System.out;
+
+    @Before
+    public void before() {
+        System.setOut(new PrintStream(outputStream));
+    }
+
+    @After
+    public void after() {
+        System.setOut(originalOut);
+        outputStream.reset();
+    }
+
+    @Test
+    public void testCreateAndExecute() throws Exception {

Review comment:
       Would it be more intuitive to rename this test as `testRunBenchmark()`?

##########
File path: flink-ml-benchmark/src/test/java/org/apache/flink/ml/benchmark/BenchmarkTest.java
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.ml.benchmark.data.InputDataGenerator;
+import org.apache.flink.ml.benchmark.data.common.DenseVectorGenerator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+import org.apache.flink.ml.clustering.kmeans.KMeansModel;
+import org.apache.flink.ml.param.WithParams;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.test.util.AbstractTestBase;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.InputStream;
+import java.io.PrintStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.ml.util.ReadWriteUtils.OBJECT_MAPPER;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/** Tests benchmarks. */
+@SuppressWarnings("unchecked")
+public class BenchmarkTest extends AbstractTestBase {
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+
+    private static final String exampleConfigFile = "kmeansmodel-benchmark.json";
+    private static final String expectedBenchmarkName = "KMeansModel-1";
+
+    private final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+    private final PrintStream originalOut = System.out;
+
+    @Before
+    public void before() {
+        System.setOut(new PrintStream(outputStream));
+    }
+
+    @After
+    public void after() {
+        System.setOut(originalOut);
+        outputStream.reset();
+    }
+
+    @Test
+    public void testCreateAndExecute() throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        KMeans kMeans = new KMeans().setK(5).setFeaturesCol("test_feature");
+        InputDataGenerator<?> inputsGenerator =
+                new DenseVectorGenerator()
+                        .setColNames("test_feature")
+                        .setNumValues(1000)
+                        .setVectorDim(10);
+
+        BenchmarkResult result =
+                BenchmarkUtils.runBenchmark(tEnv, "testBenchmarkName", kMeans, inputsGenerator);
+
+        BenchmarkUtils.printResults(result);

Review comment:
       Do you see any existing example where we check the value by printing it to stdout and check the printed stream? I find it pretty uncommon to do this, which suggests that it is not following the best practice.
   
   Would it be simpler to check the value of `result` directly, instead of printing its to stdout and checking the printed String?

##########
File path: flink-ml-benchmark/src/test/java/org/apache/flink/ml/benchmark/BenchmarkTest.java
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.ml.benchmark.data.InputDataGenerator;
+import org.apache.flink.ml.benchmark.data.common.DenseVectorGenerator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+import org.apache.flink.ml.clustering.kmeans.KMeansModel;
+import org.apache.flink.ml.param.WithParams;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.test.util.AbstractTestBase;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.InputStream;
+import java.io.PrintStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.ml.util.ReadWriteUtils.OBJECT_MAPPER;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/** Tests benchmarks. */
+@SuppressWarnings("unchecked")
+public class BenchmarkTest extends AbstractTestBase {
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+
+    private static final String exampleConfigFile = "kmeansmodel-benchmark.json";
+    private static final String expectedBenchmarkName = "KMeansModel-1";
+
+    private final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+    private final PrintStream originalOut = System.out;
+
+    @Before
+    public void before() {
+        System.setOut(new PrintStream(outputStream));
+    }
+
+    @After
+    public void after() {
+        System.setOut(originalOut);
+        outputStream.reset();
+    }
+
+    @Test
+    public void testCreateAndExecute() throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        KMeans kMeans = new KMeans().setK(5).setFeaturesCol("test_feature");
+        InputDataGenerator<?> inputsGenerator =
+                new DenseVectorGenerator()
+                        .setColNames("test_feature")
+                        .setNumValues(1000)
+                        .setVectorDim(10);
+
+        BenchmarkResult result =
+                BenchmarkUtils.runBenchmark(tEnv, "testBenchmarkName", kMeans, inputsGenerator);
+
+        BenchmarkUtils.printResults(result);
+        assertTrue(outputStream.toString().contains("testBenchmarkName"));
+        assertTrue(outputStream.toString().contains(result.totalTimeMs.toString()));
+    }
+
+    @Test
+    public void testLoadAndSave() throws Exception {

Review comment:
       I am hoping we can simply the test code here and make it more readable.
   
   The following logical steps are executed by `flink-ml-benchmark.sh`:
   - Parse command line argument
   - Parse the json config file into `Map<String, ?>`
   - Enumerate the `Map<String, ?>` and call `runBenchmark()` to get a list of BenchmarkResults.
   - Output the BenchmarkResults to file or stdout.
   
   Would it make the code more readable to update Benchmark.java and make its functions logically similar the the steps described above? For example, we can have functions such as `Map<String, ?> parseJsonFile(String path)`.
   
   And in general the best practice to test `flink-ml-benchmark.sh` is to test each step separately by testing the corresponding function (e.g. `runBenchmark`, `parseJsonFile`).
   
   Also, we might not need to explicitly test the step that converts BenchmarkResult to a string, as the corresponding function is quite straightforward and unlikely to have bug.

##########
File path: flink-ml-benchmark/src/test/java/org/apache/flink/ml/benchmark/BenchmarkTest.java
##########
@@ -0,0 +1,185 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.ml.benchmark.data.InputDataGenerator;
+import org.apache.flink.ml.benchmark.data.common.DenseVectorGenerator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+import org.apache.flink.ml.clustering.kmeans.KMeansModel;
+import org.apache.flink.ml.param.WithParams;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.test.util.AbstractTestBase;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.InputStream;
+import java.io.PrintStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.ml.util.ReadWriteUtils.OBJECT_MAPPER;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/** Tests benchmarks. */
+@SuppressWarnings("unchecked")
+public class BenchmarkTest extends AbstractTestBase {
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+
+    private static final String exampleConfigFile = "kmeansmodel-benchmark.json";
+    private static final String expectedBenchmarkName = "KMeansModel-1";
+
+    private final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+    private final PrintStream originalOut = System.out;
+
+    @Before
+    public void before() {
+        System.setOut(new PrintStream(outputStream));
+    }
+
+    @After
+    public void after() {
+        System.setOut(originalOut);
+        outputStream.reset();
+    }
+
+    @Test
+    public void testCreateAndExecute() throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        KMeans kMeans = new KMeans().setK(5).setFeaturesCol("test_feature");
+        InputDataGenerator<?> inputsGenerator =
+                new DenseVectorGenerator()
+                        .setColNames("test_feature")
+                        .setNumValues(1000)
+                        .setVectorDim(10);
+
+        BenchmarkResult result =
+                BenchmarkUtils.runBenchmark(tEnv, "testBenchmarkName", kMeans, inputsGenerator);
+
+        BenchmarkUtils.printResults(result);

Review comment:
       Do you see any existing example where we check the value by printing it to stdout and check the printed stream? I find it pretty uncommon to do this, which suggests that it is not following the best practice.
   
   Would it be simpler to check the value of `result` directly, instead of printing its to stdout and checking the printed String? 




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r840186693



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/clustering/kmeans/KMeansInputsGenerator.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.flink.ml.benchmark.clustering.kmeans;
+
+import org.apache.flink.ml.benchmark.generator.DataGenerator;
+import org.apache.flink.ml.benchmark.generator.GeneratorUtils;
+import org.apache.flink.ml.clustering.kmeans.KMeansParams;
+import org.apache.flink.ml.common.datastream.TableUtils;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Class that generates table arrays containing inputs for {@link
+ * org.apache.flink.ml.clustering.kmeans.KMeans} and {@link
+ * org.apache.flink.ml.clustering.kmeans.KMeansModel}.
+ */
+public class KMeansInputsGenerator

Review comment:
       I agree. In this case I'll remove `GeneratorUtils`, as its current functions would be replaced by `DenseVectorGenerator` and `DenseVectorArrayGenerator`.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r838059470



##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,261 @@
+# Flink ML Benchmark Guideline
+
+This document provides instructions about how to run benchmarks on Flink ML's
+stages.
+
+## Write Benchmark Programs
+
+### Add Maven Dependencies
+
+In order to write Flink ML's java benchmark programs, first make sure that the
+following dependencies have been added to your maven project's `pom.xml`.
+
+```xml
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-core_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-iteration_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-lib_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-benchmark_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>statefun-flink-core</artifactId>
+  <version>3.1.0</version>
+  <exclusions>
+    <exclusion>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-streaming-java_2.12</artifactId>
+    </exclusion>
+  </exclusions>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-table-api-java-bridge_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-table-planner_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-clients_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+```
+
+### Write Java Program
+
+Then you can write a program as follows to run benchmark on Flink ML stages. The
+example code below tests the performance of Flink ML's KMeans algorithm, with
+the default configuration parameters used.
+
+```java
+public class Main {
+    public static void main(String[] args) throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        KMeans kMeans = new KMeans();
+        KMeansInputsGenerator inputsGenerator = new KMeansInputsGenerator();
+
+        BenchmarkResult result =
+                BenchmarkUtils.runBenchmark("exampleBenchmark", tEnv, kMeans, inputsGenerator);
+
+        BenchmarkUtils.printResult(result);
+    }
+}
+```
+
+### Execute Benchmark Program
+
+After executing the `main()` method above, you will see benchmark results
+printed out in your terminal. An example of the printed content is as follows.
+
+```
+Benchmark Name: exampleBenchmark
+Total Execution Time(ms): 828.0
+```
+
+### Configure Benchmark Parameters
+
+If you want to run benchmark on customed configuration parameters, you can set
+them with Flink ML's `WithParams` API as follows.
+
+```java
+KMeans kMeans = new KMeans()
+  .setK(5)
+  .setMaxIter(50);
+KMeansInputsGenerator inputsGenerator = new KMeansInputsGenerator()
+  .setDims(3)
+  .setDataSize(10000);
+```
+
+## Execute Benchmark through Command-Line Interface (CLI)
+
+You can also configure and execute benchmarks through Command-Line Interface
+(CLI) without writing java programs.
+
+### Prerequisites
+
+Before using Flink ML's CLI, make sure you have installed Flink 1.14 in your
+local environment, and that you have started a Flink cluster locally. If not,
+you can start a standalone session with the following command.
+
+```bash
+$ start-cluster
+```
+
+In order to use Flink ML's CLI you need to have the latest binary distribution
+of Flink ML. You can acquire the distribution by building Flink ML's source code
+locally, which means to execute the following command in Flink ML repository's
+root directory.
+
+```bash
+$ mvn clean package -DskipTests
+```
+
+After executing the command above, you will be able to find the binary
+distribution under
+`./flink-ml-dist/target/flink-ml-<version>-bin/flink-ml-<version>/`.

Review comment:
       I have not been able to find a proper solution to use variables in markdown files. I tried using html and javascript, but that only works when I export markdown to html file. Besides, javascript scripts are usually now allowed to access local files for security reasons, which means it can not read the version information automatically from `pom.xml`, such that we have to modify the version variable manually.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
lindong28 commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r840995675



##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,172 @@
+# Flink ML Benchmark Getting Started
+
+This document provides instructions on how to run benchmarks on Flink ML's
+stages in a Linux/MacOS environment.
+
+## Prerequisites
+
+### Install Flink
+
+Please make sure Flink 1.14 or higher version has been installed in your local
+environment. You can refer to the [local
+installation](https://nightlies.apache.org/flink/flink-docs-master/docs/try-flink/local_installation/)
+instruction on Flink's document website for how to achieve this.
+
+### Set Up Flink Environment Variables
+
+After having installed Flink, please register `$FLINK_HOME` as an environment
+variable into your local environment. For example, suppose you have downloaded
+Flink 1.14.0 and placed Flink's binary folder under `/usr/local/`, then you need
+to run the following command:
+
+```bash
+export FLINK_HOME=`/usr/local/flink-1.14.0`
+```
+
+Then please run the following command. If this command returns 1.14.0 or a
+higher version, then it means that the required Flink environment has been
+successfully installed and registered in your local environment.
+
+```bash
+$FLINK_HOME/bin/flink --version
+```
+
+### Acquire Flink ML Binary Distribution
+
+In order to use Flink ML's CLI you need to have the latest binary distribution
+of Flink ML. You can acquire the distribution by building Flink ML's source code
+locally, which means to execute the following command in Flink ML repository's
+root directory.
+
+```bash
+mvn clean package -DskipTests
+cd ./flink-ml-dist/target/flink-ml-*-bin/flink-ml*/

Review comment:
       How about adding the `path_to_flink_ml` like below:
   
   ```
   cd ${path_to_flink_ml}/flink-ml-dist/target/flink-ml-*-bin/flink-ml*/
   ```

##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,261 @@
+# Flink ML Benchmark Guideline
+
+This document provides instructions about how to run benchmarks on Flink ML's
+stages.
+
+## Write Benchmark Programs
+
+### Add Maven Dependencies
+
+In order to write Flink ML's java benchmark programs, first make sure that the
+following dependencies have been added to your maven project's `pom.xml`.
+
+```xml
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-core_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-iteration_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-lib_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-benchmark_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>statefun-flink-core</artifactId>
+  <version>3.1.0</version>
+  <exclusions>
+    <exclusion>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-streaming-java_2.12</artifactId>
+    </exclusion>
+  </exclusions>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-table-api-java-bridge_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-table-planner_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-clients_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+```
+
+### Write Java Program
+
+Then you can write a program as follows to run benchmark on Flink ML stages. The
+example code below tests the performance of Flink ML's KMeans algorithm, with
+the default configuration parameters used.
+
+```java
+public class Main {
+    public static void main(String[] args) throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        KMeans kMeans = new KMeans();
+        KMeansInputsGenerator inputsGenerator = new KMeansInputsGenerator();
+
+        BenchmarkResult result =
+                BenchmarkUtils.runBenchmark("exampleBenchmark", tEnv, kMeans, inputsGenerator);
+
+        BenchmarkUtils.printResult(result);
+    }
+}
+```
+
+### Execute Benchmark Program
+
+After executing the `main()` method above, you will see benchmark results
+printed out in your terminal. An example of the printed content is as follows.
+
+```
+Benchmark Name: exampleBenchmark
+Total Execution Time(ms): 828.0
+```
+
+### Configure Benchmark Parameters
+
+If you want to run benchmark on customed configuration parameters, you can set
+them with Flink ML's `WithParams` API as follows.
+
+```java
+KMeans kMeans = new KMeans()
+  .setK(5)
+  .setMaxIter(50);
+KMeansInputsGenerator inputsGenerator = new KMeansInputsGenerator()
+  .setDims(3)
+  .setDataSize(10000);
+```
+
+## Execute Benchmark through Command-Line Interface (CLI)
+
+You can also configure and execute benchmarks through Command-Line Interface
+(CLI) without writing java programs.
+
+### Prerequisites
+
+Before using Flink ML's CLI, make sure you have installed Flink 1.14 in your
+local environment, and that you have started a Flink cluster locally. If not,
+you can start a standalone session with the following command.
+
+```bash
+$ start-cluster
+```
+
+In order to use Flink ML's CLI you need to have the latest binary distribution
+of Flink ML. You can acquire the distribution by building Flink ML's source code
+locally, which means to execute the following command in Flink ML repository's
+root directory.
+
+```bash
+$ mvn clean package -DskipTests
+```
+
+After executing the command above, you will be able to find the binary
+distribution under
+`./flink-ml-dist/target/flink-ml-<version>-bin/flink-ml-<version>/`.

Review comment:
       Got it. Thanks for looking into this.

##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,172 @@
+# Flink ML Benchmark Getting Started
+
+This document provides instructions on how to run benchmarks on Flink ML's
+stages in a Linux/MacOS environment.
+
+## Prerequisites
+
+### Install Flink
+
+Please make sure Flink 1.14 or higher version has been installed in your local
+environment. You can refer to the [local
+installation](https://nightlies.apache.org/flink/flink-docs-master/docs/try-flink/local_installation/)
+instruction on Flink's document website for how to achieve this.
+
+### Set Up Flink Environment Variables
+
+After having installed Flink, please register `$FLINK_HOME` as an environment
+variable into your local environment. For example, suppose you have downloaded
+Flink 1.14.0 and placed Flink's binary folder under `/usr/local/`, then you need
+to run the following command:
+
+```bash
+export FLINK_HOME=`/usr/local/flink-1.14.0`
+```
+
+Then please run the following command. If this command returns 1.14.0 or a
+higher version, then it means that the required Flink environment has been
+successfully installed and registered in your local environment.
+
+```bash
+$FLINK_HOME/bin/flink --version
+```
+
+### Acquire Flink ML Binary Distribution

Review comment:
       How about `Build Flink ML library`

##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,172 @@
+# Flink ML Benchmark Getting Started
+
+This document provides instructions on how to run benchmarks on Flink ML's
+stages in a Linux/MacOS environment.
+
+## Prerequisites
+
+### Install Flink
+
+Please make sure Flink 1.14 or higher version has been installed in your local
+environment. You can refer to the [local
+installation](https://nightlies.apache.org/flink/flink-docs-master/docs/try-flink/local_installation/)
+instruction on Flink's document website for how to achieve this.
+
+### Set Up Flink Environment Variables
+
+After having installed Flink, please register `$FLINK_HOME` as an environment
+variable into your local environment. For example, suppose you have downloaded
+Flink 1.14.0 and placed Flink's binary folder under `/usr/local/`, then you need
+to run the following command:
+
+```bash
+export FLINK_HOME=`/usr/local/flink-1.14.0`
+```
+
+Then please run the following command. If this command returns 1.14.0 or a
+higher version, then it means that the required Flink environment has been
+successfully installed and registered in your local environment.
+
+```bash
+$FLINK_HOME/bin/flink --version
+```
+
+### Acquire Flink ML Binary Distribution
+
+In order to use Flink ML's CLI you need to have the latest binary distribution
+of Flink ML. You can acquire the distribution by building Flink ML's source code
+locally, which means to execute the following command in Flink ML repository's
+root directory.
+
+```bash
+mvn clean package -DskipTests
+cd ./flink-ml-dist/target/flink-ml-*-bin/flink-ml*/
+```
+
+### Start Flink Cluster
+
+Please start a Flink standalone session in your local environment with the
+following command.
+
+```bash
+$FLINK_HOME/bin/start-cluster.sh
+```
+
+You should be able to navigate to the web UI at
+[localhost:8081](http://localhost:8081/) to view the Flink dashboard and see
+that the cluster is up and running.
+
+## Run Benchmark Example
+
+In Flink ML's binary distribution's folder, execute the following command to run
+an example benchmark.
+
+```bash
+./bin/flink-ml-benchmark.sh ./examples/benchmark-example-conf.json --output-file ./output/results.json
+```
+
+You will notice that some Flink job is submitted to your Flink cluster, and the
+following information is printed out in your terminal. This means that you have
+successfully executed a benchmark on `KMeansModel`.
+
+```
+Creating fat jar containing all flink ml dependencies to be submitted.
+Job has been submitted with JobID a5d8868d808eecfb357eb904c961c3bf
+Program execution finished
+Job with JobID a5d8868d808eecfb357eb904c961c3bf has finished.
+Job Runtime: 897 ms
+Accumulator Results: 
+- numElements (java.lang.Long): 10000
+
+
+{
+  "name" : "KMeansModel-1",
+  "totalTimeMs" : 897.0,
+  "inputRecordNum" : 10000,
+  "inputThroughput" : 11148.272017837235,
+  "outputRecordNum" : 10000,
+  "outputThroughput" : 11148.272017837235
+}
+```
+
+The command above would save the results into `./output/results.json` as below.
+
+```json
+[ {
+  "name" : "KMeansModel-1",
+  "totalTimeMs" : 897.0,
+  "inputRecordNum" : 10000,
+  "inputThroughput" : 11148.272017837235,
+  "outputRecordNum" : 10000,
+  "outputThroughput" : 11148.272017837235
+} ]
+```
+
+## Customize Benchmark Configuration
+
+`flink-ml-benchmark.sh` parses benchmarks to be executed according to the input
+configuration file, like `./examples/benchmark-example-conf.json`. It can also
+parse your custom configuration file so long as it contains a JSON object in the
+following format.
+
+- The file should contain the following as the metadata of the JSON object.
+  - `"version"`: The version of the json format. Currently its value must be 1.
+- Keys in the JSON object, except `"version"`, are regarded as the names of the
+  benchmarks.
+- The value of each benchmark name should be a JSON object containing the
+  following keys.
+  - `"stage"`: The stage to be benchmarked.
+  - `"inputs"`: The input data of the stage to be benchmarked.
+  - `"modelData"`(Optional): The model data of the stage to be benchmarked, if
+    the stage is a `Model` and needs to have its model data explicitly set.
+- The value of `"stage"`, `"inputs"` or `"modelData"` should be a JSON object
+  containing the following keys.
+  - `"className"`: The full classpath of a `WithParams` subclass. For `"stage"`,
+    the class should be a subclass of `Stage`. For `"inputs"` or `"modelData"`,
+    the class should be a subclass of `DataGenerator`.
+  - `"paramMap"`: A JSON object containing the parameters related to the
+    specific `Stage` or `DataGenerator`.
+
+Combining the format requirements above, the example configuration in
+`./examples/benchmark-example-conf.json` is as follows.
+
+```json
+{
+  "version": 1,
+  "KMeansModel-1": {
+    "stage": {
+      "className": "org.apache.flink.ml.clustering.kmeans.KMeansModel",
+      "paramMap": {
+        "featuresCol": "features",
+        "k": 2,
+        "distanceMeasure": "euclidean",
+        "predictionCol": "prediction"
+      }
+    },
+    "modelData":  {
+      "className": "org.apache.flink.ml.benchmark.clustering.kmeans.KMeansModelDataGenerator",
+      "paramMap": {
+        "seed": null,
+        "numValues": 1,
+        "arraySize": 2,
+        "vectorDim": 10
+      }
+    },
+    "inputs": {

Review comment:
       Would `inputData` be more consistent with the key name `modelData` used above?

##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,172 @@
+# Flink ML Benchmark Getting Started
+
+This document provides instructions on how to run benchmarks on Flink ML's
+stages in a Linux/MacOS environment.
+
+## Prerequisites
+
+### Install Flink
+
+Please make sure Flink 1.14 or higher version has been installed in your local
+environment. You can refer to the [local
+installation](https://nightlies.apache.org/flink/flink-docs-master/docs/try-flink/local_installation/)
+instruction on Flink's document website for how to achieve this.
+
+### Set Up Flink Environment Variables
+
+After having installed Flink, please register `$FLINK_HOME` as an environment
+variable into your local environment. For example, suppose you have downloaded
+Flink 1.14.0 and placed Flink's binary folder under `/usr/local/`, then you need
+to run the following command:
+
+```bash
+export FLINK_HOME=`/usr/local/flink-1.14.0`

Review comment:
       How about changing the command to the following, and say "Navigate to the extracted Flink directory and set `$FLINK_HOME` to the Flink directory path"
   
   ```
   cd flink-*  && export FLINK_HOME=`pwd`
   ```

##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,172 @@
+# Flink ML Benchmark Getting Started
+
+This document provides instructions on how to run benchmarks on Flink ML's
+stages in a Linux/MacOS environment.
+
+## Prerequisites
+
+### Install Flink
+
+Please make sure Flink 1.14 or higher version has been installed in your local
+environment. You can refer to the [local
+installation](https://nightlies.apache.org/flink/flink-docs-master/docs/try-flink/local_installation/)
+instruction on Flink's document website for how to achieve this.
+
+### Set Up Flink Environment Variables
+
+After having installed Flink, please register `$FLINK_HOME` as an environment
+variable into your local environment. For example, suppose you have downloaded
+Flink 1.14.0 and placed Flink's binary folder under `/usr/local/`, then you need
+to run the following command:
+
+```bash
+export FLINK_HOME=`/usr/local/flink-1.14.0`
+```
+
+Then please run the following command. If this command returns 1.14.0 or a
+higher version, then it means that the required Flink environment has been
+successfully installed and registered in your local environment.
+
+```bash
+$FLINK_HOME/bin/flink --version
+```
+
+### Acquire Flink ML Binary Distribution
+
+In order to use Flink ML's CLI you need to have the latest binary distribution
+of Flink ML. You can acquire the distribution by building Flink ML's source code
+locally, which means to execute the following command in Flink ML repository's
+root directory.
+
+```bash
+mvn clean package -DskipTests
+cd ./flink-ml-dist/target/flink-ml-*-bin/flink-ml*/
+```
+
+### Start Flink Cluster
+
+Please start a Flink standalone session in your local environment with the
+following command.
+
+```bash
+$FLINK_HOME/bin/start-cluster.sh
+```
+
+You should be able to navigate to the web UI at
+[localhost:8081](http://localhost:8081/) to view the Flink dashboard and see
+that the cluster is up and running.
+
+## Run Benchmark Example
+
+In Flink ML's binary distribution's folder, execute the following command to run
+an example benchmark.
+
+```bash
+./bin/flink-ml-benchmark.sh ./examples/benchmark-example-conf.json --output-file ./output/results.json
+```
+
+You will notice that some Flink job is submitted to your Flink cluster, and the
+following information is printed out in your terminal. This means that you have
+successfully executed a benchmark on `KMeansModel`.
+
+```
+Creating fat jar containing all flink ml dependencies to be submitted.
+Job has been submitted with JobID a5d8868d808eecfb357eb904c961c3bf
+Program execution finished
+Job with JobID a5d8868d808eecfb357eb904c961c3bf has finished.
+Job Runtime: 897 ms
+Accumulator Results: 
+- numElements (java.lang.Long): 10000
+
+
+{
+  "name" : "KMeansModel-1",
+  "totalTimeMs" : 897.0,
+  "inputRecordNum" : 10000,
+  "inputThroughput" : 11148.272017837235,
+  "outputRecordNum" : 10000,
+  "outputThroughput" : 11148.272017837235
+}
+```
+
+The command above would save the results into `./output/results.json` as below.
+
+```json
+[ {
+  "name" : "KMeansModel-1",
+  "totalTimeMs" : 897.0,
+  "inputRecordNum" : 10000,
+  "inputThroughput" : 11148.272017837235,
+  "outputRecordNum" : 10000,
+  "outputThroughput" : 11148.272017837235
+} ]
+```
+
+## Customize Benchmark Configuration
+
+`flink-ml-benchmark.sh` parses benchmarks to be executed according to the input
+configuration file, like `./examples/benchmark-example-conf.json`. It can also

Review comment:
       Would it be simpler to use the following path for the example config file:
   
   ```
   ./conf/benchmark_example.json
   ```
   

##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,172 @@
+# Flink ML Benchmark Getting Started
+
+This document provides instructions on how to run benchmarks on Flink ML's
+stages in a Linux/MacOS environment.
+
+## Prerequisites
+
+### Install Flink
+
+Please make sure Flink 1.14 or higher version has been installed in your local
+environment. You can refer to the [local
+installation](https://nightlies.apache.org/flink/flink-docs-master/docs/try-flink/local_installation/)
+instruction on Flink's document website for how to achieve this.
+
+### Set Up Flink Environment Variables
+
+After having installed Flink, please register `$FLINK_HOME` as an environment
+variable into your local environment. For example, suppose you have downloaded
+Flink 1.14.0 and placed Flink's binary folder under `/usr/local/`, then you need
+to run the following command:
+
+```bash
+export FLINK_HOME=`/usr/local/flink-1.14.0`
+```
+
+Then please run the following command. If this command returns 1.14.0 or a
+higher version, then it means that the required Flink environment has been
+successfully installed and registered in your local environment.
+
+```bash
+$FLINK_HOME/bin/flink --version
+```
+
+### Acquire Flink ML Binary Distribution
+
+In order to use Flink ML's CLI you need to have the latest binary distribution
+of Flink ML. You can acquire the distribution by building Flink ML's source code
+locally, which means to execute the following command in Flink ML repository's
+root directory.
+
+```bash
+mvn clean package -DskipTests
+cd ./flink-ml-dist/target/flink-ml-*-bin/flink-ml*/
+```
+
+### Start Flink Cluster
+
+Please start a Flink standalone session in your local environment with the
+following command.
+
+```bash
+$FLINK_HOME/bin/start-cluster.sh
+```
+
+You should be able to navigate to the web UI at
+[localhost:8081](http://localhost:8081/) to view the Flink dashboard and see
+that the cluster is up and running.
+
+## Run Benchmark Example
+
+In Flink ML's binary distribution's folder, execute the following command to run
+an example benchmark.
+
+```bash
+./bin/flink-ml-benchmark.sh ./examples/benchmark-example-conf.json --output-file ./output/results.json

Review comment:
       Would it be simpler to just use `--output-file results.json` in the quickstart?

##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,172 @@
+# Flink ML Benchmark Getting Started
+
+This document provides instructions on how to run benchmarks on Flink ML's
+stages in a Linux/MacOS environment.
+
+## Prerequisites
+
+### Install Flink
+
+Please make sure Flink 1.14 or higher version has been installed in your local
+environment. You can refer to the [local
+installation](https://nightlies.apache.org/flink/flink-docs-master/docs/try-flink/local_installation/)
+instruction on Flink's document website for how to achieve this.
+
+### Set Up Flink Environment Variables
+
+After having installed Flink, please register `$FLINK_HOME` as an environment
+variable into your local environment. For example, suppose you have downloaded
+Flink 1.14.0 and placed Flink's binary folder under `/usr/local/`, then you need
+to run the following command:
+
+```bash
+export FLINK_HOME=`/usr/local/flink-1.14.0`
+```
+
+Then please run the following command. If this command returns 1.14.0 or a
+higher version, then it means that the required Flink environment has been
+successfully installed and registered in your local environment.
+
+```bash
+$FLINK_HOME/bin/flink --version
+```
+
+### Acquire Flink ML Binary Distribution
+
+In order to use Flink ML's CLI you need to have the latest binary distribution
+of Flink ML. You can acquire the distribution by building Flink ML's source code
+locally, which means to execute the following command in Flink ML repository's
+root directory.
+
+```bash
+mvn clean package -DskipTests
+cd ./flink-ml-dist/target/flink-ml-*-bin/flink-ml*/
+```
+
+### Start Flink Cluster
+
+Please start a Flink standalone session in your local environment with the
+following command.
+
+```bash
+$FLINK_HOME/bin/start-cluster.sh
+```
+
+You should be able to navigate to the web UI at
+[localhost:8081](http://localhost:8081/) to view the Flink dashboard and see
+that the cluster is up and running.
+
+## Run Benchmark Example
+
+In Flink ML's binary distribution's folder, execute the following command to run
+an example benchmark.
+
+```bash
+./bin/flink-ml-benchmark.sh ./examples/benchmark-example-conf.json --output-file ./output/results.json
+```
+
+You will notice that some Flink job is submitted to your Flink cluster, and the
+following information is printed out in your terminal. This means that you have
+successfully executed a benchmark on `KMeansModel`.
+
+```
+Creating fat jar containing all flink ml dependencies to be submitted.
+Job has been submitted with JobID a5d8868d808eecfb357eb904c961c3bf
+Program execution finished
+Job with JobID a5d8868d808eecfb357eb904c961c3bf has finished.
+Job Runtime: 897 ms
+Accumulator Results: 
+- numElements (java.lang.Long): 10000
+
+
+{
+  "name" : "KMeansModel-1",
+  "totalTimeMs" : 897.0,
+  "inputRecordNum" : 10000,
+  "inputThroughput" : 11148.272017837235,
+  "outputRecordNum" : 10000,
+  "outputThroughput" : 11148.272017837235
+}
+```
+
+The command above would save the results into `./output/results.json` as below.
+
+```json
+[ {
+  "name" : "KMeansModel-1",
+  "totalTimeMs" : 897.0,
+  "inputRecordNum" : 10000,
+  "inputThroughput" : 11148.272017837235,
+  "outputRecordNum" : 10000,
+  "outputThroughput" : 11148.272017837235
+} ]
+```
+
+## Customize Benchmark Configuration
+
+`flink-ml-benchmark.sh` parses benchmarks to be executed according to the input
+configuration file, like `./examples/benchmark-example-conf.json`. It can also
+parse your custom configuration file so long as it contains a JSON object in the
+following format.
+
+- The file should contain the following as the metadata of the JSON object.
+  - `"version"`: The version of the json format. Currently its value must be 1.
+- Keys in the JSON object, except `"version"`, are regarded as the names of the
+  benchmarks.
+- The value of each benchmark name should be a JSON object containing the
+  following keys.
+  - `"stage"`: The stage to be benchmarked.
+  - `"inputs"`: The input data of the stage to be benchmarked.
+  - `"modelData"`(Optional): The model data of the stage to be benchmarked, if
+    the stage is a `Model` and needs to have its model data explicitly set.
+- The value of `"stage"`, `"inputs"` or `"modelData"` should be a JSON object
+  containing the following keys.
+  - `"className"`: The full classpath of a `WithParams` subclass. For `"stage"`,
+    the class should be a subclass of `Stage`. For `"inputs"` or `"modelData"`,
+    the class should be a subclass of `DataGenerator`.
+  - `"paramMap"`: A JSON object containing the parameters related to the
+    specific `Stage` or `DataGenerator`.
+
+Combining the format requirements above, the example configuration in
+`./examples/benchmark-example-conf.json` is as follows.
+
+```json
+{
+  "version": 1,
+  "KMeansModel-1": {
+    "stage": {
+      "className": "org.apache.flink.ml.clustering.kmeans.KMeansModel",
+      "paramMap": {
+        "featuresCol": "features",
+        "k": 2,
+        "distanceMeasure": "euclidean",
+        "predictionCol": "prediction"
+      }
+    },
+    "modelData":  {
+      "className": "org.apache.flink.ml.benchmark.clustering.kmeans.KMeansModelDataGenerator",
+      "paramMap": {
+        "seed": null,
+        "numValues": 1,
+        "arraySize": 2,
+        "vectorDim": 10
+      }
+    },
+    "inputs": {
+      "className": "org.apache.flink.ml.benchmark.data.DenseVectorGenerator",
+      "paramMap": {
+        "seed": null,
+        "outputCols": ["features"],

Review comment:
       There is no input for data generator. So the `output` in the name seems redundant.
   
   How about naming it as `columnNames`?




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
lindong28 commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r841074684



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/Benchmark.java
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.DefaultParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** 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 String SHELL_SCRIPT = "flink-ml-benchmark.sh";
+
+    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 =
+            Option.builder()
+                    .longOpt("output-file")
+                    .desc("The output file name to save benchmark results.")
+                    .hasArg()
+                    .build();
+
+    static final Options OPTIONS =
+            new Options().addOption(HELP_OPTION).addOption(OUTPUT_FILE_OPTION);
+
+    public static void printHelp() {
+        HelpFormatter formatter = new HelpFormatter();
+        formatter.setLeftPadding(5);
+        formatter.setWidth(80);
+
+        System.out.println("./" + SHELL_SCRIPT + " <config-file-path> [OPTIONS]");
+        System.out.println();
+        formatter.setSyntaxPrefix("The following options are available:");
+        formatter.printHelp(" ", OPTIONS);
+
+        System.out.println();
+    }
+
+    @SuppressWarnings("unchecked")
+    public static void executeBenchmarks(CommandLine commandLine) throws Exception {
+        String configFile = commandLine.getArgs()[0];
+
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        InputStream inputStream = new FileInputStream(configFile);
+        Map<String, ?> jsonMap = ReadWriteUtils.OBJECT_MAPPER.readValue(inputStream, Map.class);
+        Preconditions.checkArgument(
+                jsonMap.containsKey(VERSION_KEY) && jsonMap.get(VERSION_KEY).equals(1));
+
+        List<String> benchmarkNames =
+                jsonMap.keySet().stream()
+                        .filter(x -> !x.equals(VERSION_KEY))
+                        .collect(Collectors.toList());
+        LOG.info("Found benchmarks " + benchmarkNames);
+
+        List<BenchmarkResult> results = new ArrayList<>();
+
+        for (String benchmarkName : benchmarkNames) {
+            LOG.info("Running benchmark " + benchmarkName + ".");
+
+            BenchmarkResult result =
+                    BenchmarkUtils.runBenchmark(
+                            tEnv, benchmarkName, (Map<String, ?>) jsonMap.get(benchmarkName));
+
+            results.add(result);
+            BenchmarkUtils.printResults(result);
+        }
+
+        if (commandLine.hasOption(OUTPUT_FILE_OPTION.getLongOpt())) {

Review comment:
       The information printed on line 108 is like log, which provides early preview of benchmark results overtime. This is different from the benchmark result summary, which provides the consolidated benchmark results corresponding to the given benchmark configuration.
   
   I agree that there is not much INFO logging printed out in the terminal. But Given that there is some INFO logging, the information printed on line 108 is close to, but does not equal to the benchmark summary.
   
   Most users running a benchmark would want to see the summary of the benchmark result. Would it be nice to still provide this summary to user when user does not specify the `--output-file` option?




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r841039774



##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,172 @@
+# Flink ML Benchmark Getting Started
+
+This document provides instructions on how to run benchmarks on Flink ML's
+stages in a Linux/MacOS environment.
+
+## Prerequisites
+
+### Install Flink
+
+Please make sure Flink 1.14 or higher version has been installed in your local
+environment. You can refer to the [local
+installation](https://nightlies.apache.org/flink/flink-docs-master/docs/try-flink/local_installation/)
+instruction on Flink's document website for how to achieve this.
+
+### Set Up Flink Environment Variables
+
+After having installed Flink, please register `$FLINK_HOME` as an environment
+variable into your local environment. For example, suppose you have downloaded
+Flink 1.14.0 and placed Flink's binary folder under `/usr/local/`, then you need
+to run the following command:
+
+```bash
+export FLINK_HOME=`/usr/local/flink-1.14.0`
+```
+
+Then please run the following command. If this command returns 1.14.0 or a
+higher version, then it means that the required Flink environment has been
+successfully installed and registered in your local environment.
+
+```bash
+$FLINK_HOME/bin/flink --version
+```
+
+### Acquire Flink ML Binary Distribution
+
+In order to use Flink ML's CLI you need to have the latest binary distribution
+of Flink ML. You can acquire the distribution by building Flink ML's source code
+locally, which means to execute the following command in Flink ML repository's
+root directory.
+
+```bash
+mvn clean package -DskipTests
+cd ./flink-ml-dist/target/flink-ml-*-bin/flink-ml*/
+```
+
+### Start Flink Cluster
+
+Please start a Flink standalone session in your local environment with the
+following command.
+
+```bash
+$FLINK_HOME/bin/start-cluster.sh
+```
+
+You should be able to navigate to the web UI at
+[localhost:8081](http://localhost:8081/) to view the Flink dashboard and see
+that the cluster is up and running.
+
+## Run Benchmark Example
+
+In Flink ML's binary distribution's folder, execute the following command to run
+an example benchmark.
+
+```bash
+./bin/flink-ml-benchmark.sh ./examples/benchmark-example-conf.json --output-file ./output/results.json
+```
+
+You will notice that some Flink job is submitted to your Flink cluster, and the
+following information is printed out in your terminal. This means that you have
+successfully executed a benchmark on `KMeansModel`.
+
+```
+Creating fat jar containing all flink ml dependencies to be submitted.
+Job has been submitted with JobID a5d8868d808eecfb357eb904c961c3bf
+Program execution finished
+Job with JobID a5d8868d808eecfb357eb904c961c3bf has finished.
+Job Runtime: 897 ms
+Accumulator Results: 
+- numElements (java.lang.Long): 10000
+
+
+{
+  "name" : "KMeansModel-1",
+  "totalTimeMs" : 897.0,
+  "inputRecordNum" : 10000,
+  "inputThroughput" : 11148.272017837235,
+  "outputRecordNum" : 10000,
+  "outputThroughput" : 11148.272017837235
+}
+```
+
+The command above would save the results into `./output/results.json` as below.
+
+```json
+[ {
+  "name" : "KMeansModel-1",
+  "totalTimeMs" : 897.0,
+  "inputRecordNum" : 10000,
+  "inputThroughput" : 11148.272017837235,
+  "outputRecordNum" : 10000,
+  "outputThroughput" : 11148.272017837235
+} ]
+```
+
+## Customize Benchmark Configuration
+
+`flink-ml-benchmark.sh` parses benchmarks to be executed according to the input
+configuration file, like `./examples/benchmark-example-conf.json`. It can also

Review comment:
       I'm not sure it is a good name, because the file is not Flink or Flink ML's configuration. I have renamed the file as `./examples/kmeansmodel-benchmark.json`, and please consider whether this name is good enough.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r841022266



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/Benchmark.java
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.DefaultParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** 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 String SHELL_SCRIPT = "flink-ml-benchmark.sh";

Review comment:
       I think `args[0]` represents the script name only in shell scripts, not Java programs. `args[0]` in this Java program is `benchmark-example-conf.json` in the quick start and tests. 
   
   I'm not sure it is common to pass a script name to a Java program. In Flink's `CliFrontend`, `args[0]` is an action like `run`, `cancel` or `stop`, instead of `./bin/flink`.
   
   `SHELL_SCRIPT` is only used when printing out help massages. It is not something that needs to be specified from user inputs, so I think it is better to define it as a constant variable than trying to retrieve it from arguments.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r840345534



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/generator/GeneratorParams.java
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.flink.ml.benchmark.generator;
+
+import org.apache.flink.ml.common.param.HasSeed;
+import org.apache.flink.ml.param.IntParam;
+import org.apache.flink.ml.param.LongParam;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+
+/** Interface for the generator params. */
+public interface GeneratorParams<T> extends HasSeed<T> {

Review comment:
       OK. I'll make the change.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r841020736



##########
File path: flink-ml-lib/src/main/java/org/apache/flink/ml/common/param/HasOutputCols.java
##########
@@ -19,15 +19,13 @@
 package org.apache.flink.ml.common.param;
 
 import org.apache.flink.ml.param.Param;
-import org.apache.flink.ml.param.ParamValidators;
 import org.apache.flink.ml.param.StringArrayParam;
 import org.apache.flink.ml.param.WithParams;
 
 /** Interface for the shared outputCols param. */
 public interface HasOutputCols<T> extends WithParams<T> {
     Param<String[]> OUTPUT_COLS =
-            new StringArrayParam(
-                    "outputCols", "Output column names.", null, ParamValidators.nonEmptyArray());
+            new StringArrayParam("outputCols", "Output column names.", new String[0]);

Review comment:
       Because the current implementation uses `HasOutputCols` to specify the column name of the generated tables. As we should allow users not to config this parameter, we have to make the default value of `HasOutputCols` a valid but empty value. 
   
   It does change the semantic of `HasOutputCols`. I'll move this parameter to the `HasColNames` parameter mentioned in previous comments and keep `HasOutputCols` as it has been.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r840198302



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/clustering/kmeans/KMeansInputsGenerator.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.flink.ml.benchmark.clustering.kmeans;
+
+import org.apache.flink.ml.benchmark.generator.DataGenerator;
+import org.apache.flink.ml.benchmark.generator.GeneratorUtils;
+import org.apache.flink.ml.clustering.kmeans.KMeansParams;
+import org.apache.flink.ml.common.datastream.TableUtils;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Class that generates table arrays containing inputs for {@link
+ * org.apache.flink.ml.clustering.kmeans.KMeans} and {@link
+ * org.apache.flink.ml.clustering.kmeans.KMeansModel}.
+ */
+public class KMeansInputsGenerator

Review comment:
       Besides, I have moved these classes from` org.apache.flink.ml.benchmark.generator` to `org.apache.flink.ml.benchmark.data`, as I think the former might bring ambiguity. Classes like `DenseVectorGenerator` are now in the same package as `DataGenerator`.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r840180676



##########
File path: flink-ml-benchmark/src/test/java/org/apache/flink/ml/benchmark/BenchmarkTest.java
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.ml.benchmark.clustering.kmeans.KMeansInputsGenerator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+import org.apache.flink.ml.clustering.kmeans.KMeansModel;
+import org.apache.flink.ml.param.WithParams;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.test.util.AbstractTestBase;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.InputStream;
+import java.io.PrintStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.ml.util.ReadWriteUtils.OBJECT_MAPPER;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/** Tests benchmarks. */
+@SuppressWarnings("unchecked")
+public class BenchmarkTest extends AbstractTestBase {
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+
+    private final ByteArrayOutputStream outContent = new ByteArrayOutputStream();
+    private final PrintStream originalOut = System.out;
+
+    @Before
+    public void before() {
+        System.setOut(new PrintStream(outContent));
+    }
+
+    @After
+    public void after() {
+        System.setOut(originalOut);

Review comment:
       When we do `System.setOut(new PrintStream(outContent));`, we have already made `System.out` points to `new PrintStream(outContent)`, so `System.setOut(System.out)` cannot help to reset to stdout.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r841034725



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/clustering/kmeans/KMeansModelDataGenerator.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.flink.ml.benchmark.clustering.kmeans;

Review comment:
       I think the third option is the most proper one. I'll implement in this way.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r838060042



##########
File path: flink-ml-dist/src/main/flink-ml-bin/bin/flink-ml-benchmark.sh
##########
@@ -0,0 +1,61 @@
+#!/usr/bin/env bash
+################################################################################
+#  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.
+################################################################################
+
+current_path=$(pwd)
+flink_ml_bin_path="$( cd -- "$(dirname "$0")" >/dev/null 2>&1 ; pwd -P )"
+flink_ml_root_path="$(dirname "$flink_ml_bin_path")"
+
+# Checks flink command.
+flink_cmd="flink"

Review comment:
       OK. I'll add instructions to configure Flink environment variables.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r841034618



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/data/DenseVectorArrayGeneratorParams.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.flink.ml.benchmark.data;
+
+import org.apache.flink.ml.param.IntParam;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+
+/** Interface for the vector array generator params. */
+public interface DenseVectorArrayGeneratorParams<T> extends DenseVectorGeneratorParams<T> {

Review comment:
       I think it might be better to introduce `KMeansModelDataGeneratorParams` and to have it extend `DataGeneratorParams`, `HasVectorDim` and `HasArraySize`. We still need `DenseVectorArrayGeneratorParams` as it contains not only arraySize, but also everything in `DenseVectorGeneratorParams`. Besides, `DenseVectorArrayGenerator` should still be an `InputDataGenerator`, just that when it is used in internal implementation of `KMeansModelDataGenerator`, `numValue` would be 1 and `colNames` would not be set.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
lindong28 commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r840997700



##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,169 @@
+# Flink ML Benchmark Getting Started
+
+This document provides instructions about how to run benchmarks on Flink ML's
+stages in a Linux/MacOS environment.
+
+## Prerequisites
+
+### Installing Flink
+
+Please make sure Flink 1.14 or higher version has been installed in your local
+environment. You can refer to the [local
+installation](https://nightlies.apache.org/flink/flink-docs-master/docs/try-flink/local_installation/)
+instruction on Flink's document website for how to achieve this.
+
+### Setting Up Flink Environment Variables
+
+After having installed Flink, please register `$FLINK_HOME` as an environment
+variable into your local environment, and add `$FLINK_HOME` into your `$PATH`
+variable. This can be completed by running the following commands in the Flink's
+folder.
+
+```bash
+export FLINK_HOME=`pwd`

Review comment:
       Thanks. I will comment on the updated PR.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
lindong28 commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r841071394



##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,172 @@
+# Flink ML Benchmark Getting Started
+
+This document provides instructions on how to run benchmarks on Flink ML's
+stages in a Linux/MacOS environment.
+
+## Prerequisites
+
+### Install Flink
+
+Please make sure Flink 1.14 or higher version has been installed in your local
+environment. You can refer to the [local
+installation](https://nightlies.apache.org/flink/flink-docs-master/docs/try-flink/local_installation/)
+instruction on Flink's document website for how to achieve this.
+
+### Set Up Flink Environment Variables
+
+After having installed Flink, please register `$FLINK_HOME` as an environment
+variable into your local environment. For example, suppose you have downloaded
+Flink 1.14.0 and placed Flink's binary folder under `/usr/local/`, then you need
+to run the following command:
+
+```bash
+export FLINK_HOME=`/usr/local/flink-1.14.0`
+```
+
+Then please run the following command. If this command returns 1.14.0 or a
+higher version, then it means that the required Flink environment has been
+successfully installed and registered in your local environment.
+
+```bash
+$FLINK_HOME/bin/flink --version
+```
+
+### Acquire Flink ML Binary Distribution
+
+In order to use Flink ML's CLI you need to have the latest binary distribution
+of Flink ML. You can acquire the distribution by building Flink ML's source code
+locally, which means to execute the following command in Flink ML repository's
+root directory.
+
+```bash
+mvn clean package -DskipTests
+cd ./flink-ml-dist/target/flink-ml-*-bin/flink-ml*/

Review comment:
       Sounds good.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r840410187



##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,169 @@
+# Flink ML Benchmark Getting Started
+
+This document provides instructions about how to run benchmarks on Flink ML's
+stages in a Linux/MacOS environment.
+
+## Prerequisites
+
+### Installing Flink
+
+Please make sure Flink 1.14 or higher version has been installed in your local
+environment. You can refer to the [local
+installation](https://nightlies.apache.org/flink/flink-docs-master/docs/try-flink/local_installation/)
+instruction on Flink's document website for how to achieve this.
+
+### Setting Up Flink Environment Variables
+
+After having installed Flink, please register `$FLINK_HOME` as an environment
+variable into your local environment, and add `$FLINK_HOME` into your `$PATH`
+variable. This can be completed by running the following commands in the Flink's
+folder.
+
+```bash
+export FLINK_HOME=`pwd`
+export PATH=$FLINK_HOME/bin:$PATH

Review comment:
       OK. I'll make the change.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r838146291



##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,261 @@
+# Flink ML Benchmark Guideline
+
+This document provides instructions about how to run benchmarks on Flink ML's
+stages.
+
+## Write Benchmark Programs
+
+### Add Maven Dependencies
+
+In order to write Flink ML's java benchmark programs, first make sure that the
+following dependencies have been added to your maven project's `pom.xml`.
+
+```xml
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-core_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-iteration_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-lib_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-benchmark_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>statefun-flink-core</artifactId>
+  <version>3.1.0</version>
+  <exclusions>
+    <exclusion>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-streaming-java_2.12</artifactId>
+    </exclusion>
+  </exclusions>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-table-api-java-bridge_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-table-planner_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-clients_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+```
+
+### Write Java Program
+
+Then you can write a program as follows to run benchmark on Flink ML stages. The
+example code below tests the performance of Flink ML's KMeans algorithm, with
+the default configuration parameters used.
+
+```java
+public class Main {
+    public static void main(String[] args) throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        KMeans kMeans = new KMeans();
+        KMeansInputsGenerator inputsGenerator = new KMeansInputsGenerator();
+
+        BenchmarkResult result =
+                BenchmarkUtils.runBenchmark("exampleBenchmark", tEnv, kMeans, inputsGenerator);
+
+        BenchmarkUtils.printResult(result);
+    }
+}
+```
+
+### Execute Benchmark Program
+
+After executing the `main()` method above, you will see benchmark results
+printed out in your terminal. An example of the printed content is as follows.
+
+```
+Benchmark Name: exampleBenchmark
+Total Execution Time(ms): 828.0
+```
+
+### Configure Benchmark Parameters
+
+If you want to run benchmark on customed configuration parameters, you can set
+them with Flink ML's `WithParams` API as follows.
+
+```java
+KMeans kMeans = new KMeans()
+  .setK(5)
+  .setMaxIter(50);
+KMeansInputsGenerator inputsGenerator = new KMeansInputsGenerator()
+  .setDims(3)
+  .setDataSize(10000);
+```
+
+## Execute Benchmark through Command-Line Interface (CLI)
+
+You can also configure and execute benchmarks through Command-Line Interface
+(CLI) without writing java programs.
+
+### Prerequisites
+
+Before using Flink ML's CLI, make sure you have installed Flink 1.14 in your
+local environment, and that you have started a Flink cluster locally. If not,
+you can start a standalone session with the following command.
+
+```bash
+$ start-cluster
+```
+
+In order to use Flink ML's CLI you need to have the latest binary distribution
+of Flink ML. You can acquire the distribution by building Flink ML's source code
+locally, which means to execute the following command in Flink ML repository's
+root directory.
+
+```bash
+$ mvn clean package -DskipTests
+```
+
+After executing the command above, you will be able to find the binary
+distribution under
+`./flink-ml-dist/target/flink-ml-<version>-bin/flink-ml-<version>/`.
+
+### Run Benchmark CLI
+
+In the binary distribution's folder, execute the following command to run an
+example benchmark.
+
+```bash
+$ ./bin/flink-ml-benchmark.sh ./examples/benchmark-example-conf.json
+```
+
+You will notice that some Flink job is submitted to your Flink cluster, and the
+following information is printed out in your terminal. This means that you have
+successfully executed a benchmark on `KMeansModel`.
+
+```
+Job has been submitted with JobID 85b4a33df5c00a315e0d1142e1d743be
+Program execution finished
+Job with JobID 85b4a33df5c00a315e0d1142e1d743be has finished.
+Job Runtime: 828 ms
+
+Benchmark Name: KMeansModel-1
+Total Execution Time(ms): 828.0
+
+```
+
+### Save Benchmark Result to File
+
+`flink-ml-benchmark.sh` has redirected all warnings and process logs to stderr,
+and the benchmark results to stdout. So if you write the command in the
+following way
+
+```bash
+$ ./bin/flink-ml-benchmark.sh ./examples/benchmark-example-conf.json > output.txt

Review comment:
       I agree. I'll merge them into one.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r838059820



##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,261 @@
+# Flink ML Benchmark Guideline
+
+This document provides instructions about how to run benchmarks on Flink ML's
+stages.
+
+## Write Benchmark Programs
+
+### Add Maven Dependencies
+
+In order to write Flink ML's java benchmark programs, first make sure that the
+following dependencies have been added to your maven project's `pom.xml`.
+
+```xml
+<dependency>

Review comment:
       According to offline discussion, I'll remove the instructions to build a benchmark Java program for now, and only provide instructions to run from CLI.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r841038690



##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,172 @@
+# Flink ML Benchmark Getting Started
+
+This document provides instructions on how to run benchmarks on Flink ML's
+stages in a Linux/MacOS environment.
+
+## Prerequisites
+
+### Install Flink
+
+Please make sure Flink 1.14 or higher version has been installed in your local
+environment. You can refer to the [local
+installation](https://nightlies.apache.org/flink/flink-docs-master/docs/try-flink/local_installation/)
+instruction on Flink's document website for how to achieve this.
+
+### Set Up Flink Environment Variables
+
+After having installed Flink, please register `$FLINK_HOME` as an environment
+variable into your local environment. For example, suppose you have downloaded
+Flink 1.14.0 and placed Flink's binary folder under `/usr/local/`, then you need
+to run the following command:
+
+```bash
+export FLINK_HOME=`/usr/local/flink-1.14.0`
+```
+
+Then please run the following command. If this command returns 1.14.0 or a
+higher version, then it means that the required Flink environment has been
+successfully installed and registered in your local environment.
+
+```bash
+$FLINK_HOME/bin/flink --version
+```
+
+### Acquire Flink ML Binary Distribution
+
+In order to use Flink ML's CLI you need to have the latest binary distribution
+of Flink ML. You can acquire the distribution by building Flink ML's source code
+locally, which means to execute the following command in Flink ML repository's
+root directory.
+
+```bash
+mvn clean package -DskipTests
+cd ./flink-ml-dist/target/flink-ml-*-bin/flink-ml*/

Review comment:
       When users execute `mvn clean package`, they are supposed to have been in the flink ml package. Thus I think the following is better.
   ```shell
   cd ${path_to_flink_ml}
   mvn clean package -DskipTests
   cd ./flink-ml-dist/target/flink-ml-*-bin/flink-ml*/
   ```
   Similarly in the comment above, I'll change the command into below.
   ```shell
   cd ${path_to_flink}
   export FLINK_HOME=`pwd`
   ```




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r840168951



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/BenchmarkResult.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/** The result of executing a benchmark. */
+public class BenchmarkResult {
+    /** The name of the benchmark. */
+    public String name;

Review comment:
       I agree that these fields should be made `final`, but adding a constructor with all these field values might possibly limit the extension of the interface. For example, when in near future when we add fields like `latencyP99Ms`, all usages of this constructor needs to be modified. I would prefer to add a `BenchmarkResult.Builder` that helps setting an arbitrary number of fields before constructing the `BenchmarkResult` instance.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r840183375



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/Benchmark.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** Entry class for benchmark execution. */
+public class Benchmark {
+    private static final Logger LOG = LoggerFactory.getLogger(Benchmark.class);
+
+    @SuppressWarnings("unchecked")
+    public static void main(String[] args) throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        InputStream inputStream = new FileInputStream(args[0]);
+        Map<String, ?> jsonMap = ReadWriteUtils.OBJECT_MAPPER.readValue(inputStream, Map.class);
+        Preconditions.checkArgument(
+                jsonMap.containsKey("version") && jsonMap.get("version").equals(1));

Review comment:
       In my opinion, making it package-private might be better. `"version"` is also used in `BenchmarkTest` now and might be used in `BenchmarkUtils` in future.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r840185834



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/clustering/kmeans/KMeansModelDataGenerator.java
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.flink.ml.benchmark.clustering.kmeans;
+
+import org.apache.flink.ml.benchmark.generator.DataGenerator;
+import org.apache.flink.ml.benchmark.generator.GeneratorUtils;
+import org.apache.flink.ml.clustering.kmeans.KMeansModelData;
+import org.apache.flink.ml.clustering.kmeans.KMeansParams;
+import org.apache.flink.ml.common.datastream.TableUtils;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Class that generates table arrays containing model data for {@link
+ * org.apache.flink.ml.clustering.kmeans.KMeansModel}.
+ */
+public class KMeansModelDataGenerator

Review comment:
       We could extract a `DenseVectorArrayGenerator` from it, but `KMeansModelDataGenerator` still needs to be preserved because `KMeansModelData` still has `weights` apart from a dense vector array centroids. 




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#issuecomment-1085852592


   Thanks for the comments. I have updated the PR according to the comments.


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
lindong28 commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r840998966



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/clustering/kmeans/KMeansInputsGenerator.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.flink.ml.benchmark.clustering.kmeans;
+
+import org.apache.flink.ml.benchmark.generator.DataGenerator;
+import org.apache.flink.ml.benchmark.generator.GeneratorUtils;
+import org.apache.flink.ml.clustering.kmeans.KMeansParams;
+import org.apache.flink.ml.common.datastream.TableUtils;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Class that generates table arrays containing inputs for {@link
+ * org.apache.flink.ml.clustering.kmeans.KMeans} and {@link
+ * org.apache.flink.ml.clustering.kmeans.KMeansModel}.
+ */
+public class KMeansInputsGenerator

Review comment:
       Sounds good.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r841035426



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/data/CommonDataGeneratorParams.java
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.flink.ml.benchmark.data;
+
+import org.apache.flink.ml.common.param.HasOutputCols;
+import org.apache.flink.ml.common.param.HasSeed;
+import org.apache.flink.ml.param.LongParam;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+
+/** Interface for the common generator params. */
+public interface CommonDataGeneratorParams<T> extends HasSeed<T>, HasOutputCols<T> {

Review comment:
       I think it is a good idea. `colNames` might be a more proper name, in correspondence with params like `HasFeatureCol`.
   
   Besides, I'll place `COL_NAMES` within `InputDataGeneratorParams` for now. I think it is better not to create `benchmark.param.HasColNames` until we confirm that it is a shared parameter.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r837429430



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/Benchmark.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.io.PrintStream;
+import java.util.Map;
+
+import static org.apache.flink.ml.util.ReadWriteUtils.OBJECT_MAPPER;
+
+/** Entry class for benchmark execution. */
+public class Benchmark {
+    @SuppressWarnings("unchecked")
+    public static void main(String[] args) throws Exception {
+        final PrintStream originalOut = System.out;
+
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        InputStream inputStream = new FileInputStream(args[0]);
+        Map<String, ?> jsonMap = OBJECT_MAPPER.readValue(inputStream, Map.class);
+        Map<String, Map<String, ?>> benchmarkParamsMap =
+                BenchmarkUtils.parseBenchmarkParams(jsonMap);
+        System.err.println("Found benchmarks " + benchmarkParamsMap.keySet());
+
+        for (String benchmarkName : benchmarkParamsMap.keySet()) {
+            System.err.println("Running benchmark " + benchmarkName + ".");
+
+            // Redirect all flink execution logs to stderr.
+            System.setOut(System.err);

Review comment:
       According to offline discussion, I'll make `Benchmark` use `LOG.info` for non-result output information.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
lindong28 commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r838379671



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/BenchmarkUtils.java
##########
@@ -0,0 +1,233 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.ml.api.AlgoOperator;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.api.Model;
+import org.apache.flink.ml.api.Stage;
+import org.apache.flink.ml.benchmark.generator.DataGenerator;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.WithParams;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.bridge.java.internal.StreamTableEnvironmentImpl;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.Preconditions;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+/** Utility methods for benchmarks. */
+@SuppressWarnings("unchecked")
+public class BenchmarkUtils {
+    private static final String benchmarkNamePattern = "^[A-Za-z0-9][A-Za-z0-9_\\-]*$";
+
+    /**
+     * Loads benchmark paramMaps from the provided json map.
+     *
+     * @return A map whose key is the names of the loaded benchmarks, value is the parameters of the
+     *     benchmarks.
+     */
+    public static Map<String, Map<String, ?>> parseBenchmarkParams(Map<String, ?> jsonMap) {
+        Preconditions.checkArgument(
+                jsonMap.containsKey("_version") && jsonMap.get("_version").equals(1));
+
+        Map<String, Map<String, ?>> result = new HashMap<>();
+        for (String key : jsonMap.keySet()) {
+            if (!isValidBenchmarkName(key)) {
+                continue;
+            }
+            result.put(key, (Map<String, ?>) jsonMap.get(key));
+        }
+        return result;
+    }
+
+    /**
+     * Checks whether a string is a valid benchmark name.
+     *
+     * <p>A valid benchmark name should only contain English letters, numbers, hyphens (-) and
+     * underscores (_). The name should not start with a hyphen or underscore.
+     */
+    public static boolean isValidBenchmarkName(String name) {
+        return Pattern.matches(benchmarkNamePattern, name);
+    }
+
+    /**
+     * Instantiates a benchmark from its parameter map and executes the benchmark in the provided
+     * environment.
+     *
+     * @return Results of the executed benchmark.
+     */
+    public static BenchmarkResult runBenchmark(
+            String name, StreamTableEnvironment tEnv, Map<String, ?> benchmarkParamsMap)
+            throws Exception {
+        Stage<?> stage =
+                (Stage<?>) instantiateWithParams((Map<String, ?>) benchmarkParamsMap.get("stage"));
+
+        BenchmarkResult result;
+        if (benchmarkParamsMap.size() == 2) {
+            DataGenerator<?> inputsGenerator =
+                    (DataGenerator<?>)
+                            instantiateWithParams(
+                                    (Map<String, ?>) benchmarkParamsMap.get("inputs"));
+            result = runBenchmark(name, tEnv, stage, inputsGenerator);
+        } else if (benchmarkParamsMap.size() == 3 && stage instanceof Model) {
+            DataGenerator<?> inputsGenerator =
+                    (DataGenerator<?>)
+                            instantiateWithParams(
+                                    (Map<String, ?>) benchmarkParamsMap.get("inputs"));
+            DataGenerator<?> modelDataGenerator =
+                    (DataGenerator<?>)
+                            instantiateWithParams(
+                                    (Map<String, ?>) benchmarkParamsMap.get("modelData"));
+            result =
+                    runBenchmark(name, tEnv, (Model<?>) stage, modelDataGenerator, inputsGenerator);
+        } else {
+            throw new IllegalArgumentException(
+                    "Unsupported json map with keys " + benchmarkParamsMap.keySet());
+        }
+
+        return result;
+    }
+
+    /**
+     * Executes a benchmark from a stage and an inputsGenerator in the provided environment.
+     *
+     * @return Results of the executed benchmark.
+     */
+    public static BenchmarkResult runBenchmark(
+            String name,
+            StreamTableEnvironment tEnv,
+            Stage<?> stage,
+            DataGenerator<?> inputsGenerator)
+            throws Exception {
+        Table[] inputTables = inputsGenerator.getData(tEnv);
+
+        Table[] outputTables;
+        if (stage instanceof Estimator) {
+            outputTables = ((Estimator<?, ?>) stage).fit(inputTables).getModelData();
+        } else if (stage instanceof AlgoOperator) {
+            outputTables = ((AlgoOperator<?>) stage).transform(inputTables);
+        } else {
+            throw new IllegalArgumentException("Unsupported Stage class " + stage.getClass());
+        }
+
+        for (Table table : outputTables) {
+            tEnv.toDataStream(table).addSink(new DiscardingSink<>());
+        }
+
+        StreamExecutionEnvironment env = ((StreamTableEnvironmentImpl) tEnv).execEnv();
+        JobExecutionResult executionResult = env.execute();
+
+        BenchmarkResult result = new BenchmarkResult();
+        result.name = name;
+        result.executionTimeMillis = (double) executionResult.getNetRuntime(TimeUnit.MILLISECONDS);

Review comment:
       The alternative approach sounds good.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r837428065



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/BenchmarkUtils.java
##########
@@ -0,0 +1,233 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.ml.api.AlgoOperator;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.api.Model;
+import org.apache.flink.ml.api.Stage;
+import org.apache.flink.ml.benchmark.generator.DataGenerator;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.WithParams;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.bridge.java.internal.StreamTableEnvironmentImpl;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.Preconditions;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+/** Utility methods for benchmarks. */
+@SuppressWarnings("unchecked")
+public class BenchmarkUtils {
+    private static final String benchmarkNamePattern = "^[A-Za-z0-9][A-Za-z0-9_\\-]*$";
+
+    /**
+     * Loads benchmark paramMaps from the provided json map.
+     *
+     * @return A map whose key is the names of the loaded benchmarks, value is the parameters of the
+     *     benchmarks.
+     */
+    public static Map<String, Map<String, ?>> parseBenchmarkParams(Map<String, ?> jsonMap) {
+        Preconditions.checkArgument(
+                jsonMap.containsKey("_version") && jsonMap.get("_version").equals(1));
+
+        Map<String, Map<String, ?>> result = new HashMap<>();
+        for (String key : jsonMap.keySet()) {
+            if (!isValidBenchmarkName(key)) {

Review comment:
       According to offline discussion, I'll make all non-reserved strings valid benchmark names.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
lindong28 commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r838380741



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/BenchmarkResult.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/** The result of executing a benchmark. */
+public class BenchmarkResult {
+    /** The name of the benchmark. */
+    public String name;

Review comment:
       Since we don't need to change these field values after `BenchmarkResult` is instantiated, could we make these fields `final` and adds a constructor that takes all the field values as input?

##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,169 @@
+# Flink ML Benchmark Getting Started
+
+This document provides instructions about how to run benchmarks on Flink ML's

Review comment:
       nits: instructions about -> instructions on

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/Benchmark.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** Entry class for benchmark execution. */
+public class Benchmark {
+    private static final Logger LOG = LoggerFactory.getLogger(Benchmark.class);
+
+    @SuppressWarnings("unchecked")
+    public static void main(String[] args) throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        InputStream inputStream = new FileInputStream(args[0]);
+        Map<String, ?> jsonMap = ReadWriteUtils.OBJECT_MAPPER.readValue(inputStream, Map.class);
+        Preconditions.checkArgument(
+                jsonMap.containsKey("version") && jsonMap.get("version").equals(1));
+
+        List<String> benchmarkNames =
+                jsonMap.keySet().stream()
+                        .filter(x -> !x.equals("version"))
+                        .collect(Collectors.toList());
+        LOG.info("Found benchmarks " + benchmarkNames);
+
+        List<BenchmarkResult> results = new ArrayList<>();
+
+        for (String benchmarkName : benchmarkNames) {
+            LOG.info("Running benchmark " + benchmarkName + ".");
+
+            BenchmarkResult result =
+                    BenchmarkUtils.runBenchmark(
+                            tEnv, benchmarkName, (Map<String, ?>) jsonMap.get(benchmarkName));
+
+            results.add(result);
+        }
+
+        for (BenchmarkResult result : results) {
+            BenchmarkUtils.printResult(result);
+        }
+
+        if (args.length > 1) {

Review comment:
       For optional arguments like this, we typically make it a keyword argument instead of a positional argument.
   
   How about using `--output-file` as its key?
   
   Previously I have used `net.sourceforge.argparse4j.ArgumentParsers` to parse command line arguments. Not sure if there are better choices used in Flink.

##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,169 @@
+# Flink ML Benchmark Getting Started
+
+This document provides instructions about how to run benchmarks on Flink ML's
+stages in a Linux/MacOS environment.
+
+## Prerequisites
+
+### Installing Flink
+
+Please make sure Flink 1.14 or higher version has been installed in your local
+environment. You can refer to the [local
+installation](https://nightlies.apache.org/flink/flink-docs-master/docs/try-flink/local_installation/)
+instruction on Flink's document website for how to achieve this.
+
+### Setting Up Flink Environment Variables
+
+After having installed Flink, please register `$FLINK_HOME` as an environment
+variable into your local environment, and add `$FLINK_HOME` into your `$PATH`
+variable. This can be completed by running the following commands in the Flink's
+folder.
+
+```bash
+export FLINK_HOME=`pwd`
+export PATH=$FLINK_HOME/bin:$PATH
+```
+
+Then please run the following command. If this command returns 1.14.0 or a
+higher version, then it means that the required Flink environment has been
+successfully installed and registered in your local environment.
+
+```bash
+flink --version
+```
+
+### Acquiring Flink ML Binary Distribution
+
+In order to use Flink ML's CLI you need to have the latest binary distribution
+of Flink ML. You can acquire the distribution by building Flink ML's source code
+locally, which means to execute the following command in Flink ML repository's
+root directory.
+
+```bash
+$ mvn clean package -DskipTests
+```
+
+After executing the command above, you will be able to find the binary
+distribution under
+`./flink-ml-dist/target/flink-ml-<version>-bin/flink-ml-<version>/`.
+
+### Starting Flink Cluster
+
+Please start a Flink standalone session in your local environment with the
+following command.
+
+```bash
+start-cluster.sh
+```
+
+You should be able to navigate to the web UI at
+[localhost:8081](http://localhost:8081/) to view the Flink dashboard and see
+that the cluster is up and running.
+
+## Run Benchmark Example
+
+In Flink ML's binary distribution's folder, execute the following command to run
+an example benchmark.
+
+```bash
+$ ./bin/flink-ml-benchmark.sh ./examples/benchmark-example-conf.json ./output/results.json
+```
+
+You will notice that some Flink job is submitted to your Flink cluster, and the
+following information is printed out in your terminal. This means that you have
+successfully executed a benchmark on `KMeansModel`.
+
+```
+Creating fat jar containing all flink ml dependencies to be submitted.
+Job has been submitted with JobID bdaa54b065adf2c813619113a00337de
+Program execution finished
+Job with JobID bdaa54b065adf2c813619113a00337de has finished.
+Job Runtime: 215 ms
+Accumulator Results: 
+- numElements (java.lang.Long): 10000
+
+
+Benchmark Name: KMeansModel-1
+Total Execution Time: 215.0 ms
+Total Input Record Number: 10000
+Average Input Throughput: 46511.62790697674 events per second
+Total Output Record Number: 10000
+Average Output Throughput: 46511.62790697674 events per second
+
+```
+
+The command above would save the results into `./output/results.json` as below.
+
+```json
+[ {
+  "name" : "KMeansModel-1",
+  "totalTimeMs" : 215.0,
+  "inputRecordNum" : 10000,
+  "inputThroughput" : 46511.62790697674,
+  "outputRecordNum" : 10000,
+  "outputThroughput" : 46511.62790697674
+} ]
+```
+
+## Custom Benchmark Configuration File
+
+`flink-ml-benchmark.sh` parses benchmarks to be executed according to the input
+configuration file, like `./examples/benchmark-example-conf.json`. It can also
+parse your custom configuration file so long as it contains a JSON object in the
+following format.
+
+- The file should contain the following as the metadata of the JSON object.
+  - `"version"`: The version of the json format. Currently its value must be 1.
+- Keys in the JSON object, except `"version"`, are regarded as the names of the
+  benchmarks.
+- The value of each benchmark name should be a JSON object containing the
+  following keys.
+  - `"stage"`: The stage to be benchmarked.
+  - `"inputs"`: The input data of the stage to be benchmarked.
+  - `"modelData"`(Optional): The model data of the stage to be benchmarked, if
+    the stage is a `Model` and needs to have its model data explicitly set.
+- The value of `"stage"`, `"inputs"` or `"modelData"` should be a JSON object
+  containing the following keys.
+  - `"className"`: The full classpath of a `WithParams` subclass. For `"stage"`,
+    the class should be a subclass of `Stage`. For `"inputs"` or `"modelData"`,
+    the class should be a subclass of `DataGenerator`.
+  - `"paramMap"`: A JSON object containing the parameters related to the
+    specific `Stage` or `DataGenerator`.
+
+Combining the format requirements above, an example configuration file is as

Review comment:
       In addition to explaining the syntax of the config file, could we also explain the semantics of this configuration file, so that users understand what this quickstart actually runs?

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/generator/GeneratorUtils.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.flink.ml.benchmark.generator;
+
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.linalg.Vectors;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.util.NumberSequenceIterator;
+
+import java.util.Random;
+
+/** Utility methods to generate data for benchmarks. */
+public class GeneratorUtils {
+    /**
+     * Generates random continuous vectors.
+     *
+     * @param env The stream execution environment.
+     * @param numData Number of examples to generate in total.
+     * @param seed The seed to generate seed on each partition.
+     * @param dims Dimension of the vectors to be generated.
+     * @return The generated vector stream.
+     */
+    public static DataStream<DenseVector> generateRandomContinuousVectorStream(
+            StreamExecutionEnvironment env, long numData, long seed, int dims) {
+        return env.fromParallelCollection(
+                        new NumberSequenceIterator(1L, numData), BasicTypeInfo.LONG_TYPE_INFO)
+                .map(new GenerateRandomContinuousVectorFunction(seed, dims));
+    }
+
+    private static class GenerateRandomContinuousVectorFunction
+            extends RichMapFunction<Long, DenseVector> {
+        private final int dims;
+        private final long initSeed;
+        private Random random;
+
+        private GenerateRandomContinuousVectorFunction(long initSeed, int dims) {
+            this.dims = dims;
+            this.initSeed = initSeed;
+        }
+
+        @Override
+        public void open(Configuration parameters) throws Exception {
+            super.open(parameters);
+            int index = getRuntimeContext().getIndexOfThisSubtask();
+            random = new Random(Tuple2.of(initSeed, index).hashCode());
+        }
+
+        @Override
+        public DenseVector map(Long value) {
+            double[] values = new double[dims];
+            for (int i = 0; i < dims; i++) {
+                values[i] = random.nextDouble();
+            }
+            return Vectors.dense(values);
+        }
+    }
+
+    /**
+     * Generates random continuous vector arrays.
+     *
+     * @param env The stream execution environment.
+     * @param numData Number of examples to generate in total.
+     * @param arraySize Size of the vector array.
+     * @param seed The seed to generate seed on each partition.
+     * @param dims Dimension of the vectors to be generated.
+     * @return The generated vector stream.
+     */
+    public static DataStream<DenseVector[]> generateRandomContinuousVectorArrayStream(

Review comment:
       How about renaming this method as `getRandomDenseVectorArrays(...)`.

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/generator/GeneratorParams.java
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.flink.ml.benchmark.generator;
+
+import org.apache.flink.ml.common.param.HasSeed;
+import org.apache.flink.ml.param.IntParam;
+import org.apache.flink.ml.param.LongParam;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+
+/** Interface for the generator params. */
+public interface GeneratorParams<T> extends HasSeed<T> {
+    Param<Long> NUM_DATA =
+            new LongParam("numData", "Number of data to be generated.", 10L, ParamValidators.gt(0));
+
+    Param<Integer> DIMS =
+            new IntParam(
+                    "dims",

Review comment:
       Since this is dim is only useful for vector-typed data, how about renaming it as `vectorDim`?

##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,169 @@
+# Flink ML Benchmark Getting Started
+
+This document provides instructions about how to run benchmarks on Flink ML's
+stages in a Linux/MacOS environment.
+
+## Prerequisites
+
+### Installing Flink
+
+Please make sure Flink 1.14 or higher version has been installed in your local
+environment. You can refer to the [local
+installation](https://nightlies.apache.org/flink/flink-docs-master/docs/try-flink/local_installation/)
+instruction on Flink's document website for how to achieve this.
+
+### Setting Up Flink Environment Variables
+
+After having installed Flink, please register `$FLINK_HOME` as an environment
+variable into your local environment, and add `$FLINK_HOME` into your `$PATH`
+variable. This can be completed by running the following commands in the Flink's
+folder.
+
+```bash
+export FLINK_HOME=`pwd`
+export PATH=$FLINK_HOME/bin:$PATH
+```
+
+Then please run the following command. If this command returns 1.14.0 or a
+higher version, then it means that the required Flink environment has been
+successfully installed and registered in your local environment.
+
+```bash
+flink --version
+```
+
+### Acquiring Flink ML Binary Distribution
+
+In order to use Flink ML's CLI you need to have the latest binary distribution
+of Flink ML. You can acquire the distribution by building Flink ML's source code
+locally, which means to execute the following command in Flink ML repository's
+root directory.
+
+```bash
+$ mvn clean package -DskipTests

Review comment:
       How about asking user to copy/paste the following command:
   
   `cd ./flink-ml-dist/target/flink-ml-*-bin/flink-ml*/`
   
   The benefit of this approach is that users can just copy/paste the commands specified in the quickstart to complete the quickstart.

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/Benchmark.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** Entry class for benchmark execution. */
+public class Benchmark {
+    private static final Logger LOG = LoggerFactory.getLogger(Benchmark.class);
+
+    @SuppressWarnings("unchecked")
+    public static void main(String[] args) throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        InputStream inputStream = new FileInputStream(args[0]);
+        Map<String, ?> jsonMap = ReadWriteUtils.OBJECT_MAPPER.readValue(inputStream, Map.class);
+        Preconditions.checkArgument(
+                jsonMap.containsKey("version") && jsonMap.get("version").equals(1));
+
+        List<String> benchmarkNames =
+                jsonMap.keySet().stream()
+                        .filter(x -> !x.equals("version"))
+                        .collect(Collectors.toList());
+        LOG.info("Found benchmarks " + benchmarkNames);
+
+        List<BenchmarkResult> results = new ArrayList<>();
+
+        for (String benchmarkName : benchmarkNames) {
+            LOG.info("Running benchmark " + benchmarkName + ".");
+
+            BenchmarkResult result =
+                    BenchmarkUtils.runBenchmark(
+                            tEnv, benchmarkName, (Map<String, ?>) jsonMap.get(benchmarkName));
+
+            results.add(result);
+        }
+
+        for (BenchmarkResult result : results) {
+            BenchmarkUtils.printResult(result);
+        }
+
+        if (args.length > 1) {
+            String savePath = args[1];
+            BenchmarkUtils.saveResultsAsJson(savePath, results);

Review comment:
       Would to be useful to print the benchmark result summary to the console if user has not specified an output file path?

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/BenchmarkResult.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/** The result of executing a benchmark. */
+public class BenchmarkResult {
+    /** The name of the benchmark. */

Review comment:
       How about `The benchmark name`?

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/BenchmarkResult.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/** The result of executing a benchmark. */
+public class BenchmarkResult {
+    /** The name of the benchmark. */
+    public String name;
+
+    /** The total execution time of the benchmark flink job. Unit: milliseconds */
+    public Double totalTimeMs;
+
+    /** The total number of records input into the benchmark flink job. */
+    public Long inputRecordNum;
+
+    /**
+     * The average input throughput of the benchmark flink job. Unit: number of records processed
+     * per second
+     */
+    public Double inputThroughput;
+
+    /** The total number of records output from the benchmark flink job. */

Review comment:
       How about `The total number of output records`?

##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,169 @@
+# Flink ML Benchmark Getting Started
+
+This document provides instructions about how to run benchmarks on Flink ML's
+stages in a Linux/MacOS environment.
+
+## Prerequisites
+
+### Installing Flink
+
+Please make sure Flink 1.14 or higher version has been installed in your local
+environment. You can refer to the [local
+installation](https://nightlies.apache.org/flink/flink-docs-master/docs/try-flink/local_installation/)
+instruction on Flink's document website for how to achieve this.
+
+### Setting Up Flink Environment Variables
+
+After having installed Flink, please register `$FLINK_HOME` as an environment
+variable into your local environment, and add `$FLINK_HOME` into your `$PATH`
+variable. This can be completed by running the following commands in the Flink's
+folder.
+
+```bash
+export FLINK_HOME=`pwd`

Review comment:
       How do we guarantee that the `pwd` refers to the Flink directory? Could we change the wiki and the command to something that guarantees correctness?

##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,169 @@
+# Flink ML Benchmark Getting Started
+
+This document provides instructions about how to run benchmarks on Flink ML's
+stages in a Linux/MacOS environment.
+
+## Prerequisites
+
+### Installing Flink
+
+Please make sure Flink 1.14 or higher version has been installed in your local
+environment. You can refer to the [local
+installation](https://nightlies.apache.org/flink/flink-docs-master/docs/try-flink/local_installation/)
+instruction on Flink's document website for how to achieve this.
+
+### Setting Up Flink Environment Variables
+
+After having installed Flink, please register `$FLINK_HOME` as an environment
+variable into your local environment, and add `$FLINK_HOME` into your `$PATH`
+variable. This can be completed by running the following commands in the Flink's
+folder.
+
+```bash
+export FLINK_HOME=`pwd`
+export PATH=$FLINK_HOME/bin:$PATH
+```
+
+Then please run the following command. If this command returns 1.14.0 or a
+higher version, then it means that the required Flink environment has been
+successfully installed and registered in your local environment.
+
+```bash
+flink --version
+```
+
+### Acquiring Flink ML Binary Distribution
+
+In order to use Flink ML's CLI you need to have the latest binary distribution
+of Flink ML. You can acquire the distribution by building Flink ML's source code
+locally, which means to execute the following command in Flink ML repository's
+root directory.
+
+```bash
+$ mvn clean package -DskipTests
+```
+
+After executing the command above, you will be able to find the binary
+distribution under
+`./flink-ml-dist/target/flink-ml-<version>-bin/flink-ml-<version>/`.
+
+### Starting Flink Cluster
+
+Please start a Flink standalone session in your local environment with the
+following command.
+
+```bash
+start-cluster.sh
+```
+
+You should be able to navigate to the web UI at
+[localhost:8081](http://localhost:8081/) to view the Flink dashboard and see
+that the cluster is up and running.
+
+## Run Benchmark Example
+
+In Flink ML's binary distribution's folder, execute the following command to run
+an example benchmark.
+
+```bash
+$ ./bin/flink-ml-benchmark.sh ./examples/benchmark-example-conf.json ./output/results.json
+```
+
+You will notice that some Flink job is submitted to your Flink cluster, and the
+following information is printed out in your terminal. This means that you have
+successfully executed a benchmark on `KMeansModel`.
+
+```
+Creating fat jar containing all flink ml dependencies to be submitted.
+Job has been submitted with JobID bdaa54b065adf2c813619113a00337de
+Program execution finished
+Job with JobID bdaa54b065adf2c813619113a00337de has finished.
+Job Runtime: 215 ms
+Accumulator Results: 
+- numElements (java.lang.Long): 10000
+
+
+Benchmark Name: KMeansModel-1
+Total Execution Time: 215.0 ms
+Total Input Record Number: 10000
+Average Input Throughput: 46511.62790697674 events per second

Review comment:
       Could we limits the number of digits after `.` to `2` so that the result is more readable?

##########
File path: flink-ml-benchmark/src/test/java/org/apache/flink/ml/benchmark/BenchmarkTest.java
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.ml.benchmark.clustering.kmeans.KMeansInputsGenerator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+import org.apache.flink.ml.clustering.kmeans.KMeansModel;
+import org.apache.flink.ml.param.WithParams;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.test.util.AbstractTestBase;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.InputStream;
+import java.io.PrintStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.ml.util.ReadWriteUtils.OBJECT_MAPPER;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/** Tests benchmarks. */
+@SuppressWarnings("unchecked")
+public class BenchmarkTest extends AbstractTestBase {
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+
+    private final ByteArrayOutputStream outContent = new ByteArrayOutputStream();
+    private final PrintStream originalOut = System.out;
+
+    @Before
+    public void before() {
+        System.setOut(new PrintStream(outContent));
+    }
+
+    @After
+    public void after() {
+        System.setOut(originalOut);

Review comment:
       Would it be simpler to do `System.setOut(System.out)` and remove the `originalOut` variable?

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/Benchmark.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** Entry class for benchmark execution. */
+public class Benchmark {
+    private static final Logger LOG = LoggerFactory.getLogger(Benchmark.class);
+
+    @SuppressWarnings("unchecked")
+    public static void main(String[] args) throws Exception {

Review comment:
       Could we also support `./bin/flink-ml-benchmark.sh --help` print helper message? And if user calls ``./bin/flink-ml-benchmark.sh` without providing any argument, it might also be useful to print the helper message instead of throwing exception.

##########
File path: flink-ml-core/src/main/java/org/apache/flink/ml/util/ReadWriteUtils.java
##########
@@ -94,25 +95,29 @@
      */
     public static void saveMetadata(Stage<?> stage, String path, Map<String, ?> extraMetadata)
             throws IOException {
-        // Creates parent directories if not already created.
-        FileSystem fs = mkdirs(path);
-
         Map<String, Object> metadata = new HashMap<>(extraMetadata);
         metadata.put("className", stage.getClass().getName());
         metadata.put("timestamp", System.currentTimeMillis());
         metadata.put("paramMap", jsonEncode(stage.getParamMap()));
         // TODO: add version in the metadata.
         String metadataStr = OBJECT_MAPPER.writeValueAsString(metadata);
 
-        Path metadataPath = new Path(path, "metadata");
-        if (fs.exists(metadataPath)) {
-            throw new IOException("File " + metadataPath + " already exists.");
+        saveToFile(new Path(path, "metadata"), metadataStr);
+    }
+
+    /** Saves a given string to the specified file. */
+    public static void saveToFile(Path path, String content) throws IOException {
+        // Creates parent directories if not already created.
+        FileSystem fs = mkdirs(path.getParent().toString());
+
+        if (fs.exists(path)) {
+            throw new IOException("File " + path + " already exists.");

Review comment:
       When user explicitly specifies the output file for the benchmark execution, if the file already exists, the file is typically overwritten. Could we follow this convention?

##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,169 @@
+# Flink ML Benchmark Getting Started
+
+This document provides instructions about how to run benchmarks on Flink ML's
+stages in a Linux/MacOS environment.
+
+## Prerequisites
+
+### Installing Flink
+
+Please make sure Flink 1.14 or higher version has been installed in your local
+environment. You can refer to the [local
+installation](https://nightlies.apache.org/flink/flink-docs-master/docs/try-flink/local_installation/)
+instruction on Flink's document website for how to achieve this.
+
+### Setting Up Flink Environment Variables
+
+After having installed Flink, please register `$FLINK_HOME` as an environment
+variable into your local environment, and add `$FLINK_HOME` into your `$PATH`
+variable. This can be completed by running the following commands in the Flink's
+folder.
+
+```bash
+export FLINK_HOME=`pwd`
+export PATH=$FLINK_HOME/bin:$PATH
+```
+
+Then please run the following command. If this command returns 1.14.0 or a
+higher version, then it means that the required Flink environment has been
+successfully installed and registered in your local environment.
+
+```bash
+flink --version
+```
+
+### Acquiring Flink ML Binary Distribution
+
+In order to use Flink ML's CLI you need to have the latest binary distribution
+of Flink ML. You can acquire the distribution by building Flink ML's source code
+locally, which means to execute the following command in Flink ML repository's
+root directory.
+
+```bash
+$ mvn clean package -DskipTests
+```
+
+After executing the command above, you will be able to find the binary
+distribution under
+`./flink-ml-dist/target/flink-ml-<version>-bin/flink-ml-<version>/`.
+
+### Starting Flink Cluster
+
+Please start a Flink standalone session in your local environment with the
+following command.
+
+```bash
+start-cluster.sh
+```
+
+You should be able to navigate to the web UI at
+[localhost:8081](http://localhost:8081/) to view the Flink dashboard and see
+that the cluster is up and running.
+
+## Run Benchmark Example
+
+In Flink ML's binary distribution's folder, execute the following command to run
+an example benchmark.
+
+```bash
+$ ./bin/flink-ml-benchmark.sh ./examples/benchmark-example-conf.json ./output/results.json
+```
+
+You will notice that some Flink job is submitted to your Flink cluster, and the
+following information is printed out in your terminal. This means that you have
+successfully executed a benchmark on `KMeansModel`.
+
+```
+Creating fat jar containing all flink ml dependencies to be submitted.
+Job has been submitted with JobID bdaa54b065adf2c813619113a00337de
+Program execution finished
+Job with JobID bdaa54b065adf2c813619113a00337de has finished.
+Job Runtime: 215 ms
+Accumulator Results: 
+- numElements (java.lang.Long): 10000
+
+
+Benchmark Name: KMeansModel-1
+Total Execution Time: 215.0 ms
+Total Input Record Number: 10000
+Average Input Throughput: 46511.62790697674 events per second
+Total Output Record Number: 10000
+Average Output Throughput: 46511.62790697674 events per second
+
+```
+
+The command above would save the results into `./output/results.json` as below.
+
+```json
+[ {
+  "name" : "KMeansModel-1",
+  "totalTimeMs" : 215.0,
+  "inputRecordNum" : 10000,
+  "inputThroughput" : 46511.62790697674,
+  "outputRecordNum" : 10000,
+  "outputThroughput" : 46511.62790697674
+} ]
+```
+
+## Custom Benchmark Configuration File

Review comment:
       How about changing it to `Customize Benchmark Configuration`

##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,169 @@
+# Flink ML Benchmark Getting Started
+
+This document provides instructions about how to run benchmarks on Flink ML's
+stages in a Linux/MacOS environment.
+
+## Prerequisites
+
+### Installing Flink
+
+Please make sure Flink 1.14 or higher version has been installed in your local
+environment. You can refer to the [local
+installation](https://nightlies.apache.org/flink/flink-docs-master/docs/try-flink/local_installation/)
+instruction on Flink's document website for how to achieve this.
+
+### Setting Up Flink Environment Variables
+
+After having installed Flink, please register `$FLINK_HOME` as an environment
+variable into your local environment, and add `$FLINK_HOME` into your `$PATH`
+variable. This can be completed by running the following commands in the Flink's
+folder.
+
+```bash
+export FLINK_HOME=`pwd`
+export PATH=$FLINK_HOME/bin:$PATH

Review comment:
       Instead of asking user to change $PATH, could we check for the existence of the `FLINK_HOME` variable and use this variable explicitly in the `./bin/flink-ml-benchmark.sh` script?
   
   The benefit of this approach is that there is one less variable that users need to explicitly change to complete the quickstart.
   

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/generator/GeneratorUtils.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.flink.ml.benchmark.generator;
+
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.linalg.Vectors;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.util.NumberSequenceIterator;
+
+import java.util.Random;
+
+/** Utility methods to generate data for benchmarks. */
+public class GeneratorUtils {
+    /**
+     * Generates random continuous vectors.
+     *
+     * @param env The stream execution environment.
+     * @param numData Number of examples to generate in total.
+     * @param seed The seed to generate seed on each partition.
+     * @param dims Dimension of the vectors to be generated.
+     * @return The generated vector stream.
+     */
+    public static DataStream<DenseVector> generateRandomContinuousVectorStream(

Review comment:
       Do we need to have `continuous` in the name? Is it because we will have `discreteVectorStream` in the future?
   
   Given that we already have `DataGenerator::getData()` API, it is probably more consistent to replace `generator` with `get`.
   
   How about renaming this method as `getRandomDenseVectors(...)`?
   
   And since `DataGenerator::getData()` returns `Table[]`. Should this method return `Table` instead of `DataStream<...>`?

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/generator/GeneratorUtils.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.flink.ml.benchmark.generator;
+
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.linalg.Vectors;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.util.NumberSequenceIterator;
+
+import java.util.Random;
+
+/** Utility methods to generate data for benchmarks. */
+public class GeneratorUtils {

Review comment:
       Given that we already have the `DataGenerator` interface, how about renaming this class as `DataGeneratorUtils`?

##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,169 @@
+# Flink ML Benchmark Getting Started
+
+This document provides instructions about how to run benchmarks on Flink ML's
+stages in a Linux/MacOS environment.
+
+## Prerequisites
+
+### Installing Flink
+
+Please make sure Flink 1.14 or higher version has been installed in your local
+environment. You can refer to the [local
+installation](https://nightlies.apache.org/flink/flink-docs-master/docs/try-flink/local_installation/)
+instruction on Flink's document website for how to achieve this.
+
+### Setting Up Flink Environment Variables
+
+After having installed Flink, please register `$FLINK_HOME` as an environment
+variable into your local environment, and add `$FLINK_HOME` into your `$PATH`
+variable. This can be completed by running the following commands in the Flink's
+folder.
+
+```bash
+export FLINK_HOME=`pwd`
+export PATH=$FLINK_HOME/bin:$PATH
+```
+
+Then please run the following command. If this command returns 1.14.0 or a
+higher version, then it means that the required Flink environment has been
+successfully installed and registered in your local environment.
+
+```bash
+flink --version
+```
+
+### Acquiring Flink ML Binary Distribution
+
+In order to use Flink ML's CLI you need to have the latest binary distribution
+of Flink ML. You can acquire the distribution by building Flink ML's source code
+locally, which means to execute the following command in Flink ML repository's
+root directory.
+
+```bash
+$ mvn clean package -DskipTests
+```
+
+After executing the command above, you will be able to find the binary
+distribution under
+`./flink-ml-dist/target/flink-ml-<version>-bin/flink-ml-<version>/`.
+
+### Starting Flink Cluster
+
+Please start a Flink standalone session in your local environment with the
+following command.
+
+```bash
+start-cluster.sh

Review comment:
       How about asking user to copy/paste the following command so that the meaning of this script (e.g. where it is located) is more explicit?
   
   `$FLINK_HOME/bin/start-cluster.sh`

##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,169 @@
+# Flink ML Benchmark Getting Started
+
+This document provides instructions about how to run benchmarks on Flink ML's
+stages in a Linux/MacOS environment.
+
+## Prerequisites
+
+### Installing Flink

Review comment:
       Given that we already use `Run Benchmark Example` instead of `Running Benchmark Example`, could we change this to `Install Flink`?
   
   IMO `Install Flink` seems more appropriate than `Installing Flink` as the section name for the quickstart. Kafka's doc follows this approach https://kafka.apache.org/quickstart.

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/clustering/kmeans/KMeansInputsGenerator.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.flink.ml.benchmark.clustering.kmeans;
+
+import org.apache.flink.ml.benchmark.generator.DataGenerator;
+import org.apache.flink.ml.benchmark.generator.GeneratorUtils;
+import org.apache.flink.ml.clustering.kmeans.KMeansParams;
+import org.apache.flink.ml.common.datastream.TableUtils;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Class that generates table arrays containing inputs for {@link
+ * org.apache.flink.ml.clustering.kmeans.KMeans} and {@link
+ * org.apache.flink.ml.clustering.kmeans.KMeansModel}.
+ */
+public class KMeansInputsGenerator

Review comment:
       How about we rename this class as `DenseVectorGenerator` so that this class could be re-used for other algorithms whose input is a stream of dense vectors?

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/generator/GeneratorParams.java
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.flink.ml.benchmark.generator;
+
+import org.apache.flink.ml.common.param.HasSeed;
+import org.apache.flink.ml.param.IntParam;
+import org.apache.flink.ml.param.LongParam;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+
+/** Interface for the generator params. */
+public interface GeneratorParams<T> extends HasSeed<T> {

Review comment:
       This interface will contain the parameters used by all data generators (e.g. numData), instead all possible parameters (e.g. `dims`), right?
   
   Note that `dims` won't be useful for non-vector-typed data. It seems a bit weird to call `getDims()` on a data generator that generates int values.
   
   Given that we already have the `DataGenerator` interface, would it be more consistent to rename this class as `CommonDataGeneratorParams`?

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/generator/GeneratorParams.java
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.flink.ml.benchmark.generator;
+
+import org.apache.flink.ml.common.param.HasSeed;
+import org.apache.flink.ml.param.IntParam;
+import org.apache.flink.ml.param.LongParam;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+
+/** Interface for the generator params. */
+public interface GeneratorParams<T> extends HasSeed<T> {
+    Param<Long> NUM_DATA =
+            new LongParam("numData", "Number of data to be generated.", 10L, ParamValidators.gt(0));

Review comment:
       Would `numValues` be more intuitive?

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/generator/GeneratorParams.java
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.flink.ml.benchmark.generator;
+
+import org.apache.flink.ml.common.param.HasSeed;
+import org.apache.flink.ml.param.IntParam;
+import org.apache.flink.ml.param.LongParam;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+
+/** Interface for the generator params. */
+public interface GeneratorParams<T> extends HasSeed<T> {
+    Param<Long> NUM_DATA =
+            new LongParam("numData", "Number of data to be generated.", 10L, ParamValidators.gt(0));
+
+    Param<Integer> DIMS =
+            new IntParam(
+                    "dims",
+                    "Dimension of vector-typed data to be generated.",
+                    1,
+                    ParamValidators.gt(0));
+
+    default long getNumData() {
+        return get(NUM_DATA);
+    }
+
+    default T setNumData(long value) {
+        return set(NUM_DATA, value);
+    }
+
+    default int getDims() {

Review comment:
       It would be useful for this benchmark tool to also support data generators developers outside the Flink ML repo. This means that this file won't be able to contain all possible data generators' parameters. 
   
   Therefore it seems better to move the parameters needed only by a subset of data generators from this class to the constructor of selected generators.
   
   

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/clustering/kmeans/KMeansModelDataGenerator.java
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.flink.ml.benchmark.clustering.kmeans;
+
+import org.apache.flink.ml.benchmark.generator.DataGenerator;
+import org.apache.flink.ml.benchmark.generator.GeneratorUtils;
+import org.apache.flink.ml.clustering.kmeans.KMeansModelData;
+import org.apache.flink.ml.clustering.kmeans.KMeansParams;
+import org.apache.flink.ml.common.datastream.TableUtils;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Class that generates table arrays containing model data for {@link
+ * org.apache.flink.ml.clustering.kmeans.KMeansModel}.
+ */
+public class KMeansModelDataGenerator

Review comment:
       How about we rename this class as `DenseVectorArrayGenerator` so that this class could be re-used for other algorithms whose input is a stream of dense vector arrays?

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/Benchmark.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** Entry class for benchmark execution. */
+public class Benchmark {
+    private static final Logger LOG = LoggerFactory.getLogger(Benchmark.class);
+
+    @SuppressWarnings("unchecked")
+    public static void main(String[] args) throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        InputStream inputStream = new FileInputStream(args[0]);
+        Map<String, ?> jsonMap = ReadWriteUtils.OBJECT_MAPPER.readValue(inputStream, Map.class);
+        Preconditions.checkArgument(
+                jsonMap.containsKey("version") && jsonMap.get("version").equals(1));
+
+        List<String> benchmarkNames =
+                jsonMap.keySet().stream()
+                        .filter(x -> !x.equals("version"))
+                        .collect(Collectors.toList());
+        LOG.info("Found benchmarks " + benchmarkNames);
+
+        List<BenchmarkResult> results = new ArrayList<>();
+
+        for (String benchmarkName : benchmarkNames) {
+            LOG.info("Running benchmark " + benchmarkName + ".");
+
+            BenchmarkResult result =
+                    BenchmarkUtils.runBenchmark(
+                            tEnv, benchmarkName, (Map<String, ?>) jsonMap.get(benchmarkName));
+
+            results.add(result);
+        }
+
+        for (BenchmarkResult result : results) {
+            BenchmarkUtils.printResult(result);

Review comment:
       Would it be useful to print the result in the loop body so that the result could be presented to the user in a more interactive way?

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/Benchmark.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** Entry class for benchmark execution. */
+public class Benchmark {
+    private static final Logger LOG = LoggerFactory.getLogger(Benchmark.class);
+
+    @SuppressWarnings("unchecked")
+    public static void main(String[] args) throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        InputStream inputStream = new FileInputStream(args[0]);
+        Map<String, ?> jsonMap = ReadWriteUtils.OBJECT_MAPPER.readValue(inputStream, Map.class);
+        Preconditions.checkArgument(
+                jsonMap.containsKey("version") && jsonMap.get("version").equals(1));

Review comment:
       Would we define version as a private static final variable?

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/BenchmarkResult.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/** The result of executing a benchmark. */
+public class BenchmarkResult {
+    /** The name of the benchmark. */
+    public String name;
+
+    /** The total execution time of the benchmark flink job. Unit: milliseconds */

Review comment:
       How about `The total execution time of the benchmark in milliseconds`?

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/BenchmarkResult.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/** The result of executing a benchmark. */
+public class BenchmarkResult {
+    /** The name of the benchmark. */
+    public String name;
+
+    /** The total execution time of the benchmark flink job. Unit: milliseconds */
+    public Double totalTimeMs;
+
+    /** The total number of records input into the benchmark flink job. */

Review comment:
       It seems unnecessary to specify `benchmark flink job` here. How about `The total number of input records`?

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/BenchmarkResult.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/** The result of executing a benchmark. */
+public class BenchmarkResult {
+    /** The name of the benchmark. */
+    public String name;
+
+    /** The total execution time of the benchmark flink job. Unit: milliseconds */
+    public Double totalTimeMs;
+
+    /** The total number of records input into the benchmark flink job. */
+    public Long inputRecordNum;
+
+    /**
+     * The average input throughput of the benchmark flink job. Unit: number of records processed

Review comment:
       Would it be simpler to use `The average input throughput in number of records per second`?

##########
File path: flink-ml-benchmark/src/test/java/org/apache/flink/ml/benchmark/BenchmarkTest.java
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.ml.benchmark.clustering.kmeans.KMeansInputsGenerator;
+import org.apache.flink.ml.clustering.kmeans.KMeans;
+import org.apache.flink.ml.clustering.kmeans.KMeansModel;
+import org.apache.flink.ml.param.WithParams;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.test.util.AbstractTestBase;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.InputStream;
+import java.io.PrintStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.ml.util.ReadWriteUtils.OBJECT_MAPPER;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/** Tests benchmarks. */
+@SuppressWarnings("unchecked")
+public class BenchmarkTest extends AbstractTestBase {
+    @Rule public final TemporaryFolder tempFolder = new TemporaryFolder();
+
+    private final ByteArrayOutputStream outContent = new ByteArrayOutputStream();

Review comment:
       Would it be simpler to rename `outContent` as `outputStream`?

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/BenchmarkUtils.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.ml.api.AlgoOperator;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.api.Model;
+import org.apache.flink.ml.api.Stage;
+import org.apache.flink.ml.benchmark.generator.DataGenerator;
+import org.apache.flink.ml.common.datastream.TableUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/** Utility methods for benchmarks. */
+public class BenchmarkUtils {
+    /**
+     * Instantiates a benchmark from its parameter map and executes the benchmark in the provided
+     * environment.
+     *
+     * @return Results of the executed benchmark.
+     */
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    public static BenchmarkResult runBenchmark(
+            StreamTableEnvironment tEnv, String name, Map<String, ?> params) throws Exception {
+        Stage stage = ReadWriteUtils.instantiateWithParams((Map<String, ?>) params.get("stage"));
+        DataGenerator inputsGenerator =
+                ReadWriteUtils.instantiateWithParams((Map<String, ?>) params.get("inputs"));
+        DataGenerator modelDataGenerator = null;
+        if (params.containsKey("modelData")) {
+            modelDataGenerator =
+                    ReadWriteUtils.instantiateWithParams((Map<String, ?>) params.get("modelData"));
+        }
+
+        return runBenchmark(tEnv, name, stage, inputsGenerator, modelDataGenerator);
+    }
+
+    /**
+     * Executes a benchmark from a stage with its inputsGenerator in the provided environment.
+     *
+     * @return Results of the executed benchmark.
+     */
+    public static BenchmarkResult runBenchmark(
+            StreamTableEnvironment tEnv,
+            String name,
+            Stage<?> stage,
+            DataGenerator<?> inputsGenerator)
+            throws Exception {
+        return runBenchmark(tEnv, name, stage, inputsGenerator, null);
+    }
+
+    /**
+     * Executes a benchmark from a stage with its inputsGenerator and modelDataGenerator in the
+     * provided environment.
+     *
+     * @return Results of the executed benchmark.
+     */
+    public static BenchmarkResult runBenchmark(
+            StreamTableEnvironment tEnv,
+            String name,
+            Stage<?> stage,
+            DataGenerator<?> inputsGenerator,
+            DataGenerator<?> modelDataGenerator)
+            throws Exception {
+        StreamExecutionEnvironment env = TableUtils.getExecutionEnvironment(tEnv);
+
+        Table[] inputTables = inputsGenerator.getData(tEnv);
+        if (modelDataGenerator != null) {
+            ((Model<?>) stage).setModelData(modelDataGenerator.getData(tEnv));
+        }
+
+        Table[] outputTables;
+        if (stage instanceof Estimator) {
+            outputTables = ((Estimator<?, ?>) stage).fit(inputTables).getModelData();
+        } else if (stage instanceof AlgoOperator) {
+            outputTables = ((AlgoOperator<?>) stage).transform(inputTables);
+        } else {
+            throw new IllegalArgumentException("Unsupported Stage class " + stage.getClass());
+        }
+
+        for (Table table : outputTables) {
+            tEnv.toDataStream(table).addSink(new CountingAndDiscardingSink<>());
+        }
+
+        JobExecutionResult executionResult = env.execute();
+
+        BenchmarkResult result = new BenchmarkResult();
+        result.name = name;
+        result.totalTimeMs = (double) executionResult.getNetRuntime(TimeUnit.MILLISECONDS);
+        result.inputRecordNum = inputsGenerator.getNumData();
+        result.inputThroughput = result.inputRecordNum * 1000.0 / result.totalTimeMs;
+        result.outputRecordNum =
+                executionResult.getAccumulatorResult(CountingAndDiscardingSink.COUNTER_NAME);
+        result.outputThroughput = result.outputRecordNum * 1000.0 / result.totalTimeMs;
+
+        return result;
+    }
+
+    /** Prints out the provided benchmark result. */
+    public static void printResult(BenchmarkResult result) {

Review comment:
       I personally find that the output from `saveResultsAsJson(...)` to be more compact and readable than the strings printed by this method. Thus I am wondering if it is simpler to remove this method. What do you think?

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/BenchmarkResult.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/** The result of executing a benchmark. */
+public class BenchmarkResult {
+    /** The name of the benchmark. */
+    public String name;
+
+    /** The total execution time of the benchmark flink job. Unit: milliseconds */
+    public Double totalTimeMs;
+
+    /** The total number of records input into the benchmark flink job. */
+    public Long inputRecordNum;
+
+    /**
+     * The average input throughput of the benchmark flink job. Unit: number of records processed
+     * per second
+     */
+    public Double inputThroughput;
+
+    /** The total number of records output from the benchmark flink job. */
+    public Long outputRecordNum;
+
+    /**
+     * The average output throughput of the benchmark flink job. Unit: number of records processed

Review comment:
       Would it be simpler to use `The average output throughput in number of records per second`?

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/BenchmarkUtils.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.ml.api.AlgoOperator;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.api.Model;
+import org.apache.flink.ml.api.Stage;
+import org.apache.flink.ml.benchmark.generator.DataGenerator;
+import org.apache.flink.ml.common.datastream.TableUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/** Utility methods for benchmarks. */
+public class BenchmarkUtils {
+    /**
+     * Instantiates a benchmark from its parameter map and executes the benchmark in the provided
+     * environment.
+     *
+     * @return Results of the executed benchmark.
+     */
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    public static BenchmarkResult runBenchmark(
+            StreamTableEnvironment tEnv, String name, Map<String, ?> params) throws Exception {
+        Stage stage = ReadWriteUtils.instantiateWithParams((Map<String, ?>) params.get("stage"));

Review comment:
       Currently we require every parameter of the DataGenerator to be defined as a `Param<...>`.
   
   We need algorithms parameters to be defined as `Param<...>` because `save()` and `load()` needs to handle those parameters properly. But there is no such need for data generators.
   
   Would it be simpler to just pass the parameter map to the data generator constructor?
   




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r840348963



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/Benchmark.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** Entry class for benchmark execution. */
+public class Benchmark {
+    private static final Logger LOG = LoggerFactory.getLogger(Benchmark.class);
+
+    @SuppressWarnings("unchecked")
+    public static void main(String[] args) throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        InputStream inputStream = new FileInputStream(args[0]);
+        Map<String, ?> jsonMap = ReadWriteUtils.OBJECT_MAPPER.readValue(inputStream, Map.class);
+        Preconditions.checkArgument(
+                jsonMap.containsKey("version") && jsonMap.get("version").equals(1));
+
+        List<String> benchmarkNames =
+                jsonMap.keySet().stream()
+                        .filter(x -> !x.equals("version"))
+                        .collect(Collectors.toList());
+        LOG.info("Found benchmarks " + benchmarkNames);
+
+        List<BenchmarkResult> results = new ArrayList<>();
+
+        for (String benchmarkName : benchmarkNames) {
+            LOG.info("Running benchmark " + benchmarkName + ".");
+
+            BenchmarkResult result =
+                    BenchmarkUtils.runBenchmark(
+                            tEnv, benchmarkName, (Map<String, ?>) jsonMap.get(benchmarkName));
+
+            results.add(result);
+        }
+
+        for (BenchmarkResult result : results) {
+            BenchmarkUtils.printResult(result);
+        }
+
+        if (args.length > 1) {
+            String savePath = args[1];
+            BenchmarkUtils.saveResultsAsJson(savePath, results);

Review comment:
       I agree. I'll print the results in the main loop body as in the comment above.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r840351792



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/BenchmarkUtils.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.ml.api.AlgoOperator;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.api.Model;
+import org.apache.flink.ml.api.Stage;
+import org.apache.flink.ml.benchmark.generator.DataGenerator;
+import org.apache.flink.ml.common.datastream.TableUtils;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/** Utility methods for benchmarks. */
+public class BenchmarkUtils {
+    /**
+     * Instantiates a benchmark from its parameter map and executes the benchmark in the provided
+     * environment.
+     *
+     * @return Results of the executed benchmark.
+     */
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    public static BenchmarkResult runBenchmark(
+            StreamTableEnvironment tEnv, String name, Map<String, ?> params) throws Exception {
+        Stage stage = ReadWriteUtils.instantiateWithParams((Map<String, ?>) params.get("stage"));

Review comment:
       According to offline discussion, let's keep using `Param<>` in this review and see if there is better option after I have refined the code within this direction.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r841018540



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/Benchmark.java
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.DefaultParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** 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 String SHELL_SCRIPT = "flink-ml-benchmark.sh";
+
+    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 =
+            Option.builder()
+                    .longOpt("output-file")
+                    .desc("The output file name to save benchmark results.")
+                    .hasArg()
+                    .build();
+
+    static final Options OPTIONS =
+            new Options().addOption(HELP_OPTION).addOption(OUTPUT_FILE_OPTION);
+
+    public static void printHelp() {
+        HelpFormatter formatter = new HelpFormatter();
+        formatter.setLeftPadding(5);
+        formatter.setWidth(80);
+
+        System.out.println("./" + SHELL_SCRIPT + " <config-file-path> [OPTIONS]");
+        System.out.println();
+        formatter.setSyntaxPrefix("The following options are available:");
+        formatter.printHelp(" ", OPTIONS);
+
+        System.out.println();
+    }
+
+    @SuppressWarnings("unchecked")
+    public static void executeBenchmarks(CommandLine commandLine) throws Exception {
+        String configFile = commandLine.getArgs()[0];
+
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        InputStream inputStream = new FileInputStream(configFile);
+        Map<String, ?> jsonMap = ReadWriteUtils.OBJECT_MAPPER.readValue(inputStream, Map.class);
+        Preconditions.checkArgument(
+                jsonMap.containsKey(VERSION_KEY) && jsonMap.get(VERSION_KEY).equals(1));
+
+        List<String> benchmarkNames =
+                jsonMap.keySet().stream()
+                        .filter(x -> !x.equals(VERSION_KEY))
+                        .collect(Collectors.toList());
+        LOG.info("Found benchmarks " + benchmarkNames);
+
+        List<BenchmarkResult> results = new ArrayList<>();
+
+        for (String benchmarkName : benchmarkNames) {
+            LOG.info("Running benchmark " + benchmarkName + ".");
+
+            BenchmarkResult result =
+                    BenchmarkUtils.runBenchmark(
+                            tEnv, benchmarkName, (Map<String, ?>) jsonMap.get(benchmarkName));
+
+            results.add(result);
+            BenchmarkUtils.printResults(result);
+        }
+
+        if (commandLine.hasOption(OUTPUT_FILE_OPTION.getLongOpt())) {

Review comment:
       We have printed the summary of each benchmark result in the for loop above, as you can see `BenchmarkUtils.printResults` on line 108. As there is not much process information printed out in the terminal, which means the output of `BenchmarkUtils.printResults` will not be overwhelmed by process logs, I think there is unnecessary to print them out again when all benchmark finishes.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r840327273



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/Benchmark.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** Entry class for benchmark execution. */
+public class Benchmark {
+    private static final Logger LOG = LoggerFactory.getLogger(Benchmark.class);
+
+    @SuppressWarnings("unchecked")
+    public static void main(String[] args) throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        InputStream inputStream = new FileInputStream(args[0]);
+        Map<String, ?> jsonMap = ReadWriteUtils.OBJECT_MAPPER.readValue(inputStream, Map.class);
+        Preconditions.checkArgument(
+                jsonMap.containsKey("version") && jsonMap.get("version").equals(1));
+
+        List<String> benchmarkNames =
+                jsonMap.keySet().stream()
+                        .filter(x -> !x.equals("version"))
+                        .collect(Collectors.toList());
+        LOG.info("Found benchmarks " + benchmarkNames);
+
+        List<BenchmarkResult> results = new ArrayList<>();
+
+        for (String benchmarkName : benchmarkNames) {
+            LOG.info("Running benchmark " + benchmarkName + ".");
+
+            BenchmarkResult result =
+                    BenchmarkUtils.runBenchmark(
+                            tEnv, benchmarkName, (Map<String, ?>) jsonMap.get(benchmarkName));
+
+            results.add(result);
+        }
+
+        for (BenchmarkResult result : results) {
+            BenchmarkUtils.printResult(result);
+        }
+
+        if (args.length > 1) {

Review comment:
       I found `org.apache.commons.cli.Option` a good choice. Flink has been using it so we can easily follow the command line style of Flink. I'll use it in the code.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
lindong28 commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r840999097



##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/clustering/kmeans/KMeansModelDataGenerator.java
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.flink.ml.benchmark.clustering.kmeans;
+
+import org.apache.flink.ml.benchmark.generator.DataGenerator;
+import org.apache.flink.ml.benchmark.generator.GeneratorUtils;
+import org.apache.flink.ml.clustering.kmeans.KMeansModelData;
+import org.apache.flink.ml.clustering.kmeans.KMeansParams;
+import org.apache.flink.ml.common.datastream.TableUtils;
+import org.apache.flink.ml.linalg.DenseVector;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Class that generates table arrays containing model data for {@link
+ * org.apache.flink.ml.clustering.kmeans.KMeansModel}.
+ */
+public class KMeansModelDataGenerator

Review comment:
       Sounds good.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] yunfengzhou-hub commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
yunfengzhou-hub commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r841049347



##########
File path: flink-ml-core/src/main/java/org/apache/flink/ml/util/ReadWriteUtils.java
##########
@@ -94,25 +95,29 @@
      */
     public static void saveMetadata(Stage<?> stage, String path, Map<String, ?> extraMetadata)
             throws IOException {
-        // Creates parent directories if not already created.
-        FileSystem fs = mkdirs(path);
-
         Map<String, Object> metadata = new HashMap<>(extraMetadata);
         metadata.put("className", stage.getClass().getName());
         metadata.put("timestamp", System.currentTimeMillis());
         metadata.put("paramMap", jsonEncode(stage.getParamMap()));
         // TODO: add version in the metadata.
         String metadataStr = OBJECT_MAPPER.writeValueAsString(metadata);
 
-        Path metadataPath = new Path(path, "metadata");
-        if (fs.exists(metadataPath)) {
-            throw new IOException("File " + metadataPath + " already exists.");
+        saveToFile(new Path(path, "metadata"), metadataStr);
+    }
+
+    /** Saves a given string to the specified file. */
+    public static void saveToFile(Path path, String content) throws IOException {
+        // Creates parent directories if not already created.
+        FileSystem fs = mkdirs(path.getParent().toString());
+
+        if (fs.exists(path)) {
+            throw new IOException("File " + path + " already exists.");

Review comment:
       Got it. I'll add a `boolean` option to specify whether to overwrite existing files. When saving model data it is false, and when saving benchmark results it would be true.




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink-ml] lindong28 commented on a change in pull request #71: [FLINK-26443] Add benchmark framework

Posted by GitBox <gi...@apache.org>.
lindong28 commented on a change in pull request #71:
URL: https://github.com/apache/flink-ml/pull/71#discussion_r836029049



##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,261 @@
+# Flink ML Benchmark Guideline
+
+This document provides instructions about how to run benchmarks on Flink ML's
+stages.
+
+## Write Benchmark Programs
+
+### Add Maven Dependencies
+
+In order to write Flink ML's java benchmark programs, first make sure that the
+following dependencies have been added to your maven project's `pom.xml`.
+
+```xml
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-core_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-iteration_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-lib_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-benchmark_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>statefun-flink-core</artifactId>
+  <version>3.1.0</version>
+  <exclusions>
+    <exclusion>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-streaming-java_2.12</artifactId>
+    </exclusion>
+  </exclusions>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-table-api-java-bridge_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-table-planner_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-clients_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+```
+
+### Write Java Program
+
+Then you can write a program as follows to run benchmark on Flink ML stages. The
+example code below tests the performance of Flink ML's KMeans algorithm, with
+the default configuration parameters used.
+
+```java
+public class Main {
+    public static void main(String[] args) throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        KMeans kMeans = new KMeans();
+        KMeansInputsGenerator inputsGenerator = new KMeansInputsGenerator();
+
+        BenchmarkResult result =
+                BenchmarkUtils.runBenchmark("exampleBenchmark", tEnv, kMeans, inputsGenerator);
+
+        BenchmarkUtils.printResult(result);
+    }
+}
+```
+
+### Execute Benchmark Program
+
+After executing the `main()` method above, you will see benchmark results
+printed out in your terminal. An example of the printed content is as follows.
+
+```
+Benchmark Name: exampleBenchmark
+Total Execution Time(ms): 828.0
+```
+
+### Configure Benchmark Parameters
+
+If you want to run benchmark on customed configuration parameters, you can set
+them with Flink ML's `WithParams` API as follows.
+
+```java
+KMeans kMeans = new KMeans()
+  .setK(5)
+  .setMaxIter(50);
+KMeansInputsGenerator inputsGenerator = new KMeansInputsGenerator()
+  .setDims(3)
+  .setDataSize(10000);
+```
+
+## Execute Benchmark through Command-Line Interface (CLI)
+
+You can also configure and execute benchmarks through Command-Line Interface
+(CLI) without writing java programs.
+
+### Prerequisites
+
+Before using Flink ML's CLI, make sure you have installed Flink 1.14 in your
+local environment, and that you have started a Flink cluster locally. If not,
+you can start a standalone session with the following command.
+
+```bash
+$ start-cluster
+```
+
+In order to use Flink ML's CLI you need to have the latest binary distribution
+of Flink ML. You can acquire the distribution by building Flink ML's source code
+locally, which means to execute the following command in Flink ML repository's
+root directory.
+
+```bash
+$ mvn clean package -DskipTests
+```
+
+After executing the command above, you will be able to find the binary
+distribution under
+`./flink-ml-dist/target/flink-ml-<version>-bin/flink-ml-<version>/`.
+
+### Run Benchmark CLI
+
+In the binary distribution's folder, execute the following command to run an
+example benchmark.
+
+```bash
+$ ./bin/flink-ml-benchmark.sh ./examples/benchmark-example-conf.json
+```
+
+You will notice that some Flink job is submitted to your Flink cluster, and the
+following information is printed out in your terminal. This means that you have
+successfully executed a benchmark on `KMeansModel`.
+
+```
+Job has been submitted with JobID 85b4a33df5c00a315e0d1142e1d743be
+Program execution finished
+Job with JobID 85b4a33df5c00a315e0d1142e1d743be has finished.
+Job Runtime: 828 ms
+
+Benchmark Name: KMeansModel-1
+Total Execution Time(ms): 828.0
+
+```
+
+### Save Benchmark Result to File
+
+`flink-ml-benchmark.sh` has redirected all warnings and process logs to stderr,
+and the benchmark results to stdout. So if you write the command in the
+following way
+
+```bash
+$ ./bin/flink-ml-benchmark.sh ./examples/benchmark-example-conf.json > output.txt

Review comment:
       This command is almost the same as the command above.
   
   Would it be simpler to just have one section `Run Benchmark CLI`? And in this section, we can explain how to get and interpret the benchmark results.

##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,261 @@
+# Flink ML Benchmark Guideline
+
+This document provides instructions about how to run benchmarks on Flink ML's
+stages.
+
+## Write Benchmark Programs
+
+### Add Maven Dependencies
+
+In order to write Flink ML's java benchmark programs, first make sure that the
+following dependencies have been added to your maven project's `pom.xml`.
+
+```xml
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-core_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-iteration_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-lib_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-benchmark_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>statefun-flink-core</artifactId>
+  <version>3.1.0</version>
+  <exclusions>
+    <exclusion>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-streaming-java_2.12</artifactId>
+    </exclusion>
+  </exclusions>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-table-api-java-bridge_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-table-planner_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-clients_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+```
+
+### Write Java Program
+
+Then you can write a program as follows to run benchmark on Flink ML stages. The
+example code below tests the performance of Flink ML's KMeans algorithm, with
+the default configuration parameters used.
+
+```java
+public class Main {
+    public static void main(String[] args) throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        KMeans kMeans = new KMeans();
+        KMeansInputsGenerator inputsGenerator = new KMeansInputsGenerator();
+
+        BenchmarkResult result =
+                BenchmarkUtils.runBenchmark("exampleBenchmark", tEnv, kMeans, inputsGenerator);
+
+        BenchmarkUtils.printResult(result);
+    }
+}
+```
+
+### Execute Benchmark Program
+
+After executing the `main()` method above, you will see benchmark results
+printed out in your terminal. An example of the printed content is as follows.
+
+```
+Benchmark Name: exampleBenchmark
+Total Execution Time(ms): 828.0
+```
+
+### Configure Benchmark Parameters
+
+If you want to run benchmark on customed configuration parameters, you can set
+them with Flink ML's `WithParams` API as follows.
+
+```java
+KMeans kMeans = new KMeans()
+  .setK(5)
+  .setMaxIter(50);
+KMeansInputsGenerator inputsGenerator = new KMeansInputsGenerator()
+  .setDims(3)
+  .setDataSize(10000);
+```
+
+## Execute Benchmark through Command-Line Interface (CLI)
+
+You can also configure and execute benchmarks through Command-Line Interface
+(CLI) without writing java programs.
+
+### Prerequisites
+
+Before using Flink ML's CLI, make sure you have installed Flink 1.14 in your
+local environment, and that you have started a Flink cluster locally. If not,
+you can start a standalone session with the following command.
+
+```bash
+$ start-cluster
+```
+
+In order to use Flink ML's CLI you need to have the latest binary distribution
+of Flink ML. You can acquire the distribution by building Flink ML's source code
+locally, which means to execute the following command in Flink ML repository's
+root directory.
+
+```bash
+$ mvn clean package -DskipTests
+```
+
+After executing the command above, you will be able to find the binary
+distribution under
+`./flink-ml-dist/target/flink-ml-<version>-bin/flink-ml-<version>/`.
+
+### Run Benchmark CLI
+
+In the binary distribution's folder, execute the following command to run an
+example benchmark.
+
+```bash
+$ ./bin/flink-ml-benchmark.sh ./examples/benchmark-example-conf.json
+```
+
+You will notice that some Flink job is submitted to your Flink cluster, and the
+following information is printed out in your terminal. This means that you have
+successfully executed a benchmark on `KMeansModel`.
+
+```
+Job has been submitted with JobID 85b4a33df5c00a315e0d1142e1d743be
+Program execution finished
+Job with JobID 85b4a33df5c00a315e0d1142e1d743be has finished.
+Job Runtime: 828 ms
+
+Benchmark Name: KMeansModel-1
+Total Execution Time(ms): 828.0
+
+```
+
+### Save Benchmark Result to File
+
+`flink-ml-benchmark.sh` has redirected all warnings and process logs to stderr,
+and the benchmark results to stdout. So if you write the command in the
+following way
+
+```bash
+$ ./bin/flink-ml-benchmark.sh ./examples/benchmark-example-conf.json > output.txt
+```
+
+You will get a clean benchmark result saved in `output.txt` as follows.
+
+```
+Benchmark Name: KMeansModel-1
+Total Execution Time(ms): 828.0
+
+```
+
+### Configuration File Format
+
+The benchmark CLI creates stages and test data according to the input
+configuration file. The configuration file should contain a JSON object in the
+following format.
+
+First of all, the file should contain the following configurations as the
+metadata of the JSON object.
+
+- `"_version"`: The version of the json format. Currently its value must be 1.
+
+Then, each benchmark should be specified through key-object pairs.
+
+The key for each benchmark JSON object will be regarded as the name of the
+benchmark. The benchmark name can contain English letters, numbers, hyphens(-)
+and underscores(_), but should not start with a hyphen or underscore.
+
+The value of each benchmark name should be a JSON object containing at least
+`"stage"` and `"inputs"`. If the stage to be tested is a `Model` and its model
+data needs to be explicitly set, the JSON object should also contain
+`"modelData"`.
+
+The value of `"stage"`, `"inputs"` or `"modelData"` should be a JSON object
+containing `"className"` and `paramMap`.
+
+- `"className"`'s value should be the full classpath of a `WithParams` subclass.
+  For `"stage"`, the class should be a subclass of `Stage`. For `"inputs"` or
+  `"modelData"`, the class should be a subclass of `DataGenerator`.
+
+- `"paramMap"`'s value should be a JSON object containing the parameters related
+  to the specific `Stage` or `DataGenerator`. Note that all parameter values
+  should be wrapped as strings.
+
+Combining the format requirements above, an example configuration file is as
+follows.
+
+```json
+{
+  "_version": 1,
+  "KMeansModel-1": {
+    "stage": {
+      "className": "org.apache.flink.ml.clustering.kmeans.KMeansModel",
+      "paramMap": {
+        "featuresCol": "\"features\"",

Review comment:
       Could we allow user to just specify `"featuresCol": "features"` instead of `"featuresCol": "\"features\""`?

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/Benchmark.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.io.PrintStream;
+import java.util.Map;
+
+import static org.apache.flink.ml.util.ReadWriteUtils.OBJECT_MAPPER;
+
+/** Entry class for benchmark execution. */
+public class Benchmark {
+    @SuppressWarnings("unchecked")
+    public static void main(String[] args) throws Exception {
+        final PrintStream originalOut = System.out;
+
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        InputStream inputStream = new FileInputStream(args[0]);
+        Map<String, ?> jsonMap = OBJECT_MAPPER.readValue(inputStream, Map.class);
+        Map<String, Map<String, ?>> benchmarkParamsMap =
+                BenchmarkUtils.parseBenchmarkParams(jsonMap);
+        System.err.println("Found benchmarks " + benchmarkParamsMap.keySet());
+
+        for (String benchmarkName : benchmarkParamsMap.keySet()) {
+            System.err.println("Running benchmark " + benchmarkName + ".");
+
+            // Redirect all flink execution logs to stderr.
+            System.setOut(System.err);

Review comment:
       Can you explain why we need to do this?

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/BenchmarkUtils.java
##########
@@ -0,0 +1,233 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.ml.api.AlgoOperator;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.api.Model;
+import org.apache.flink.ml.api.Stage;
+import org.apache.flink.ml.benchmark.generator.DataGenerator;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.WithParams;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.bridge.java.internal.StreamTableEnvironmentImpl;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.Preconditions;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+/** Utility methods for benchmarks. */
+@SuppressWarnings("unchecked")
+public class BenchmarkUtils {
+    private static final String benchmarkNamePattern = "^[A-Za-z0-9][A-Za-z0-9_\\-]*$";
+
+    /**
+     * Loads benchmark paramMaps from the provided json map.
+     *
+     * @return A map whose key is the names of the loaded benchmarks, value is the parameters of the
+     *     benchmarks.
+     */
+    public static Map<String, Map<String, ?>> parseBenchmarkParams(Map<String, ?> jsonMap) {
+        Preconditions.checkArgument(
+                jsonMap.containsKey("_version") && jsonMap.get("_version").equals(1));
+
+        Map<String, Map<String, ?>> result = new HashMap<>();
+        for (String key : jsonMap.keySet()) {
+            if (!isValidBenchmarkName(key)) {
+                continue;
+            }
+            result.put(key, (Map<String, ?>) jsonMap.get(key));
+        }
+        return result;
+    }
+
+    /**
+     * Checks whether a string is a valid benchmark name.
+     *
+     * <p>A valid benchmark name should only contain English letters, numbers, hyphens (-) and
+     * underscores (_). The name should not start with a hyphen or underscore.
+     */
+    public static boolean isValidBenchmarkName(String name) {
+        return Pattern.matches(benchmarkNamePattern, name);
+    }
+
+    /**
+     * Instantiates a benchmark from its parameter map and executes the benchmark in the provided
+     * environment.
+     *
+     * @return Results of the executed benchmark.
+     */
+    public static BenchmarkResult runBenchmark(
+            String name, StreamTableEnvironment tEnv, Map<String, ?> benchmarkParamsMap)
+            throws Exception {
+        Stage<?> stage =
+                (Stage<?>) instantiateWithParams((Map<String, ?>) benchmarkParamsMap.get("stage"));
+
+        BenchmarkResult result;
+        if (benchmarkParamsMap.size() == 2) {
+            DataGenerator<?> inputsGenerator =
+                    (DataGenerator<?>)
+                            instantiateWithParams(
+                                    (Map<String, ?>) benchmarkParamsMap.get("inputs"));
+            result = runBenchmark(name, tEnv, stage, inputsGenerator);
+        } else if (benchmarkParamsMap.size() == 3 && stage instanceof Model) {
+            DataGenerator<?> inputsGenerator =
+                    (DataGenerator<?>)
+                            instantiateWithParams(
+                                    (Map<String, ?>) benchmarkParamsMap.get("inputs"));
+            DataGenerator<?> modelDataGenerator =
+                    (DataGenerator<?>)
+                            instantiateWithParams(
+                                    (Map<String, ?>) benchmarkParamsMap.get("modelData"));
+            result =
+                    runBenchmark(name, tEnv, (Model<?>) stage, modelDataGenerator, inputsGenerator);
+        } else {
+            throw new IllegalArgumentException(
+                    "Unsupported json map with keys " + benchmarkParamsMap.keySet());
+        }
+
+        return result;
+    }
+
+    /**
+     * Executes a benchmark from a stage and an inputsGenerator in the provided environment.
+     *
+     * @return Results of the executed benchmark.
+     */
+    public static BenchmarkResult runBenchmark(
+            String name,
+            StreamTableEnvironment tEnv,
+            Stage<?> stage,
+            DataGenerator<?> inputsGenerator)
+            throws Exception {
+        Table[] inputTables = inputsGenerator.getData(tEnv);
+
+        Table[] outputTables;
+        if (stage instanceof Estimator) {
+            outputTables = ((Estimator<?, ?>) stage).fit(inputTables).getModelData();
+        } else if (stage instanceof AlgoOperator) {
+            outputTables = ((AlgoOperator<?>) stage).transform(inputTables);
+        } else {
+            throw new IllegalArgumentException("Unsupported Stage class " + stage.getClass());
+        }
+
+        for (Table table : outputTables) {
+            tEnv.toDataStream(table).addSink(new DiscardingSink<>());
+        }
+
+        StreamExecutionEnvironment env = ((StreamTableEnvironmentImpl) tEnv).execEnv();
+        JobExecutionResult executionResult = env.execute();
+
+        BenchmarkResult result = new BenchmarkResult();
+        result.name = name;
+        result.executionTimeMillis = (double) executionResult.getNetRuntime(TimeUnit.MILLISECONDS);
+
+        return result;
+    }
+
+    /**
+     * Executes a benchmark from a model with its modelDataGenerator and inputsGenerator in the
+     * provided environment.
+     *
+     * @return Results of the executed benchmark.
+     */
+    public static BenchmarkResult runBenchmark(
+            String name,
+            StreamTableEnvironment tEnv,
+            Model<?> model,
+            DataGenerator<?> modelDataGenerator,
+            DataGenerator<?> inputsGenerator)
+            throws Exception {
+        model.setModelData(modelDataGenerator.getData(tEnv));
+
+        Table[] outputTables = model.transform(inputsGenerator.getData(tEnv));
+
+        for (Table table : outputTables) {
+            tEnv.toDataStream(table).addSink(new DiscardingSink<>());
+        }
+
+        StreamExecutionEnvironment env = ((StreamTableEnvironmentImpl) tEnv).execEnv();
+        JobExecutionResult executionResult = env.execute();
+
+        BenchmarkResult result = new BenchmarkResult();
+        result.name = name;
+        result.executionTimeMillis = (double) executionResult.getNetRuntime(TimeUnit.MILLISECONDS);
+
+        return result;
+    }
+
+    /**
+     * Instantiates a WithParams subclass from the provided json map.
+     *
+     * @param jsonMap a map containing className and paramMap.
+     * @return the instantiated WithParams subclass.
+     */
+    public static WithParams<?> instantiateWithParams(Map<String, ?> jsonMap) throws Exception {
+        String className = (String) jsonMap.get("className");
+        Class<WithParams<?>> clazz = (Class<WithParams<?>>) Class.forName(className);
+        WithParams<?> instance = InstantiationUtil.instantiate(clazz);
+
+        Map<String, Param<?>> nameToParam = new HashMap<>();
+        for (Param<?> param : ParamUtils.getPublicFinalParamFields(instance)) {
+            nameToParam.put(param.name, param);
+        }
+
+        Map<String, String> paramMap = (Map<String, String>) jsonMap.get("paramMap");
+        for (Map.Entry<String, String> entry : paramMap.entrySet()) {
+            Param<?> param = nameToParam.get(entry.getKey());
+            setParam(instance, param, param.jsonDecode(entry.getValue()));
+        }
+
+        return instance;
+    }
+
+    // A helper method that sets an object's parameter value. We can not call stage.set(param,
+    // value) directly because stage::set(...) needs the actual type of the value.
+    private static <T> void setParam(WithParams<?> withParams, Param<T> param, Object value) {
+        withParams.set(param, (T) value);
+    }
+
+    /** Prints out the provided benchmark result. */
+    public static void printResult(BenchmarkResult result) {
+        Preconditions.checkNotNull(result.name);
+        System.out.println("Benchmark Name: " + result.name);
+
+        if (result.executionTimeMillis != null) {

Review comment:
       When would this value be null? Could it be simpler to skip the `if` statement and always print its value (including null)?

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/BenchmarkUtils.java
##########
@@ -0,0 +1,233 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.ml.api.AlgoOperator;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.api.Model;
+import org.apache.flink.ml.api.Stage;
+import org.apache.flink.ml.benchmark.generator.DataGenerator;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.WithParams;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.bridge.java.internal.StreamTableEnvironmentImpl;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.Preconditions;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+/** Utility methods for benchmarks. */
+@SuppressWarnings("unchecked")
+public class BenchmarkUtils {
+    private static final String benchmarkNamePattern = "^[A-Za-z0-9][A-Za-z0-9_\\-]*$";
+
+    /**
+     * Loads benchmark paramMaps from the provided json map.
+     *
+     * @return A map whose key is the names of the loaded benchmarks, value is the parameters of the
+     *     benchmarks.
+     */
+    public static Map<String, Map<String, ?>> parseBenchmarkParams(Map<String, ?> jsonMap) {
+        Preconditions.checkArgument(
+                jsonMap.containsKey("_version") && jsonMap.get("_version").equals(1));
+
+        Map<String, Map<String, ?>> result = new HashMap<>();
+        for (String key : jsonMap.keySet()) {
+            if (!isValidBenchmarkName(key)) {

Review comment:
       Is it necessary to enforce the naming pattern here? Would it be simpler to just skip the reserved field names (e.g. version) here?

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/BenchmarkUtils.java
##########
@@ -0,0 +1,233 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.ml.api.AlgoOperator;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.api.Model;
+import org.apache.flink.ml.api.Stage;
+import org.apache.flink.ml.benchmark.generator.DataGenerator;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.WithParams;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.bridge.java.internal.StreamTableEnvironmentImpl;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.Preconditions;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+/** Utility methods for benchmarks. */
+@SuppressWarnings("unchecked")
+public class BenchmarkUtils {
+    private static final String benchmarkNamePattern = "^[A-Za-z0-9][A-Za-z0-9_\\-]*$";
+
+    /**
+     * Loads benchmark paramMaps from the provided json map.
+     *
+     * @return A map whose key is the names of the loaded benchmarks, value is the parameters of the
+     *     benchmarks.
+     */
+    public static Map<String, Map<String, ?>> parseBenchmarkParams(Map<String, ?> jsonMap) {
+        Preconditions.checkArgument(
+                jsonMap.containsKey("_version") && jsonMap.get("_version").equals(1));

Review comment:
       Hmm... did we learn the name `_version` from some other projects? If no, would it be simpler to just use `version` here?

##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,261 @@
+# Flink ML Benchmark Guideline
+
+This document provides instructions about how to run benchmarks on Flink ML's
+stages.
+
+## Write Benchmark Programs
+
+### Add Maven Dependencies
+
+In order to write Flink ML's java benchmark programs, first make sure that the
+following dependencies have been added to your maven project's `pom.xml`.
+
+```xml
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-core_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-iteration_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-lib_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-benchmark_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>statefun-flink-core</artifactId>
+  <version>3.1.0</version>
+  <exclusions>
+    <exclusion>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-streaming-java_2.12</artifactId>
+    </exclusion>
+  </exclusions>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-table-api-java-bridge_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-table-planner_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-clients_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+```
+
+### Write Java Program
+
+Then you can write a program as follows to run benchmark on Flink ML stages. The
+example code below tests the performance of Flink ML's KMeans algorithm, with
+the default configuration parameters used.
+
+```java
+public class Main {
+    public static void main(String[] args) throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        KMeans kMeans = new KMeans();
+        KMeansInputsGenerator inputsGenerator = new KMeansInputsGenerator();
+
+        BenchmarkResult result =
+                BenchmarkUtils.runBenchmark("exampleBenchmark", tEnv, kMeans, inputsGenerator);
+
+        BenchmarkUtils.printResult(result);
+    }
+}
+```
+
+### Execute Benchmark Program
+
+After executing the `main()` method above, you will see benchmark results
+printed out in your terminal. An example of the printed content is as follows.
+
+```
+Benchmark Name: exampleBenchmark
+Total Execution Time(ms): 828.0
+```
+
+### Configure Benchmark Parameters
+
+If you want to run benchmark on customed configuration parameters, you can set
+them with Flink ML's `WithParams` API as follows.
+
+```java
+KMeans kMeans = new KMeans()
+  .setK(5)
+  .setMaxIter(50);
+KMeansInputsGenerator inputsGenerator = new KMeansInputsGenerator()
+  .setDims(3)
+  .setDataSize(10000);
+```
+
+## Execute Benchmark through Command-Line Interface (CLI)
+
+You can also configure and execute benchmarks through Command-Line Interface
+(CLI) without writing java programs.
+
+### Prerequisites
+
+Before using Flink ML's CLI, make sure you have installed Flink 1.14 in your
+local environment, and that you have started a Flink cluster locally. If not,
+you can start a standalone session with the following command.
+
+```bash
+$ start-cluster

Review comment:
       Flink ML repo does not provide this script. Could we provide more information on how to run this script? We can provide an installation section similar to [1].
   
   [1] https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/sql/gettingstarted/#installation

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/Benchmark.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.io.PrintStream;
+import java.util.Map;
+
+import static org.apache.flink.ml.util.ReadWriteUtils.OBJECT_MAPPER;
+
+/** Entry class for benchmark execution. */
+public class Benchmark {
+    @SuppressWarnings("unchecked")
+    public static void main(String[] args) throws Exception {
+        final PrintStream originalOut = System.out;
+
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        InputStream inputStream = new FileInputStream(args[0]);
+        Map<String, ?> jsonMap = OBJECT_MAPPER.readValue(inputStream, Map.class);
+        Map<String, Map<String, ?>> benchmarkParamsMap =
+                BenchmarkUtils.parseBenchmarkParams(jsonMap);
+        System.err.println("Found benchmarks " + benchmarkParamsMap.keySet());

Review comment:
       Should this be `System.out`? In general we only use `System.err` for error messages.
   
   Same for other System.err.

##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,261 @@
+# Flink ML Benchmark Guideline
+
+This document provides instructions about how to run benchmarks on Flink ML's
+stages.
+
+## Write Benchmark Programs
+
+### Add Maven Dependencies
+
+In order to write Flink ML's java benchmark programs, first make sure that the
+following dependencies have been added to your maven project's `pom.xml`.
+
+```xml
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-core_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-iteration_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-lib_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-ml-benchmark_${scala.binary.version}</artifactId>
+  <version>${flink.ml.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>statefun-flink-core</artifactId>
+  <version>3.1.0</version>
+  <exclusions>
+    <exclusion>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-streaming-java_2.12</artifactId>
+    </exclusion>
+  </exclusions>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-table-api-java-bridge_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-table-planner_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+
+<dependency>
+  <groupId>org.apache.flink</groupId>
+  <artifactId>flink-clients_${scala.binary.version}</artifactId>
+  <version>${flink.version}</version>
+</dependency>
+```
+
+### Write Java Program
+
+Then you can write a program as follows to run benchmark on Flink ML stages. The
+example code below tests the performance of Flink ML's KMeans algorithm, with
+the default configuration parameters used.
+
+```java
+public class Main {
+    public static void main(String[] args) throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        KMeans kMeans = new KMeans();
+        KMeansInputsGenerator inputsGenerator = new KMeansInputsGenerator();
+
+        BenchmarkResult result =
+                BenchmarkUtils.runBenchmark("exampleBenchmark", tEnv, kMeans, inputsGenerator);
+
+        BenchmarkUtils.printResult(result);
+    }
+}
+```
+
+### Execute Benchmark Program
+
+After executing the `main()` method above, you will see benchmark results
+printed out in your terminal. An example of the printed content is as follows.
+
+```
+Benchmark Name: exampleBenchmark
+Total Execution Time(ms): 828.0
+```
+
+### Configure Benchmark Parameters
+
+If you want to run benchmark on customed configuration parameters, you can set
+them with Flink ML's `WithParams` API as follows.
+
+```java
+KMeans kMeans = new KMeans()
+  .setK(5)
+  .setMaxIter(50);
+KMeansInputsGenerator inputsGenerator = new KMeansInputsGenerator()
+  .setDims(3)
+  .setDataSize(10000);
+```
+
+## Execute Benchmark through Command-Line Interface (CLI)
+
+You can also configure and execute benchmarks through Command-Line Interface
+(CLI) without writing java programs.
+
+### Prerequisites
+
+Before using Flink ML's CLI, make sure you have installed Flink 1.14 in your
+local environment, and that you have started a Flink cluster locally. If not,
+you can start a standalone session with the following command.
+
+```bash
+$ start-cluster
+```
+
+In order to use Flink ML's CLI you need to have the latest binary distribution
+of Flink ML. You can acquire the distribution by building Flink ML's source code
+locally, which means to execute the following command in Flink ML repository's
+root directory.
+
+```bash
+$ mvn clean package -DskipTests
+```
+
+After executing the command above, you will be able to find the binary
+distribution under
+`./flink-ml-dist/target/flink-ml-<version>-bin/flink-ml-<version>/`.

Review comment:
       Is it possible to fill in the version of the corresponding branch using a variable?

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/BenchmarkUtils.java
##########
@@ -0,0 +1,233 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.ml.api.AlgoOperator;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.api.Model;
+import org.apache.flink.ml.api.Stage;
+import org.apache.flink.ml.benchmark.generator.DataGenerator;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.WithParams;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.bridge.java.internal.StreamTableEnvironmentImpl;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.Preconditions;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+/** Utility methods for benchmarks. */
+@SuppressWarnings("unchecked")
+public class BenchmarkUtils {
+    private static final String benchmarkNamePattern = "^[A-Za-z0-9][A-Za-z0-9_\\-]*$";
+
+    /**
+     * Loads benchmark paramMaps from the provided json map.
+     *
+     * @return A map whose key is the names of the loaded benchmarks, value is the parameters of the
+     *     benchmarks.
+     */
+    public static Map<String, Map<String, ?>> parseBenchmarkParams(Map<String, ?> jsonMap) {
+        Preconditions.checkArgument(
+                jsonMap.containsKey("_version") && jsonMap.get("_version").equals(1));
+
+        Map<String, Map<String, ?>> result = new HashMap<>();
+        for (String key : jsonMap.keySet()) {
+            if (!isValidBenchmarkName(key)) {
+                continue;
+            }
+            result.put(key, (Map<String, ?>) jsonMap.get(key));
+        }
+        return result;
+    }
+
+    /**
+     * Checks whether a string is a valid benchmark name.
+     *
+     * <p>A valid benchmark name should only contain English letters, numbers, hyphens (-) and
+     * underscores (_). The name should not start with a hyphen or underscore.
+     */
+    public static boolean isValidBenchmarkName(String name) {
+        return Pattern.matches(benchmarkNamePattern, name);
+    }
+
+    /**
+     * Instantiates a benchmark from its parameter map and executes the benchmark in the provided
+     * environment.
+     *
+     * @return Results of the executed benchmark.
+     */
+    public static BenchmarkResult runBenchmark(

Review comment:
       Would the following implementation be simpler and shorter?
   
   Note that we can change the signature of `instantiateWithParams(...)` to `<T extends WithParams<?>> T instantiateWithParams(Map<String, ?> jsonMap)`
   
   ```
       public static BenchmarkResult runBenchmark(
               StreamExecutionEnvironment env, String name, Map<String, ?> params)
               throws Exception {
           Stage<?> stage = instantiateWithParams((Map<String, ?>) params.get("stage"));
           DataGenerator<?> inputsGenerator = instantiateWithParams((Map<String, ?>) params.get("inputs"));
           DataGenerator<?> modelDataGenerator = null;
           if (params.containsKey("modelData")) {
               modelDataGenerator = instantiateWithParams((Map<String, ?>) params.get("modelData"));
           }
   
           return runBenchmark(env, name, stage, inputsGenerator, modelDataGenerator);
       }
   
       public static BenchmarkResult runBenchmark(
               StreamExecutionEnvironment env,
               String name,
               Stage<?> stage,
               DataGenerator<?> inputsGenerator,
               DataGenerator<?> modelDataGenerator)
               throws Exception {
           StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
   
           Table[] inputTables = inputsGenerator.getData(tEnv);
           if (modelDataGenerator != null) {
               ((Model<?>) stage).setModelData(modelDataGenerator.getData(tEnv));
           }
   
           Table[] outputTables;
           if (stage instanceof Estimator) {
               outputTables = ((Estimator<?, ?>) stage).fit(inputTables).getModelData();
           } else if (stage instanceof AlgoOperator) {
               outputTables = ((AlgoOperator<?>) stage).transform(inputTables);
           } else {
               throw new IllegalArgumentException("Unsupported Stage class " + stage.getClass());
           }
   
           for (Table table : outputTables) {
               tEnv.toDataStream(table).addSink(new DiscardingSink<>());
           }
   
           JobExecutionResult executionResult = env.execute();
   
           BenchmarkResult result = new BenchmarkResult();
           result.name = name;
           result.executionTimeMillis = (double) executionResult.getNetRuntime(TimeUnit.MILLISECONDS);
   
           return result;
       }
   ```

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/BenchmarkUtils.java
##########
@@ -0,0 +1,233 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.ml.api.AlgoOperator;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.api.Model;
+import org.apache.flink.ml.api.Stage;
+import org.apache.flink.ml.benchmark.generator.DataGenerator;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.WithParams;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.bridge.java.internal.StreamTableEnvironmentImpl;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.Preconditions;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+/** Utility methods for benchmarks. */
+@SuppressWarnings("unchecked")
+public class BenchmarkUtils {
+    private static final String benchmarkNamePattern = "^[A-Za-z0-9][A-Za-z0-9_\\-]*$";
+
+    /**
+     * Loads benchmark paramMaps from the provided json map.
+     *
+     * @return A map whose key is the names of the loaded benchmarks, value is the parameters of the
+     *     benchmarks.
+     */
+    public static Map<String, Map<String, ?>> parseBenchmarkParams(Map<String, ?> jsonMap) {
+        Preconditions.checkArgument(
+                jsonMap.containsKey("_version") && jsonMap.get("_version").equals(1));
+
+        Map<String, Map<String, ?>> result = new HashMap<>();
+        for (String key : jsonMap.keySet()) {
+            if (!isValidBenchmarkName(key)) {
+                continue;
+            }
+            result.put(key, (Map<String, ?>) jsonMap.get(key));
+        }
+        return result;
+    }
+
+    /**
+     * Checks whether a string is a valid benchmark name.
+     *
+     * <p>A valid benchmark name should only contain English letters, numbers, hyphens (-) and
+     * underscores (_). The name should not start with a hyphen or underscore.
+     */
+    public static boolean isValidBenchmarkName(String name) {
+        return Pattern.matches(benchmarkNamePattern, name);
+    }
+
+    /**
+     * Instantiates a benchmark from its parameter map and executes the benchmark in the provided
+     * environment.
+     *
+     * @return Results of the executed benchmark.
+     */
+    public static BenchmarkResult runBenchmark(
+            String name, StreamTableEnvironment tEnv, Map<String, ?> benchmarkParamsMap)
+            throws Exception {
+        Stage<?> stage =
+                (Stage<?>) instantiateWithParams((Map<String, ?>) benchmarkParamsMap.get("stage"));
+
+        BenchmarkResult result;
+        if (benchmarkParamsMap.size() == 2) {
+            DataGenerator<?> inputsGenerator =
+                    (DataGenerator<?>)
+                            instantiateWithParams(
+                                    (Map<String, ?>) benchmarkParamsMap.get("inputs"));
+            result = runBenchmark(name, tEnv, stage, inputsGenerator);
+        } else if (benchmarkParamsMap.size() == 3 && stage instanceof Model) {
+            DataGenerator<?> inputsGenerator =
+                    (DataGenerator<?>)
+                            instantiateWithParams(
+                                    (Map<String, ?>) benchmarkParamsMap.get("inputs"));
+            DataGenerator<?> modelDataGenerator =
+                    (DataGenerator<?>)
+                            instantiateWithParams(
+                                    (Map<String, ?>) benchmarkParamsMap.get("modelData"));
+            result =
+                    runBenchmark(name, tEnv, (Model<?>) stage, modelDataGenerator, inputsGenerator);
+        } else {
+            throw new IllegalArgumentException(
+                    "Unsupported json map with keys " + benchmarkParamsMap.keySet());
+        }
+
+        return result;
+    }
+
+    /**
+     * Executes a benchmark from a stage and an inputsGenerator in the provided environment.
+     *
+     * @return Results of the executed benchmark.
+     */
+    public static BenchmarkResult runBenchmark(
+            String name,
+            StreamTableEnvironment tEnv,
+            Stage<?> stage,
+            DataGenerator<?> inputsGenerator)
+            throws Exception {
+        Table[] inputTables = inputsGenerator.getData(tEnv);
+
+        Table[] outputTables;
+        if (stage instanceof Estimator) {
+            outputTables = ((Estimator<?, ?>) stage).fit(inputTables).getModelData();
+        } else if (stage instanceof AlgoOperator) {
+            outputTables = ((AlgoOperator<?>) stage).transform(inputTables);
+        } else {
+            throw new IllegalArgumentException("Unsupported Stage class " + stage.getClass());
+        }
+
+        for (Table table : outputTables) {
+            tEnv.toDataStream(table).addSink(new DiscardingSink<>());
+        }
+
+        StreamExecutionEnvironment env = ((StreamTableEnvironmentImpl) tEnv).execEnv();
+        JobExecutionResult executionResult = env.execute();
+
+        BenchmarkResult result = new BenchmarkResult();
+        result.name = name;
+        result.executionTimeMillis = (double) executionResult.getNetRuntime(TimeUnit.MILLISECONDS);

Review comment:
       We need a general solution to calculate the throughput.
   
   How about this:
   - Expose API in the DataGenerator to get the number of input values used in the fit/transform.
   - Get the number of output values from outputTables.
   - Expose the inputValueNum, outputNum, inputThoughput and outputThroughput in the BenchmarkResult.

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/BenchmarkUtils.java
##########
@@ -0,0 +1,233 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.ml.api.AlgoOperator;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.api.Model;
+import org.apache.flink.ml.api.Stage;
+import org.apache.flink.ml.benchmark.generator.DataGenerator;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.WithParams;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.bridge.java.internal.StreamTableEnvironmentImpl;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.Preconditions;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+/** Utility methods for benchmarks. */
+@SuppressWarnings("unchecked")
+public class BenchmarkUtils {
+    private static final String benchmarkNamePattern = "^[A-Za-z0-9][A-Za-z0-9_\\-]*$";
+
+    /**
+     * Loads benchmark paramMaps from the provided json map.
+     *
+     * @return A map whose key is the names of the loaded benchmarks, value is the parameters of the
+     *     benchmarks.
+     */
+    public static Map<String, Map<String, ?>> parseBenchmarkParams(Map<String, ?> jsonMap) {

Review comment:
       Logic of this method seems pretty straightforward. Can we remove this method, move the version check logic to `Benchmark`, and do the type conversion right before invoking `BenchmarkUtils.runBenchmark(...)`?

##########
File path: flink-ml-dist/src/main/flink-ml-bin/bin/flink-ml-benchmark.sh
##########
@@ -0,0 +1,61 @@
+#!/usr/bin/env bash
+################################################################################
+#  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.
+################################################################################
+
+current_path=$(pwd)
+flink_ml_bin_path="$( cd -- "$(dirname "$0")" >/dev/null 2>&1 ; pwd -P )"
+flink_ml_root_path="$(dirname "$flink_ml_bin_path")"
+
+# Checks flink command.
+flink_cmd="flink"

Review comment:
       Since Flink ML does not provide this script, users typically would download Flink binary distribution to get the script, which is typically in a different directory from the Flink ML directory.
   
   Should we provide a way for users to specify the path to Flink scripts used in this script? This could be specified by using environment variable.

##########
File path: flink-ml-benchmark/src/main/java/org/apache/flink/ml/benchmark/BenchmarkUtils.java
##########
@@ -0,0 +1,233 @@
+/*
+ * 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.flink.ml.benchmark;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.ml.api.AlgoOperator;
+import org.apache.flink.ml.api.Estimator;
+import org.apache.flink.ml.api.Model;
+import org.apache.flink.ml.api.Stage;
+import org.apache.flink.ml.benchmark.generator.DataGenerator;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.WithParams;
+import org.apache.flink.ml.util.ParamUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.bridge.java.internal.StreamTableEnvironmentImpl;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.Preconditions;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+/** Utility methods for benchmarks. */
+@SuppressWarnings("unchecked")
+public class BenchmarkUtils {
+    private static final String benchmarkNamePattern = "^[A-Za-z0-9][A-Za-z0-9_\\-]*$";
+
+    /**
+     * Loads benchmark paramMaps from the provided json map.
+     *
+     * @return A map whose key is the names of the loaded benchmarks, value is the parameters of the
+     *     benchmarks.
+     */
+    public static Map<String, Map<String, ?>> parseBenchmarkParams(Map<String, ?> jsonMap) {
+        Preconditions.checkArgument(
+                jsonMap.containsKey("_version") && jsonMap.get("_version").equals(1));
+
+        Map<String, Map<String, ?>> result = new HashMap<>();
+        for (String key : jsonMap.keySet()) {
+            if (!isValidBenchmarkName(key)) {
+                continue;
+            }
+            result.put(key, (Map<String, ?>) jsonMap.get(key));
+        }
+        return result;
+    }
+
+    /**
+     * Checks whether a string is a valid benchmark name.
+     *
+     * <p>A valid benchmark name should only contain English letters, numbers, hyphens (-) and
+     * underscores (_). The name should not start with a hyphen or underscore.
+     */
+    public static boolean isValidBenchmarkName(String name) {
+        return Pattern.matches(benchmarkNamePattern, name);
+    }
+
+    /**
+     * Instantiates a benchmark from its parameter map and executes the benchmark in the provided
+     * environment.
+     *
+     * @return Results of the executed benchmark.
+     */
+    public static BenchmarkResult runBenchmark(
+            String name, StreamTableEnvironment tEnv, Map<String, ?> benchmarkParamsMap)
+            throws Exception {
+        Stage<?> stage =
+                (Stage<?>) instantiateWithParams((Map<String, ?>) benchmarkParamsMap.get("stage"));
+
+        BenchmarkResult result;
+        if (benchmarkParamsMap.size() == 2) {
+            DataGenerator<?> inputsGenerator =
+                    (DataGenerator<?>)
+                            instantiateWithParams(
+                                    (Map<String, ?>) benchmarkParamsMap.get("inputs"));
+            result = runBenchmark(name, tEnv, stage, inputsGenerator);
+        } else if (benchmarkParamsMap.size() == 3 && stage instanceof Model) {
+            DataGenerator<?> inputsGenerator =
+                    (DataGenerator<?>)
+                            instantiateWithParams(
+                                    (Map<String, ?>) benchmarkParamsMap.get("inputs"));
+            DataGenerator<?> modelDataGenerator =
+                    (DataGenerator<?>)
+                            instantiateWithParams(
+                                    (Map<String, ?>) benchmarkParamsMap.get("modelData"));
+            result =
+                    runBenchmark(name, tEnv, (Model<?>) stage, modelDataGenerator, inputsGenerator);
+        } else {
+            throw new IllegalArgumentException(
+                    "Unsupported json map with keys " + benchmarkParamsMap.keySet());
+        }
+
+        return result;
+    }
+
+    /**
+     * Executes a benchmark from a stage and an inputsGenerator in the provided environment.
+     *
+     * @return Results of the executed benchmark.
+     */
+    public static BenchmarkResult runBenchmark(
+            String name,
+            StreamTableEnvironment tEnv,
+            Stage<?> stage,
+            DataGenerator<?> inputsGenerator)
+            throws Exception {
+        Table[] inputTables = inputsGenerator.getData(tEnv);
+
+        Table[] outputTables;
+        if (stage instanceof Estimator) {
+            outputTables = ((Estimator<?, ?>) stage).fit(inputTables).getModelData();
+        } else if (stage instanceof AlgoOperator) {
+            outputTables = ((AlgoOperator<?>) stage).transform(inputTables);
+        } else {
+            throw new IllegalArgumentException("Unsupported Stage class " + stage.getClass());
+        }
+
+        for (Table table : outputTables) {
+            tEnv.toDataStream(table).addSink(new DiscardingSink<>());
+        }
+
+        StreamExecutionEnvironment env = ((StreamTableEnvironmentImpl) tEnv).execEnv();
+        JobExecutionResult executionResult = env.execute();
+
+        BenchmarkResult result = new BenchmarkResult();
+        result.name = name;
+        result.executionTimeMillis = (double) executionResult.getNetRuntime(TimeUnit.MILLISECONDS);
+
+        return result;
+    }
+
+    /**
+     * Executes a benchmark from a model with its modelDataGenerator and inputsGenerator in the
+     * provided environment.
+     *
+     * @return Results of the executed benchmark.
+     */
+    public static BenchmarkResult runBenchmark(
+            String name,
+            StreamTableEnvironment tEnv,
+            Model<?> model,
+            DataGenerator<?> modelDataGenerator,
+            DataGenerator<?> inputsGenerator)
+            throws Exception {
+        model.setModelData(modelDataGenerator.getData(tEnv));
+
+        Table[] outputTables = model.transform(inputsGenerator.getData(tEnv));
+
+        for (Table table : outputTables) {
+            tEnv.toDataStream(table).addSink(new DiscardingSink<>());
+        }
+
+        StreamExecutionEnvironment env = ((StreamTableEnvironmentImpl) tEnv).execEnv();
+        JobExecutionResult executionResult = env.execute();
+
+        BenchmarkResult result = new BenchmarkResult();
+        result.name = name;
+        result.executionTimeMillis = (double) executionResult.getNetRuntime(TimeUnit.MILLISECONDS);
+
+        return result;
+    }
+
+    /**
+     * Instantiates a WithParams subclass from the provided json map.
+     *
+     * @param jsonMap a map containing className and paramMap.
+     * @return the instantiated WithParams subclass.
+     */
+    public static WithParams<?> instantiateWithParams(Map<String, ?> jsonMap) throws Exception {

Review comment:
       It appears that this method shares a lot of code with `ReadWriteUtils::loadStageParam(...)`. How about we add this method to ReadWriteUtils and update `loadStageParam(...)` to use `instantiateWithParams(...)`?

##########
File path: flink-ml-benchmark/README.md
##########
@@ -0,0 +1,261 @@
+# Flink ML Benchmark Guideline
+
+This document provides instructions about how to run benchmarks on Flink ML's
+stages.
+
+## Write Benchmark Programs
+
+### Add Maven Dependencies
+
+In order to write Flink ML's java benchmark programs, first make sure that the
+following dependencies have been added to your maven project's `pom.xml`.
+
+```xml
+<dependency>

Review comment:
       Is this the minimal dependencies that user need to explicitly put in the pom.xml? Could we simplify it by using flink-ml-uber?




-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org