-
Notifications
You must be signed in to change notification settings - Fork 2.4k
/
StreamReadMonitoringFunction.java
254 lines (216 loc) · 8.64 KB
/
StreamReadMonitoringFunction.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
/*
* 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.hudi.source;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.table.format.mor.MergeOnReadInputSplit;
import org.apache.hudi.util.StreamerUtil;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.state.ListState;
import org.apache.flink.api.common.state.ListStateDescriptor;
import org.apache.flink.api.common.typeutils.base.StringSerializer;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.fs.Path;
import org.apache.flink.runtime.state.FunctionInitializationContext;
import org.apache.flink.runtime.state.FunctionSnapshotContext;
import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.List;
import java.util.Set;
import java.util.concurrent.TimeUnit;
/**
* This is the single (non-parallel) monitoring task which takes a {@link MergeOnReadInputSplit}
* , it is responsible for:
*
* <ol>
* <li>Monitoring a user-provided hoodie table path.</li>
* <li>Deciding which files(or split) should be further read and processed.</li>
* <li>Creating the {@link MergeOnReadInputSplit splits} corresponding to those files.</li>
* <li>Assigning them to downstream tasks for further processing.</li>
* </ol>
*
* <p>The splits to be read are forwarded to the downstream {@link StreamReadOperator}
* which can have parallelism greater than one.
*
* <p><b>IMPORTANT NOTE: </b> Splits are forwarded downstream for reading in ascending instant commits time order,
* in each downstream task, the splits are also read in receiving sequence. We do not ensure split consuming sequence
* among the downstream tasks.
*/
public class StreamReadMonitoringFunction
extends RichSourceFunction<MergeOnReadInputSplit> implements CheckpointedFunction {
private static final Logger LOG = LoggerFactory.getLogger(StreamReadMonitoringFunction.class);
private static final long serialVersionUID = 1L;
/**
* The path to monitor.
*/
private final Path path;
/**
* The interval between consecutive path scans.
*/
private final long interval;
private transient Object checkpointLock;
private volatile boolean isRunning = true;
private String issuedInstant;
private transient ListState<String> instantState;
private final Configuration conf;
private transient org.apache.hadoop.conf.Configuration hadoopConf;
private HoodieTableMetaClient metaClient;
private final IncrementalInputSplits incrementalInputSplits;
public StreamReadMonitoringFunction(
Configuration conf,
Path path,
long maxCompactionMemoryInBytes,
@Nullable Set<String> requiredPartitionPaths) {
this.conf = conf;
this.path = path;
this.interval = conf.getInteger(FlinkOptions.READ_STREAMING_CHECK_INTERVAL);
this.incrementalInputSplits = IncrementalInputSplits.builder()
.conf(conf)
.path(path)
.maxCompactionMemoryInBytes(maxCompactionMemoryInBytes)
.requiredPartitions(requiredPartitionPaths)
.skipCompaction(conf.getBoolean(FlinkOptions.READ_STREAMING_SKIP_COMPACT))
.build();
}
@Override
public void initializeState(FunctionInitializationContext context) throws Exception {
ValidationUtils.checkState(this.instantState == null,
"The " + getClass().getSimpleName() + " has already been initialized.");
this.instantState = context.getOperatorStateStore().getListState(
new ListStateDescriptor<>(
"file-monitoring-state",
StringSerializer.INSTANCE
)
);
if (context.isRestored()) {
LOG.info("Restoring state for the class {} with table {} and base path {}.",
getClass().getSimpleName(), conf.getString(FlinkOptions.TABLE_NAME), path);
List<String> retrievedStates = new ArrayList<>();
for (String entry : this.instantState.get()) {
retrievedStates.add(entry);
}
ValidationUtils.checkArgument(retrievedStates.size() <= 1,
getClass().getSimpleName() + " retrieved invalid state.");
if (retrievedStates.size() == 1 && issuedInstant != null) {
// this is the case where we have both legacy and new state.
// the two should be mutually exclusive for the operator, thus we throw the exception.
throw new IllegalArgumentException(
"The " + getClass().getSimpleName() + " has already restored from a previous Flink version.");
} else if (retrievedStates.size() == 1) {
this.issuedInstant = retrievedStates.get(0);
if (LOG.isDebugEnabled()) {
LOG.debug("{} retrieved a issued instant of time {} for table {} with path {}.",
getClass().getSimpleName(), issuedInstant, conf.get(FlinkOptions.TABLE_NAME), path);
}
}
}
}
@Override
public void open(Configuration parameters) throws Exception {
super.open(parameters);
this.hadoopConf = StreamerUtil.getHadoopConf();
}
@Override
public void run(SourceFunction.SourceContext<MergeOnReadInputSplit> context) throws Exception {
checkpointLock = context.getCheckpointLock();
while (isRunning) {
synchronized (checkpointLock) {
monitorDirAndForwardSplits(context);
}
TimeUnit.SECONDS.sleep(interval);
}
}
@Nullable
private HoodieTableMetaClient getOrCreateMetaClient() {
if (this.metaClient != null) {
return this.metaClient;
}
if (StreamerUtil.tableExists(this.path.toString(), hadoopConf)) {
this.metaClient = StreamerUtil.createMetaClient(this.path.toString(), hadoopConf);
return this.metaClient;
}
// fallback
return null;
}
@VisibleForTesting
public void monitorDirAndForwardSplits(SourceContext<MergeOnReadInputSplit> context) {
HoodieTableMetaClient metaClient = getOrCreateMetaClient();
if (metaClient == null) {
// table does not exist
return;
}
IncrementalInputSplits.Result result =
incrementalInputSplits.inputSplits(metaClient, this.hadoopConf, this.issuedInstant);
if (result.isEmpty()) {
// no new instants, returns early
return;
}
for (MergeOnReadInputSplit split : result.getInputSplits()) {
context.collect(split);
}
// update the issues instant time
this.issuedInstant = result.getEndInstant();
LOG.info("\n"
+ "------------------------------------------------------------\n"
+ "---------- consumed to instant: {}\n"
+ "------------------------------------------------------------",
this.issuedInstant);
}
@Override
public void close() throws Exception {
super.close();
if (checkpointLock != null) {
synchronized (checkpointLock) {
issuedInstant = null;
isRunning = false;
}
}
if (LOG.isDebugEnabled()) {
LOG.debug("Closed File Monitoring Source for path: " + path + ".");
}
}
@Override
public void cancel() {
if (checkpointLock != null) {
// this is to cover the case where cancel() is called before the run()
synchronized (checkpointLock) {
issuedInstant = null;
isRunning = false;
}
} else {
issuedInstant = null;
isRunning = false;
}
}
// -------------------------------------------------------------------------
// Checkpointing
// -------------------------------------------------------------------------
@Override
public void snapshotState(FunctionSnapshotContext context) throws Exception {
this.instantState.clear();
if (this.issuedInstant != null) {
this.instantState.add(this.issuedInstant);
}
}
}