/
DataSinkTask.java
384 lines (332 loc) · 12.4 KB
/
DataSinkTask.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
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
/*
* 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.runtime.operators;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.io.CleanupWhenUnsuccessful;
import org.apache.flink.api.common.io.OutputFormat;
import org.apache.flink.api.common.typeutils.TypeComparatorFactory;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.io.IOReadableWritable;
import org.apache.flink.runtime.execution.CancelTaskException;
import org.apache.flink.runtime.io.network.api.reader.MutableReader;
import org.apache.flink.runtime.io.network.api.reader.MutableRecordReader;
import org.apache.flink.runtime.io.network.partition.consumer.UnionInputGate;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
import org.apache.flink.runtime.operators.chaining.ExceptionInChainedStubException;
import org.apache.flink.runtime.operators.sort.UnilateralSortMerger;
import org.apache.flink.runtime.operators.util.CloseableInputProvider;
import org.apache.flink.runtime.operators.util.ReaderIterator;
import org.apache.flink.runtime.operators.util.TaskConfig;
import org.apache.flink.runtime.plugable.DeserializationDelegate;
import org.apache.flink.util.InstantiationUtil;
import org.apache.flink.util.MutableObjectIterator;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
/**
* DataSinkTask which is executed by a task manager. The task hands the data to an output format.
*
* @see OutputFormat
*/
public class DataSinkTask<IT> extends AbstractInvokable {
// Obtain DataSinkTask Logger
private static final Logger LOG = LoggerFactory.getLogger(DataSinkTask.class);
// --------------------------------------------------------------------------------------------
// OutputFormat instance. volatile, because the asynchronous canceller may access it
private volatile OutputFormat<IT> format;
private MutableReader<?> inputReader;
// input reader
private MutableObjectIterator<IT> reader;
// The serializer for the input type
private TypeSerializerFactory<IT> inputTypeSerializerFactory;
// local strategy
private CloseableInputProvider<IT> localStrategy;
// task configuration
private TaskConfig config;
// cancel flag
private volatile boolean taskCanceled;
private volatile boolean cleanupCalled;
@Override
public void registerInputOutput() {
if (LOG.isDebugEnabled()) {
LOG.debug(getLogString("Start registering input and output"));
}
// initialize OutputFormat
initOutputFormat();
// initialize input readers
try {
initInputReaders();
} catch (Exception e) {
throw new RuntimeException("Initializing the input streams failed" +
(e.getMessage() == null ? "." : ": " + e.getMessage()), e);
}
if (LOG.isDebugEnabled()) {
LOG.debug(getLogString("Finished registering input and output"));
}
}
@Override
public void invoke() throws Exception
{
if (LOG.isDebugEnabled()) {
LOG.debug(getLogString("Starting data sink operator"));
}
ExecutionConfig executionConfig;
try {
ExecutionConfig c = (ExecutionConfig) InstantiationUtil.readObjectFromConfig(
getJobConfiguration(),
ExecutionConfig.CONFIG_KEY,
getUserCodeClassLoader());
if (c != null) {
executionConfig = c;
} else {
LOG.warn("The execution config returned by the configuration was null");
executionConfig = new ExecutionConfig();
}
} catch (IOException e) {
throw new RuntimeException("Could not load ExecutionConfig from Job Configuration: " + e);
} catch (ClassNotFoundException e) {
throw new RuntimeException("Could not load ExecutionConfig from Job Configuration: " + e);
}
boolean objectReuseEnabled = executionConfig.isObjectReuseEnabled();
try {
// initialize local strategies
MutableObjectIterator<IT> input1;
switch (this.config.getInputLocalStrategy(0)) {
case NONE:
// nothing to do
localStrategy = null;
input1 = reader;
break;
case SORT:
// initialize sort local strategy
try {
// get type comparator
TypeComparatorFactory<IT> compFact = this.config.getInputComparator(0,
getUserCodeClassLoader());
if (compFact == null) {
throw new Exception("Missing comparator factory for local strategy on input " + 0);
}
// initialize sorter
UnilateralSortMerger<IT> sorter = new UnilateralSortMerger<IT>(
getEnvironment().getMemoryManager(),
getEnvironment().getIOManager(),
this.reader, this, this.inputTypeSerializerFactory, compFact.createComparator(),
this.config.getRelativeMemoryInput(0), this.config.getFilehandlesInput(0),
this.config.getSpillingThresholdInput(0));
this.localStrategy = sorter;
input1 = sorter.getIterator();
} catch (Exception e) {
throw new RuntimeException("Initializing the input processing failed" +
(e.getMessage() == null ? "." : ": " + e.getMessage()), e);
}
break;
default:
throw new RuntimeException("Invalid local strategy for DataSinkTask");
}
// read the reader and write it to the output
final TypeSerializer<IT> serializer = this.inputTypeSerializerFactory.getSerializer();
final MutableObjectIterator<IT> input = input1;
final OutputFormat<IT> format = this.format;
// check if task has been canceled
if (this.taskCanceled) {
return;
}
if (LOG.isDebugEnabled()) {
LOG.debug(getLogString("Starting to produce output"));
}
// open
format.open(this.getEnvironment().getIndexInSubtaskGroup(), this.getEnvironment().getNumberOfSubtasks());
if (objectReuseEnabled) {
IT record = serializer.createInstance();
// work!
while (!this.taskCanceled && ((record = input.next(record)) != null)) {
format.writeRecord(record);
}
} else {
IT record;
// work!
while (!this.taskCanceled && ((record = input.next()) != null)) {
format.writeRecord(record);
}
}
// close. We close here such that a regular close throwing an exception marks a task as failed.
if (!this.taskCanceled) {
this.format.close();
this.format = null;
}
}
catch (Exception ex) {
// make a best effort to clean up
try {
if (!cleanupCalled && format instanceof CleanupWhenUnsuccessful) {
cleanupCalled = true;
((CleanupWhenUnsuccessful) format).tryCleanupOnError();
}
}
catch (Throwable t) {
LOG.error("Cleanup on error failed.", t);
}
ex = ExceptionInChainedStubException.exceptionUnwrap(ex);
if (ex instanceof CancelTaskException) {
// forward canceling exception
throw ex;
}
// drop, if the task was canceled
else if (!this.taskCanceled) {
if (LOG.isErrorEnabled()) {
LOG.error(getLogString("Error in user code: " + ex.getMessage()), ex);
}
throw ex;
}
}
finally {
if (this.format != null) {
// close format, if it has not been closed, yet.
// This should only be the case if we had a previous error, or were canceled.
try {
this.format.close();
}
catch (Throwable t) {
if (LOG.isWarnEnabled()) {
LOG.warn(getLogString("Error closing the ouput format."), t);
}
}
}
// close local strategy if necessary
if (localStrategy != null) {
try {
this.localStrategy.close();
} catch (Throwable t) {
LOG.error("Error closing local strategy", t);
}
}
RegularPactTask.clearReaders(new MutableReader[]{inputReader});
}
if (!this.taskCanceled) {
if (LOG.isDebugEnabled()) {
LOG.debug(getLogString("Finished data sink operator"));
}
}
else {
if (LOG.isDebugEnabled()) {
LOG.debug(getLogString("Data sink operator cancelled"));
}
}
}
@Override
public void cancel() throws Exception {
this.taskCanceled = true;
OutputFormat<IT> format = this.format;
if (format != null) {
try {
this.format.close();
} catch (Throwable t) {}
// make a best effort to clean up
try {
if (!cleanupCalled && format instanceof CleanupWhenUnsuccessful) {
cleanupCalled = true;
((CleanupWhenUnsuccessful) format).tryCleanupOnError();
}
}
catch (Throwable t) {
LOG.error("Cleanup on error failed.", t);
}
}
if (LOG.isDebugEnabled()) {
LOG.debug(getLogString("Cancelling data sink operator"));
}
}
/**
* Initializes the OutputFormat implementation and configuration.
*
* @throws RuntimeException
* Throws if instance of OutputFormat implementation can not be
* obtained.
*/
private void initOutputFormat() {
ClassLoader userCodeClassLoader = getUserCodeClassLoader();
// obtain task configuration (including stub parameters)
Configuration taskConf = getTaskConfiguration();
this.config = new TaskConfig(taskConf);
try {
this.format = config.<OutputFormat<IT>>getStubWrapper(userCodeClassLoader).getUserCodeObject(OutputFormat.class, userCodeClassLoader);
// check if the class is a subclass, if the check is required
if (!OutputFormat.class.isAssignableFrom(this.format.getClass())) {
throw new RuntimeException("The class '" + this.format.getClass().getName() + "' is not a subclass of '" +
OutputFormat.class.getName() + "' as is required.");
}
}
catch (ClassCastException ccex) {
throw new RuntimeException("The stub class is not a proper subclass of " + OutputFormat.class.getName(), ccex);
}
// configure the stub. catch exceptions here extra, to report them as originating from the user code
try {
this.format.configure(this.config.getStubParameters());
}
catch (Throwable t) {
throw new RuntimeException("The user defined 'configure()' method in the Output Format caused an error: "
+ t.getMessage(), t);
}
}
/**
* Initializes the input readers of the DataSinkTask.
*
* @throws RuntimeException
* Thrown in case of invalid task input configuration.
*/
@SuppressWarnings("unchecked")
private void initInputReaders() throws Exception {
int numGates = 0;
// ---------------- create the input readers ---------------------
// in case where a logical input unions multiple physical inputs, create a union reader
final int groupSize = this.config.getGroupSize(0);
numGates += groupSize;
if (groupSize == 1) {
// non-union case
inputReader = new MutableRecordReader<DeserializationDelegate<IT>>(getEnvironment().getInputGate(0));
} else if (groupSize > 1){
// union case
inputReader = new MutableRecordReader<IOReadableWritable>(new UnionInputGate(getEnvironment().getAllInputGates()));
} else {
throw new Exception("Illegal input group size in task configuration: " + groupSize);
}
this.inputTypeSerializerFactory = this.config.getInputSerializer(0, getUserCodeClassLoader());
@SuppressWarnings({ "rawtypes" })
final MutableObjectIterator<?> iter = new ReaderIterator(inputReader, this.inputTypeSerializerFactory.getSerializer());
this.reader = (MutableObjectIterator<IT>)iter;
// final sanity check
if (numGates != this.config.getNumInputs()) {
throw new Exception("Illegal configuration: Number of input gates and group sizes are not consistent.");
}
}
// ------------------------------------------------------------------------
// Utilities
// ------------------------------------------------------------------------
/**
* Utility function that composes a string for logging purposes. The string includes the given message and
* the index of the task in its task group together with the number of tasks in the task group.
*
* @param message The main message for the log.
* @return The string ready for logging.
*/
private String getLogString(String message) {
return RegularPactTask.constructLogString(message, this.getEnvironment().getTaskName(), this);
}
}