tillrohrmann commented on a change in pull request #12398:
URL: https://github.com/apache/flink/pull/12398#discussion_r436680451
##########
File path: flink-examples/flink-examples-streaming/pom.xml
##########
@@ -88,6 +93,31 @@ under the License.
<version>${project.version}</version>
</dependency>
+ <!-- dependencies for MatrixVectorMul -->
+ <dependency>
+ <groupId>org.jcuda</groupId>
+ <artifactId>jcuda</artifactId>
+ <version>${jcuda.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.jcuda</groupId>
+ <artifactId>jcuda-natives</artifactId>
+ </exclusion>
Review comment:
Please add a comment why we can exclude this dependency.
##########
File path: flink-dist/src/main/assemblies/bin.xml
##########
@@ -84,6 +84,14 @@ under the License.
<fileMode>0644</fileMode>
</file>
+ <!-- copy helper script for MatrixVectorMul example -->
+ <file>
+
<source>../flink-examples/flink-examples-streaming/src/main/resources/add-jcuda-dependency.sh</source>
+ <outputDirectory>examples/streaming</outputDirectory>
+ <destName>add-jcuda-dependency.sh</destName>
+ <fileMode>0755</fileMode>
+ </file>
Review comment:
I wouldn't include this example into the resulting `flink-dist`.
##########
File path:
flink-examples/flink-examples-streaming/src/main/resources/add-jcuda-dependency.sh
##########
@@ -0,0 +1,56 @@
+#!/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.
+################################################################################
+
+echo "Usage: ./add-jcuda-dependency.sh <cuda-version>"
+
+CUDA_VERSION="${1:-10.0.0}"
+case $(uname -s) in
+ Darwin*)
+ OS_NAME="apple"
+ ;;
+ Linux*)
+ OS_NAME="linux"
+ ;;
+ CYGWIN*)
+ OS_NAME="windows"
+ ;;
+ *)
+ echo "Could not recognize current os" $(uname -s)
+ exit 1
+ ;;
+esac
+OS_ARCH=$(uname -m)
+
+JCUDA_URL="http://www.jcuda.org/downloads/JCuda-All-$CUDA_VERSION.zip"
+JCUDA_NATIVE_LIB="jcuda-natives-$CUDA_VERSION-$OS_NAME-$OS_ARCH.jar"
+JCUBLAS_NATIVE_LIB="jcublas-natives-$CUDA_VERSION-$OS_NAME-$OS_ARCH.jar"
+
+# Download and move the native libraries to "lib/" of Flink distribution
+wget $JCUDA_URL -O jcuda.zip
+unzip jcuda.zip
+
+if [ -e ./JCuda-All-$CUDA_VERSION/$JCUDA_NATIVE_LIB ] && [ -e
./JCuda-All-$CUDA_VERSION/$JCUBLAS_NATIVE_LIB ]; then
+ cp
./JCuda-All-$CUDA_VERSION/jcublas-natives-$CUDA_VERSION-$OS_NAME-$OS_ARCH.jar
$(dirname "$0")/../../lib
+ cp
./JCuda-All-$CUDA_VERSION/jcuda-natives-$CUDA_VERSION-$OS_NAME-$OS_ARCH.jar
$(dirname "$0")/../../lib
Review comment:
`$(dirname "$0")/../../lib` looks a bit brittle because it requires that
the script is always located two directories below Flink home. Maybe it could
help to specify the lib directory with a default value of `../../lib`.
##########
File path:
flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/gpu/MatrixVectorMul.java
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.streaming.examples.gpu;
+
+import org.apache.flink.api.common.externalresource.ExternalResourceInfo;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.serialization.SimpleStringEncoder;
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import
org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.util.Preconditions;
+
+import jcuda.Pointer;
+import jcuda.Sizeof;
+import jcuda.jcublas.JCublas;
+import jcuda.runtime.JCuda;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+
+/**
+ * Implement the matrix-vector multiplication program that show how to
leverage GPU resources in Flink.
+ *
+ * <p>The input is a vector stream from a {@link RandomVectorSource}, which
will generate random vectors with specified
+ * dimension. The data size of the vector stream could be specified by user.
Each vector will be multiplied with a random
+ * dimension * dimension matrix in {@link Multiplier} and the result would be
emitted to output.
+ *
+ * <p>Usage: MatrixVectorMul [--output <path>] [--dimension
<dimension> --data-size <data_size>]
+ *
+ * <p>If no parameters are provided, the program is run with default vector
dimension 10 and data size 100.
+ *
+ * <p>This example shows how to:
+ * <ul>
+ * <li>leverage external resource in operators,
+ * <li>accelerate complex calculation with GPU resources.
+ * </ul>
+ *
+ * <p>Notice that you need to add jcuda natives libraries by executing
example/streaming/add-jcuda-dependency.sh in your Flink distribution.
+ */
+public class MatrixVectorMul {
+
+ private static final int DEFAULT_DIM = 10;
+ private static final int DEFAULT_DATA_SIZE = 100;
+ private static final String DEFAULT_RESOURCE_NAME = "gpu";
+
+ public static void main(String[] args) throws Exception {
+
+ // Checking input parameters
+ final ParameterTool params = ParameterTool.fromArgs(args);
+ System.out.println("Usage: MatrixVectorMul [--output <path>]
[--dimension <dimension> --data-size <data_size>] [--resource-name
<resource_name>]");
+
+ // Set up the execution environment
+ final StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
+
+ // Make parameters available in the web interface
+ env.getConfig().setGlobalJobParameters(params);
+
+ int dimension;
+ if (params.has("dimension")) {
+ dimension = params.getInt("dimension");
Review comment:
You could also use `params.getInt("dimension", DEFAULT_DIM)`
##########
File path:
flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/gpu/MatrixVectorMul.java
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.streaming.examples.gpu;
+
+import org.apache.flink.api.common.externalresource.ExternalResourceInfo;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.serialization.SimpleStringEncoder;
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import
org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.util.Preconditions;
+
+import jcuda.Pointer;
+import jcuda.Sizeof;
+import jcuda.jcublas.JCublas;
+import jcuda.runtime.JCuda;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+
+/**
+ * Implement the matrix-vector multiplication program that show how to
leverage GPU resources in Flink.
+ *
+ * <p>The input is a vector stream from a {@link RandomVectorSource}, which
will generate random vectors with specified
+ * dimension. The data size of the vector stream could be specified by user.
Each vector will be multiplied with a random
+ * dimension * dimension matrix in {@link Multiplier} and the result would be
emitted to output.
+ *
+ * <p>Usage: MatrixVectorMul [--output <path>] [--dimension
<dimension> --data-size <data_size>]
+ *
+ * <p>If no parameters are provided, the program is run with default vector
dimension 10 and data size 100.
+ *
+ * <p>This example shows how to:
+ * <ul>
+ * <li>leverage external resource in operators,
+ * <li>accelerate complex calculation with GPU resources.
+ * </ul>
+ *
+ * <p>Notice that you need to add jcuda natives libraries by executing
example/streaming/add-jcuda-dependency.sh in your Flink distribution.
+ */
+public class MatrixVectorMul {
+
+ private static final int DEFAULT_DIM = 10;
+ private static final int DEFAULT_DATA_SIZE = 100;
+ private static final String DEFAULT_RESOURCE_NAME = "gpu";
+
+ public static void main(String[] args) throws Exception {
+
+ // Checking input parameters
+ final ParameterTool params = ParameterTool.fromArgs(args);
+ System.out.println("Usage: MatrixVectorMul [--output <path>]
[--dimension <dimension> --data-size <data_size>] [--resource-name
<resource_name>]");
+
+ // Set up the execution environment
+ final StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
+
+ // Make parameters available in the web interface
+ env.getConfig().setGlobalJobParameters(params);
+
+ int dimension;
+ if (params.has("dimension")) {
+ dimension = params.getInt("dimension");
+ } else {
+ System.out.println(String.format("Executing
MatrixVectorMul example with default vector dimension %d.\nUse --dimension to
specify dimension of input vector.", DEFAULT_DIM));
+ dimension = DEFAULT_DIM;
+ }
+
+ int dataSize;
+ if (params.has("data-size")) {
+ dataSize = params.getInt("data-size");
+ } else {
+ System.out.println(String.format("Executing
MatrixVectorMul example with default data size %d.\nUse --data-size to specify
size of the vector stream.", DEFAULT_DATA_SIZE));
+ dataSize = DEFAULT_DATA_SIZE;
+ }
+
+ String resourceName;
+ if (params.has("resource-name")) {
+ resourceName = params.get("resource-name");
+ } else {
+ System.out.println(String.format("Executing
MatrixVectorMul example with default resource name %s.\nUse --resource-name to
specify resource name of GPU.", DEFAULT_RESOURCE_NAME));
+ resourceName = DEFAULT_RESOURCE_NAME;
+ }
+
+ DataStream<List<Float>> result = env.addSource(new
RandomVectorSource(dimension, dataSize))
+ .map(new Multiplier(dimension,
resourceName));
+
+ // Emit result
+ if (params.has("output")) {
+ result.addSink(StreamingFileSink.forRowFormat(new
Path(params.get("output")),
+ new
SimpleStringEncoder<List<Float>>()).build());
+ } else {
+ System.out.println("Printing result to stdout. Use
--output to specify output path.");
+ result.print();
+ }
+ // Execute program
+ env.execute("Matrix-Vector Multiplication");
+ }
+
+ //
*************************************************************************
+ // USER FUNCTIONS
+ //
*************************************************************************
+
+ /**
+ * Random vector source which generates random vectors with specified
dimension and total data size.
+ */
+ private static final class RandomVectorSource extends
RichSourceFunction<List<Float>> {
+
+ private transient volatile boolean running;
+ private final int dimension;
+ private final int dataSize;
+
+ RandomVectorSource(int dimension, int dataSize) {
+ this.dimension = dimension;
+ this.dataSize = dataSize;
+ }
+
+ @Override
+ public void open(Configuration parameters) {
+ running = true;
+ }
+
+ @Override
+ public void run(SourceContext<List<Float>> ctx) {
+ int count = 0;
+ while (running && count < dataSize) {
+ List<Float> randomRecord = new ArrayList<>();
+ for (int i = 0; i < dimension; ++i) {
+ randomRecord.add((float) Math.random());
+ }
+ ctx.collect(randomRecord);
+ count += 1;
+ }
+ }
+
+ @Override
+ public void cancel() {
+ running = false;
+ }
+ }
+
+ /**
+ * Matrix-Vector multiplier using CUBLAS library.
+ */
+ private static final class Multiplier extends
RichMapFunction<List<Float>, List<Float>> {
+ private final int dimension;
+ private final String resourceName;
+ private Pointer matrixPointer;
+
+ Multiplier(int dimension, String resourceName) {
+ this.dimension = dimension;
+ this.resourceName = resourceName;
+ }
+
+ @Override
+ public void open(Configuration parameters) {
+ // When multiple instances of this class and JCuda
exist in different class loaders, then we will get UnsatisfiedLinkError.
+ // To avoid that, we need to temporarily override the
java.io.tmpdir, where the JCuda store its native library, with a random path.
Review comment:
Can you explain the problem a bit more in detail? I would like to
understand what's the underlying problem. Is there an official issue describing
the problem?
##########
File path: flink-examples/flink-examples-streaming/pom.xml
##########
@@ -88,6 +93,31 @@ under the License.
<version>${project.version}</version>
</dependency>
+ <!-- dependencies for MatrixVectorMul -->
+ <dependency>
+ <groupId>org.jcuda</groupId>
+ <artifactId>jcuda</artifactId>
+ <version>${jcuda.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.jcuda</groupId>
+ <artifactId>jcuda-natives</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+
+ <dependency>
+ <groupId>org.jcuda</groupId>
+ <artifactId>jcublas</artifactId>
+ <version>${jcuda.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.jcuda</groupId>
+ <artifactId>jcublas-natives</artifactId>
+ </exclusion>
Review comment:
Same here. Why can we exclude it.
##########
File path:
flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/gpu/MatrixVectorMul.java
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.streaming.examples.gpu;
+
+import org.apache.flink.api.common.externalresource.ExternalResourceInfo;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.serialization.SimpleStringEncoder;
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import
org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.util.Preconditions;
+
+import jcuda.Pointer;
+import jcuda.Sizeof;
+import jcuda.jcublas.JCublas;
+import jcuda.runtime.JCuda;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+
+/**
+ * Implement the matrix-vector multiplication program that show how to
leverage GPU resources in Flink.
Review comment:
```suggestion
* Implements the matrix-vector multiplication program that shows how to use
GPU resources in Flink.
```
##########
File path: flink-examples/flink-examples-streaming/pom.xml
##########
@@ -365,6 +395,42 @@ under the License.
</executions>
</plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-shade-plugin</artifactId>
+ <executions>
+ <execution>
+ <id>MatrixVectorMul</id>
+ <phase>package</phase>
+ <goals>
+ <goal>shade</goal>
+ </goals>
+ <configuration>
+
<shadeTestJar>false</shadeTestJar>
+
<finalName>MatrixVectorMul</finalName>
+ <artifactSet>
+ <includes>
+
<include>org.jcuda:*</include>
+ </includes>
+ </artifactSet>
+ <filters>
+ <filter>
+
<artifact>org.apache.flink:*</artifact>
+
<includes>
+
<include>org/apache/flink/streaming/examples/gpu/MatrixVectorMul.class</include>
+
<include>org/apache/flink/streaming/examples/gpu/MatrixVectorMul$*.class</include>
+
</includes>
+ </filter>
+ </filters>
+ <transformers>
+ <transformer
implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
+
<mainClass>org.apache.flink.streaming.examples.gpu.MatrixVectorMul</mainClass>
+ </transformer>
+ </transformers>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
Review comment:
I wouldn't include this example in `flink-dist`. I think it should be
good enough if users can find this example in the Flink repository/source code
distribution.
##########
File path: flink-examples/flink-examples-streaming/pom.xml
##########
@@ -88,6 +93,31 @@ under the License.
<version>${project.version}</version>
</dependency>
+ <!-- dependencies for MatrixVectorMul -->
+ <dependency>
+ <groupId>org.jcuda</groupId>
+ <artifactId>jcuda</artifactId>
+ <version>${jcuda.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.jcuda</groupId>
+ <artifactId>jcuda-natives</artifactId>
+ </exclusion>
Review comment:
Thanks for the clarification. I meant an in-code comment so that other
devs know why this exclusion is there.
##########
File path:
flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/gpu/MatrixVectorMul.java
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.streaming.examples.gpu;
+
+import org.apache.flink.api.common.externalresource.ExternalResourceInfo;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.serialization.SimpleStringEncoder;
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import
org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.util.Preconditions;
+
+import jcuda.Pointer;
+import jcuda.Sizeof;
+import jcuda.jcublas.JCublas;
+import jcuda.runtime.JCuda;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+
+/**
+ * Implement the matrix-vector multiplication program that show how to
leverage GPU resources in Flink.
+ *
+ * <p>The input is a vector stream from a {@link RandomVectorSource}, which
will generate random vectors with specified
+ * dimension. The data size of the vector stream could be specified by user.
Each vector will be multiplied with a random
+ * dimension * dimension matrix in {@link Multiplier} and the result would be
emitted to output.
+ *
+ * <p>Usage: MatrixVectorMul [--output <path>] [--dimension
<dimension> --data-size <data_size>]
+ *
+ * <p>If no parameters are provided, the program is run with default vector
dimension 10 and data size 100.
+ *
+ * <p>This example shows how to:
+ * <ul>
+ * <li>leverage external resource in operators,
+ * <li>accelerate complex calculation with GPU resources.
+ * </ul>
+ *
+ * <p>Notice that you need to add jcuda natives libraries by executing
example/streaming/add-jcuda-dependency.sh in your Flink distribution.
+ */
+public class MatrixVectorMul {
+
+ private static final int DEFAULT_DIM = 10;
+ private static final int DEFAULT_DATA_SIZE = 100;
+ private static final String DEFAULT_RESOURCE_NAME = "gpu";
+
+ public static void main(String[] args) throws Exception {
+
+ // Checking input parameters
+ final ParameterTool params = ParameterTool.fromArgs(args);
+ System.out.println("Usage: MatrixVectorMul [--output <path>]
[--dimension <dimension> --data-size <data_size>] [--resource-name
<resource_name>]");
+
+ // Set up the execution environment
+ final StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
+
+ // Make parameters available in the web interface
+ env.getConfig().setGlobalJobParameters(params);
+
+ int dimension;
+ if (params.has("dimension")) {
+ dimension = params.getInt("dimension");
+ } else {
+ System.out.println(String.format("Executing
MatrixVectorMul example with default vector dimension %d.\nUse --dimension to
specify dimension of input vector.", DEFAULT_DIM));
+ dimension = DEFAULT_DIM;
+ }
+
+ int dataSize;
+ if (params.has("data-size")) {
+ dataSize = params.getInt("data-size");
+ } else {
+ System.out.println(String.format("Executing
MatrixVectorMul example with default data size %d.\nUse --data-size to specify
size of the vector stream.", DEFAULT_DATA_SIZE));
+ dataSize = DEFAULT_DATA_SIZE;
+ }
+
+ String resourceName;
+ if (params.has("resource-name")) {
+ resourceName = params.get("resource-name");
+ } else {
+ System.out.println(String.format("Executing
MatrixVectorMul example with default resource name %s.\nUse --resource-name to
specify resource name of GPU.", DEFAULT_RESOURCE_NAME));
+ resourceName = DEFAULT_RESOURCE_NAME;
+ }
+
+ DataStream<List<Float>> result = env.addSource(new
RandomVectorSource(dimension, dataSize))
+ .map(new Multiplier(dimension,
resourceName));
+
+ // Emit result
+ if (params.has("output")) {
+ result.addSink(StreamingFileSink.forRowFormat(new
Path(params.get("output")),
+ new
SimpleStringEncoder<List<Float>>()).build());
+ } else {
+ System.out.println("Printing result to stdout. Use
--output to specify output path.");
+ result.print();
+ }
+ // Execute program
+ env.execute("Matrix-Vector Multiplication");
+ }
+
+ //
*************************************************************************
+ // USER FUNCTIONS
+ //
*************************************************************************
+
+ /**
+ * Random vector source which generates random vectors with specified
dimension and total data size.
+ */
+ private static final class RandomVectorSource extends
RichSourceFunction<List<Float>> {
+
+ private transient volatile boolean running;
+ private final int dimension;
+ private final int dataSize;
+
+ RandomVectorSource(int dimension, int dataSize) {
+ this.dimension = dimension;
+ this.dataSize = dataSize;
+ }
+
+ @Override
+ public void open(Configuration parameters) {
+ running = true;
+ }
+
+ @Override
+ public void run(SourceContext<List<Float>> ctx) {
+ int count = 0;
+ while (running && count < dataSize) {
+ List<Float> randomRecord = new ArrayList<>();
+ for (int i = 0; i < dimension; ++i) {
+ randomRecord.add((float) Math.random());
+ }
+ ctx.collect(randomRecord);
+ count += 1;
+ }
+ }
+
+ @Override
+ public void cancel() {
+ running = false;
+ }
+ }
+
+ /**
+ * Matrix-Vector multiplier using CUBLAS library.
+ */
+ private static final class Multiplier extends
RichMapFunction<List<Float>, List<Float>> {
+ private final int dimension;
+ private final String resourceName;
+ private Pointer matrixPointer;
+
+ Multiplier(int dimension, String resourceName) {
+ this.dimension = dimension;
+ this.resourceName = resourceName;
+ }
+
+ @Override
+ public void open(Configuration parameters) {
+ // When multiple instances of this class and JCuda
exist in different class loaders, then we will get UnsatisfiedLinkError.
+ // To avoid that, we need to temporarily override the
java.io.tmpdir, where the JCuda store its native library, with a random path.
Review comment:
Could you add these pointers to the in-code comments so that other devs
can find it?
##########
File path:
flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/gpu/MatrixVectorMul.java
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.streaming.examples.gpu;
+
+import org.apache.flink.api.common.externalresource.ExternalResourceInfo;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.serialization.SimpleStringEncoder;
+import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import
org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.util.Preconditions;
+
+import jcuda.Pointer;
+import jcuda.Sizeof;
+import jcuda.jcublas.JCublas;
+import jcuda.runtime.JCuda;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+
+/**
+ * Implement the matrix-vector multiplication program that show how to
leverage GPU resources in Flink.
+ *
+ * <p>The input is a vector stream from a {@link RandomVectorSource}, which
will generate random vectors with specified
+ * dimension. The data size of the vector stream could be specified by user.
Each vector will be multiplied with a random
+ * dimension * dimension matrix in {@link Multiplier} and the result would be
emitted to output.
+ *
+ * <p>Usage: MatrixVectorMul [--output <path>] [--dimension
<dimension> --data-size <data_size>]
+ *
+ * <p>If no parameters are provided, the program is run with default vector
dimension 10 and data size 100.
+ *
+ * <p>This example shows how to:
+ * <ul>
+ * <li>leverage external resource in operators,
+ * <li>accelerate complex calculation with GPU resources.
+ * </ul>
+ *
+ * <p>Notice that you need to add jcuda natives libraries by executing
example/streaming/add-jcuda-dependency.sh in your Flink distribution.
+ */
+public class MatrixVectorMul {
+
+ private static final int DEFAULT_DIM = 10;
+ private static final int DEFAULT_DATA_SIZE = 100;
+ private static final String DEFAULT_RESOURCE_NAME = "gpu";
+
+ public static void main(String[] args) throws Exception {
+
+ // Checking input parameters
+ final ParameterTool params = ParameterTool.fromArgs(args);
+ System.out.println("Usage: MatrixVectorMul [--output <path>]
[--dimension <dimension> --data-size <data_size>] [--resource-name
<resource_name>]");
+
+ // Set up the execution environment
+ final StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
+
+ // Make parameters available in the web interface
+ env.getConfig().setGlobalJobParameters(params);
+
+ int dimension;
+ if (params.has("dimension")) {
+ dimension = params.getInt("dimension");
+ } else {
+ System.out.println(String.format("Executing
MatrixVectorMul example with default vector dimension %d.\nUse --dimension to
specify dimension of input vector.", DEFAULT_DIM));
+ dimension = DEFAULT_DIM;
+ }
+
+ int dataSize;
+ if (params.has("data-size")) {
+ dataSize = params.getInt("data-size");
+ } else {
+ System.out.println(String.format("Executing
MatrixVectorMul example with default data size %d.\nUse --data-size to specify
size of the vector stream.", DEFAULT_DATA_SIZE));
+ dataSize = DEFAULT_DATA_SIZE;
+ }
+
+ String resourceName;
+ if (params.has("resource-name")) {
+ resourceName = params.get("resource-name");
+ } else {
+ System.out.println(String.format("Executing
MatrixVectorMul example with default resource name %s.\nUse --resource-name to
specify resource name of GPU.", DEFAULT_RESOURCE_NAME));
+ resourceName = DEFAULT_RESOURCE_NAME;
+ }
+
+ DataStream<List<Float>> result = env.addSource(new
RandomVectorSource(dimension, dataSize))
+ .map(new Multiplier(dimension,
resourceName));
+
+ // Emit result
+ if (params.has("output")) {
+ result.addSink(StreamingFileSink.forRowFormat(new
Path(params.get("output")),
+ new
SimpleStringEncoder<List<Float>>()).build());
+ } else {
+ System.out.println("Printing result to stdout. Use
--output to specify output path.");
+ result.print();
+ }
+ // Execute program
+ env.execute("Matrix-Vector Multiplication");
+ }
+
+ //
*************************************************************************
+ // USER FUNCTIONS
+ //
*************************************************************************
+
+ /**
+ * Random vector source which generates random vectors with specified
dimension and total data size.
+ */
+ private static final class RandomVectorSource extends
RichSourceFunction<List<Float>> {
+
+ private transient volatile boolean running;
+ private final int dimension;
+ private final int dataSize;
+
+ RandomVectorSource(int dimension, int dataSize) {
+ this.dimension = dimension;
+ this.dataSize = dataSize;
+ }
+
+ @Override
+ public void open(Configuration parameters) {
+ running = true;
+ }
+
+ @Override
+ public void run(SourceContext<List<Float>> ctx) {
+ int count = 0;
+ while (running && count < dataSize) {
+ List<Float> randomRecord = new ArrayList<>();
+ for (int i = 0; i < dimension; ++i) {
+ randomRecord.add((float) Math.random());
+ }
+ ctx.collect(randomRecord);
+ count += 1;
+ }
+ }
+
+ @Override
+ public void cancel() {
+ running = false;
+ }
+ }
+
+ /**
+ * Matrix-Vector multiplier using CUBLAS library.
+ */
+ private static final class Multiplier extends
RichMapFunction<List<Float>, List<Float>> {
+ private final int dimension;
+ private final String resourceName;
+ private Pointer matrixPointer;
+
+ Multiplier(int dimension, String resourceName) {
+ this.dimension = dimension;
+ this.resourceName = resourceName;
+ }
+
+ @Override
+ public void open(Configuration parameters) {
+ // When multiple instances of this class and JCuda
exist in different class loaders, then we will get UnsatisfiedLinkError.
+ // To avoid that, we need to temporarily override the
java.io.tmpdir, where the JCuda store its native library, with a random path.
Review comment:
Could you try whether this problem still exists because we have changed
how user code class loaders are instantiated and how long the live with the
1.11 release (see https://issues.apache.org/jira/browse/FLINK-16408)
##########
File path: flink-examples/flink-examples-streaming/pom.xml
##########
@@ -365,6 +395,42 @@ under the License.
</executions>
</plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-shade-plugin</artifactId>
+ <executions>
+ <execution>
+ <id>MatrixVectorMul</id>
+ <phase>package</phase>
+ <goals>
+ <goal>shade</goal>
+ </goals>
+ <configuration>
+
<shadeTestJar>false</shadeTestJar>
+
<finalName>MatrixVectorMul</finalName>
+ <artifactSet>
+ <includes>
+
<include>org.jcuda:*</include>
+ </includes>
+ </artifactSet>
+ <filters>
+ <filter>
+
<artifact>org.apache.flink:*</artifact>
+
<includes>
+
<include>org/apache/flink/streaming/examples/gpu/MatrixVectorMul.class</include>
+
<include>org/apache/flink/streaming/examples/gpu/MatrixVectorMul$*.class</include>
+
</includes>
+ </filter>
+ </filters>
+ <transformers>
+ <transformer
implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
+
<mainClass>org.apache.flink.streaming.examples.gpu.MatrixVectorMul</mainClass>
+ </transformer>
+ </transformers>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
Review comment:
You are right. We should still build the self-contained jars. The only
thing which we shouldn't do is to ship it with `flink-dist`.
----------------------------------------------------------------
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.
For queries about this service, please contact Infrastructure at:
[email protected]