-
Notifications
You must be signed in to change notification settings - Fork 13k
/
InputOutputFormatVertex.java
193 lines (166 loc) · 8.27 KB
/
InputOutputFormatVertex.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
/*
* 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.jobgraph;
import org.apache.flink.api.common.io.FinalizeOnMaster;
import org.apache.flink.api.common.io.FinalizeOnMaster.FinalizationContext;
import org.apache.flink.api.common.io.InitializeOnMaster;
import org.apache.flink.api.common.io.InputFormat;
import org.apache.flink.api.common.io.OutputFormat;
import org.apache.flink.api.common.operators.util.UserCodeWrapper;
import org.apache.flink.runtime.OperatorIDPair;
import org.apache.flink.runtime.operators.util.TaskConfig;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.apache.flink.util.Preconditions.checkNotNull;
/**
* A task vertex that runs an initialization and a finalization on the master. If necessary, it
* tries to deserialize input and output formats, and initialize and finalize them on master.
*/
public class InputOutputFormatVertex extends JobVertex {
private static final long serialVersionUID = 1L;
private final Map<OperatorID, String> formatDescriptions = new HashMap<>();
public InputOutputFormatVertex(String name) {
super(name);
}
public InputOutputFormatVertex(
String name, JobVertexID id, List<OperatorIDPair> operatorIDPairs) {
super(name, id, operatorIDPairs);
}
@Override
public void initializeOnMaster(InitializeOnMasterContext context) throws Exception {
ClassLoader loader = context.getClassLoader();
final InputOutputFormatContainer formatContainer = initInputOutputformatContainer(loader);
final ClassLoader original = Thread.currentThread().getContextClassLoader();
try {
// set user classloader before calling user code
Thread.currentThread().setContextClassLoader(loader);
// configure the input format and setup input splits
Map<OperatorID, UserCodeWrapper<? extends InputFormat<?, ?>>> inputFormats =
formatContainer.getInputFormats();
if (inputFormats.size() > 1) {
throw new UnsupportedOperationException(
"Multiple input formats are not supported in a job vertex.");
}
for (Map.Entry<OperatorID, UserCodeWrapper<? extends InputFormat<?, ?>>> entry :
inputFormats.entrySet()) {
final InputFormat<?, ?> inputFormat;
try {
inputFormat = entry.getValue().getUserCodeObject();
inputFormat.configure(formatContainer.getParameters(entry.getKey()));
} catch (Throwable t) {
throw new Exception(
"Configuring the input format ("
+ getFormatDescription(entry.getKey())
+ ") failed: "
+ t.getMessage(),
t);
}
setInputSplitSource(inputFormat);
}
// configure output formats and invoke initializeGlobal()
Map<OperatorID, UserCodeWrapper<? extends OutputFormat<?>>> outputFormats =
formatContainer.getOutputFormats();
for (Map.Entry<OperatorID, UserCodeWrapper<? extends OutputFormat<?>>> entry :
outputFormats.entrySet()) {
final OutputFormat<?> outputFormat;
try {
outputFormat = entry.getValue().getUserCodeObject();
outputFormat.configure(formatContainer.getParameters(entry.getKey()));
} catch (Throwable t) {
throw new Exception(
"Configuring the output format ("
+ getFormatDescription(entry.getKey())
+ ") failed: "
+ t.getMessage(),
t);
}
if (outputFormat instanceof InitializeOnMaster) {
int executionParallelism = context.getExecutionParallelism();
((InitializeOnMaster) outputFormat).initializeGlobal(executionParallelism);
}
}
} finally {
// restore original classloader
Thread.currentThread().setContextClassLoader(original);
}
}
@Override
public void finalizeOnMaster(FinalizeOnMasterContext context) throws Exception {
final ClassLoader loader = context.getClassLoader();
final InputOutputFormatContainer formatContainer = initInputOutputformatContainer(loader);
final ClassLoader original = Thread.currentThread().getContextClassLoader();
try {
// set user classloader before calling user code
Thread.currentThread().setContextClassLoader(loader);
// configure output formats and invoke finalizeGlobal()
Map<OperatorID, UserCodeWrapper<? extends OutputFormat<?>>> outputFormats =
formatContainer.getOutputFormats();
for (Map.Entry<OperatorID, UserCodeWrapper<? extends OutputFormat<?>>> entry :
outputFormats.entrySet()) {
final OutputFormat<?> outputFormat;
try {
outputFormat = entry.getValue().getUserCodeObject();
outputFormat.configure(formatContainer.getParameters(entry.getKey()));
} catch (Throwable t) {
throw new Exception(
"Configuring the output format ("
+ getFormatDescription(entry.getKey())
+ ") failed: "
+ t.getMessage(),
t);
}
if (outputFormat instanceof FinalizeOnMaster) {
int executionParallelism = context.getExecutionParallelism();
((FinalizeOnMaster) outputFormat)
.finalizeGlobal(
new FinalizationContext() {
@Override
public int getParallelism() {
return executionParallelism;
}
@Override
public int getFinishedAttempt(int subtaskIndex) {
return context.getFinishedAttempt(subtaskIndex);
}
});
}
}
} finally {
// restore original classloader
Thread.currentThread().setContextClassLoader(original);
}
}
public String getFormatDescription(OperatorID operatorID) {
return formatDescriptions.get(operatorID);
}
public void setFormatDescription(OperatorID operatorID, String formatDescription) {
formatDescriptions.put(checkNotNull(operatorID), formatDescription);
}
private InputOutputFormatContainer initInputOutputformatContainer(ClassLoader classLoader)
throws Exception {
try {
return new InputOutputFormatContainer(new TaskConfig(getConfiguration()), classLoader);
} catch (Throwable t) {
throw new Exception(
"Loading the input/output formats failed: "
+ String.join(",", formatDescriptions.values()),
t);
}
}
}