-
Notifications
You must be signed in to change notification settings - Fork 4.2k
/
FlinkJobServerDriver.java
255 lines (228 loc) · 9.86 KB
/
FlinkJobServerDriver.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.beam.runners.flink;
import com.google.common.base.Strings;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import java.io.IOException;
import java.nio.file.Paths;
import java.util.concurrent.Executors;
import java.util.concurrent.ThreadFactory;
import org.apache.beam.model.pipeline.v1.Endpoints;
import org.apache.beam.runners.fnexecution.GrpcFnServer;
import org.apache.beam.runners.fnexecution.ServerFactory;
import org.apache.beam.runners.fnexecution.artifact.BeamFileSystemArtifactStagingService;
import org.apache.beam.runners.fnexecution.jobsubmission.InMemoryJobService;
import org.apache.beam.runners.fnexecution.jobsubmission.JobInvoker;
import org.apache.beam.sdk.io.FileSystems;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.options.PortablePipelineOptions;
import org.kohsuke.args4j.CmdLineException;
import org.kohsuke.args4j.CmdLineParser;
import org.kohsuke.args4j.Option;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/** Driver program that starts a job server. */
public class FlinkJobServerDriver implements Runnable {
private static final Logger LOG = LoggerFactory.getLogger(FlinkJobServerDriver.class);
private final ListeningExecutorService executor;
private final ServerConfiguration configuration;
private final ServerFactory jobServerFactory;
private final ServerFactory artifactServerFactory;
private GrpcFnServer<InMemoryJobService> jobServer;
private GrpcFnServer<BeamFileSystemArtifactStagingService> artifactStagingServer;
/** Configuration for the jobServer. */
public static class ServerConfiguration {
@Option(name = "--job-host", usage = "The job server host name")
private String host = "";
@Option(name = "--job-port", usage = "The job service port. (Default: 8099)")
private int port = 8099;
@Option(name = "--artifact-port", usage = "The artifact service port. (Default: 8098)")
private int artifactPort = 8098;
@Option(name = "--artifacts-dir", usage = "The location to store staged artifact files")
private String artifactStagingPath =
Paths.get(System.getProperty("java.io.tmpdir"), "beam-artifact-staging").toString();
@Option(
name = "--clean-artifacts-per-job",
usage = "When true, remove each job's staged artifacts when it completes"
)
private Boolean cleanArtifactsPerJob = false;
@Option(name = "--flink-master-url", usage = "Flink master url to submit job.")
private String flinkMasterUrl = "[auto]";
public String getFlinkMasterUrl() {
return this.flinkMasterUrl;
}
@Option(
name = "--sdk-worker-parallelism",
usage = "Default parallelism for SDK worker processes (see portable pipeline options)"
)
private String sdkWorkerParallelism = PortablePipelineOptions.SDK_WORKER_PARALLELISM_PIPELINE;
public String getSdkWorkerParallelism() {
return this.sdkWorkerParallelism;
}
}
public static void main(String[] args) throws Exception {
//TODO: Expose the fileSystem related options.
// Register standard file systems.
FileSystems.setDefaultPipelineOptions(PipelineOptionsFactory.create());
fromParams(args).run();
}
private static void printUsage(CmdLineParser parser) {
System.err.println(
String.format("Usage: java %s arguments...", FlinkJobServerDriver.class.getSimpleName()));
parser.printUsage(System.err);
System.err.println();
}
public static FlinkJobServerDriver fromParams(String[] args) {
ServerConfiguration configuration = new ServerConfiguration();
CmdLineParser parser = new CmdLineParser(configuration);
try {
parser.parseArgument(args);
} catch (CmdLineException e) {
LOG.error("Unable to parse command line arguments.", e);
printUsage(parser);
throw new IllegalArgumentException("Unable to parse command line arguments.", e);
}
return fromConfig(configuration);
}
public static FlinkJobServerDriver fromConfig(ServerConfiguration configuration) {
ThreadFactory threadFactory =
new ThreadFactoryBuilder().setNameFormat("flink-runner-job-server").setDaemon(true).build();
ListeningExecutorService executor =
MoreExecutors.listeningDecorator(Executors.newCachedThreadPool(threadFactory));
ServerFactory jobServerFactory = ServerFactory.createWithPortSupplier(() -> configuration.port);
ServerFactory artifactServerFactory =
ServerFactory.createWithPortSupplier(() -> configuration.artifactPort);
return create(configuration, executor, jobServerFactory, artifactServerFactory);
}
public static FlinkJobServerDriver create(
ServerConfiguration configuration,
ListeningExecutorService executor,
ServerFactory jobServerFactory,
ServerFactory artifactServerFactory) {
return new FlinkJobServerDriver(
configuration, executor, jobServerFactory, artifactServerFactory);
}
private FlinkJobServerDriver(
ServerConfiguration configuration,
ListeningExecutorService executor,
ServerFactory jobServerFactory,
ServerFactory artifactServerFactory) {
this.configuration = configuration;
this.executor = executor;
this.jobServerFactory = jobServerFactory;
this.artifactServerFactory = artifactServerFactory;
}
@Override
public void run() {
try {
jobServer = createJobServer();
jobServer.getServer().awaitTermination();
} catch (InterruptedException e) {
LOG.warn("Job server interrupted", e);
} catch (Exception e) {
LOG.warn("Exception during job server creation", e);
} finally {
stop();
}
}
public String start() throws IOException {
jobServer = createJobServer();
return jobServer.getApiServiceDescriptor().getUrl();
}
public void stop() {
if (jobServer != null) {
try {
jobServer.close();
LOG.info("JobServer stopped on {}", jobServer.getApiServiceDescriptor().getUrl());
jobServer = null;
} catch (Exception e) {
LOG.error("Error while closing the jobServer.", e);
}
}
if (artifactStagingServer != null) {
try {
artifactStagingServer.close();
LOG.info(
"ArtifactStagingServer stopped on {}",
artifactStagingServer.getApiServiceDescriptor().getUrl());
artifactStagingServer = null;
} catch (Exception e) {
LOG.error("Error while closing the artifactStagingServer.", e);
}
}
}
private GrpcFnServer<InMemoryJobService> createJobServer() throws IOException {
InMemoryJobService service = createJobService();
GrpcFnServer<InMemoryJobService> jobServiceGrpcFnServer;
if (Strings.isNullOrEmpty(configuration.host)) {
jobServiceGrpcFnServer = GrpcFnServer.allocatePortAndCreateFor(service, jobServerFactory);
} else {
Endpoints.ApiServiceDescriptor descriptor =
Endpoints.ApiServiceDescriptor.newBuilder()
.setUrl(configuration.host + ":" + configuration.port)
.build();
jobServiceGrpcFnServer = GrpcFnServer.create(service, descriptor, jobServerFactory);
}
LOG.info("JobServer started on {}", jobServiceGrpcFnServer.getApiServiceDescriptor().getUrl());
return jobServiceGrpcFnServer;
}
private InMemoryJobService createJobService() throws IOException {
artifactStagingServer = createArtifactStagingService();
JobInvoker invoker = createJobInvoker();
return InMemoryJobService.create(
artifactStagingServer.getApiServiceDescriptor(),
(String session) -> {
try {
return BeamFileSystemArtifactStagingService.generateStagingSessionToken(
session, configuration.artifactStagingPath);
} catch (Exception exn) {
throw new RuntimeException(exn);
}
},
(String stagingSessionToken) -> {
if (configuration.cleanArtifactsPerJob) {
artifactStagingServer.getService().removeArtifacts(stagingSessionToken);
}
},
invoker);
}
private GrpcFnServer<BeamFileSystemArtifactStagingService> createArtifactStagingService()
throws IOException {
BeamFileSystemArtifactStagingService service = new BeamFileSystemArtifactStagingService();
final GrpcFnServer<BeamFileSystemArtifactStagingService> artifactStagingService;
if (Strings.isNullOrEmpty(configuration.host)) {
artifactStagingService =
GrpcFnServer.allocatePortAndCreateFor(service, artifactServerFactory);
} else {
Endpoints.ApiServiceDescriptor descriptor =
Endpoints.ApiServiceDescriptor.newBuilder()
.setUrl(configuration.host + ":" + configuration.artifactPort)
.build();
artifactStagingService = GrpcFnServer.create(service, descriptor, artifactServerFactory);
}
LOG.info(
"ArtifactStagingService started on {}",
artifactStagingService.getApiServiceDescriptor().getUrl());
return artifactStagingService;
}
private JobInvoker createJobInvoker() {
return FlinkJobInvoker.create(executor, configuration);
}
}