/
MatrixVectorMul.java
253 lines (217 loc) · 10.3 KB
/
MatrixVectorMul.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
/*
* 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.eventtime.WatermarkStrategy;
import org.apache.flink.api.common.externalresource.ExternalResourceInfo;
import org.apache.flink.api.common.functions.OpenContext;
import org.apache.flink.api.common.functions.RichMapFunction;
import org.apache.flink.api.common.serialization.SimpleStringEncoder;
import org.apache.flink.api.common.typeinfo.Types;
import org.apache.flink.api.java.utils.ParameterTool;
import org.apache.flink.connector.datagen.source.DataGeneratorSource;
import org.apache.flink.connector.datagen.source.GeneratorFunction;
import org.apache.flink.connector.file.sink.FileSink;
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.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;
/**
* Implements the matrix-vector multiplication program that shows how to use GPU resources in Flink.
*
* <p>The input is a vector stream, 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 in your Flink distribution by the
* following steps:
*
* <ul>
* <li>download the JCuda native libraries bundle for your CUDA version from
* http://www.jcuda.org/downloads/
* <li>copy the native libraries jcuda-natives and jcublas-natives for your CUDA version,
* operating system and architecture to the "lib/" folder of your Flink distribution
* </ul>
*/
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);
final int dimension = params.getInt("dimension", DEFAULT_DIM);
final int dataSize = params.getInt("data-size", DEFAULT_DATA_SIZE);
final String resourceName = params.get("resource-name", DEFAULT_RESOURCE_NAME);
GeneratorFunction<Long, List<Float>> generatorFunction =
index -> {
List<Float> randomRecord = new ArrayList<>();
for (int i = 0; i < dimension; ++i) {
randomRecord.add((float) Math.random());
}
return randomRecord;
};
// Generates random vectors with specified dimension
DataGeneratorSource<List<Float>> generatorSource =
new DataGeneratorSource<>(generatorFunction, dataSize, Types.LIST(Types.FLOAT));
DataStream<List<Float>> result =
env.fromSource(generatorSource, WatermarkStrategy.noWatermarks(), "Vectors Source")
.map(new Multiplier(dimension, resourceName));
// Emit result
if (params.has("output")) {
result.sinkTo(
FileSink.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
// *************************************************************************
/** 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(OpenContext openContext) {
// 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.
// For more details please refer to https://issues.apache.org/jira/browse/FLINK-5408 and
// the discussion in
// http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Classloader-and-removal-of-native-libraries-td14808.html
final String originTempDir = System.getProperty("java.io.tmpdir");
final String newTempDir = originTempDir + "/jcuda-" + UUID.randomUUID();
System.setProperty("java.io.tmpdir", newTempDir);
final Set<ExternalResourceInfo> externalResourceInfos =
getRuntimeContext().getExternalResourceInfos(resourceName);
Preconditions.checkState(
!externalResourceInfos.isEmpty(),
"The MatrixVectorMul needs at least one GPU device while finding 0 GPU.");
final Optional<String> firstIndexOptional =
externalResourceInfos.iterator().next().getProperty("index");
Preconditions.checkState(firstIndexOptional.isPresent());
matrixPointer = new Pointer();
final float[] matrix = new float[dimension * dimension];
// Initialize a random matrix
for (int i = 0; i < dimension * dimension; ++i) {
matrix[i] = (float) Math.random();
}
// Set the CUDA device
JCuda.cudaSetDevice(Integer.parseInt(firstIndexOptional.get()));
// Initialize JCublas
JCublas.cublasInit();
// Allocate device memory for the matrix
JCublas.cublasAlloc(dimension * dimension, Sizeof.FLOAT, matrixPointer);
JCublas.cublasSetVector(
dimension * dimension, Sizeof.FLOAT, Pointer.to(matrix), 1, matrixPointer, 1);
// Change the java.io.tmpdir back to its original value.
System.setProperty("java.io.tmpdir", originTempDir);
}
@Override
public List<Float> map(List<Float> value) {
final float[] input = new float[dimension];
final float[] output = new float[dimension];
final Pointer inputPointer = new Pointer();
final Pointer outputPointer = new Pointer();
// Fill the input and output vector
for (int i = 0; i < dimension; i++) {
input[i] = value.get(i);
output[i] = 0;
}
// Allocate device memory for the input and output
JCublas.cublasAlloc(dimension, Sizeof.FLOAT, inputPointer);
JCublas.cublasAlloc(dimension, Sizeof.FLOAT, outputPointer);
// Initialize the device matrices
JCublas.cublasSetVector(dimension, Sizeof.FLOAT, Pointer.to(input), 1, inputPointer, 1);
JCublas.cublasSetVector(
dimension, Sizeof.FLOAT, Pointer.to(output), 1, outputPointer, 1);
// Performs operation using JCublas
JCublas.cublasSgemv(
'n',
dimension,
dimension,
1.0f,
matrixPointer,
dimension,
inputPointer,
1,
0.0f,
outputPointer,
1);
// Read the result back
JCublas.cublasGetVector(
dimension, Sizeof.FLOAT, outputPointer, 1, Pointer.to(output), 1);
// Memory clean up
JCublas.cublasFree(inputPointer);
JCublas.cublasFree(outputPointer);
List<Float> outputList = new ArrayList<>();
for (int i = 0; i < dimension; ++i) {
outputList.add(output[i]);
}
return outputList;
}
@Override
public void close() {
// Memory clean up
JCublas.cublasFree(matrixPointer);
// Shutdown cublas
JCublas.cublasShutdown();
}
}
}