-
Notifications
You must be signed in to change notification settings - Fork 2.4k
/
Pipelines.java
399 lines (375 loc) · 18.2 KB
/
Pipelines.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
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
/*
* 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.sink.utils;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.configuration.OptionsResolver;
import org.apache.hudi.sink.CleanFunction;
import org.apache.hudi.sink.StreamWriteOperator;
import org.apache.hudi.sink.append.AppendWriteOperator;
import org.apache.hudi.sink.bootstrap.BootstrapOperator;
import org.apache.hudi.sink.bootstrap.batch.BatchBootstrapOperator;
import org.apache.hudi.sink.bucket.BucketBulkInsertWriterHelper;
import org.apache.hudi.sink.bucket.BucketStreamWriteOperator;
import org.apache.hudi.sink.bulk.BulkInsertWriteOperator;
import org.apache.hudi.sink.bulk.RowDataKeyGen;
import org.apache.hudi.sink.bulk.sort.SortOperatorGen;
import org.apache.hudi.sink.common.WriteOperatorFactory;
import org.apache.hudi.sink.compact.CompactFunction;
import org.apache.hudi.sink.compact.CompactionCommitEvent;
import org.apache.hudi.sink.compact.CompactionCommitSink;
import org.apache.hudi.sink.compact.CompactionPlanEvent;
import org.apache.hudi.sink.compact.CompactionPlanOperator;
import org.apache.hudi.sink.partitioner.BucketAssignFunction;
import org.apache.hudi.sink.partitioner.BucketIndexPartitioner;
import org.apache.hudi.sink.transform.RowDataToHoodieFunctions;
import org.apache.hudi.table.format.FilePathUtils;
import org.apache.hudi.util.HoodiePipeline;
import org.apache.flink.api.common.functions.Partitioner;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.DataStreamSink;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.sink.SinkFunction;
import org.apache.flink.streaming.api.graph.StreamGraphGenerator;
import org.apache.flink.streaming.api.operators.KeyedProcessOperator;
import org.apache.flink.streaming.api.operators.ProcessOperator;
import org.apache.flink.table.catalog.ResolvedSchema;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.types.Row;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/**
* Utilities to generate all kinds of sub-pipelines.
*/
public class Pipelines {
/**
* Bulk insert the input dataset at once.
*
* <p>By default, the input dataset would shuffle by the partition path first then
* sort by the partition path before passing around to the write function.
* The whole pipeline looks like the following:
*
* <pre>
* | input1 | ===\ /=== |sorter| === | task1 | (p1, p2)
* shuffle
* | input2 | ===/ \=== |sorter| === | task2 | (p3, p4)
*
* Note: Both input1 and input2's dataset come from partitions: p1, p2, p3, p4
* </pre>
*
* <p>The write task switches to new file handle each time it receives a record
* from the different partition path, the shuffle and sort would reduce small files.
*
* <p>The bulk insert should be run in batch execution mode.
*
* @param conf The configuration
* @param rowType The input row type
* @param dataStream The input data stream
* @return the bulk insert data stream sink
*/
public static DataStreamSink<Object> bulkInsert(Configuration conf, RowType rowType, DataStream<RowData> dataStream) {
WriteOperatorFactory<RowData> operatorFactory = BulkInsertWriteOperator.getFactory(conf, rowType);
if (OptionsResolver.isBucketIndexType(conf)) {
String indexKeys = conf.getString(FlinkOptions.INDEX_KEY_FIELD);
int numBuckets = conf.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS);
BucketIndexPartitioner<String> partitioner = new BucketIndexPartitioner<>(numBuckets, indexKeys);
RowDataKeyGen keyGen = RowDataKeyGen.instance(conf, rowType);
RowType rowTypeWithFileId = BucketBulkInsertWriterHelper.rowTypeWithFileId(rowType);
InternalTypeInfo<RowData> typeInfo = InternalTypeInfo.of(rowTypeWithFileId);
Map<String, String> bucketIdToFileId = new HashMap<>();
dataStream = dataStream.partitionCustom(partitioner, keyGen::getRecordKey)
.map(record -> BucketBulkInsertWriterHelper.rowWithFileId(bucketIdToFileId, keyGen, record, indexKeys, numBuckets), typeInfo)
.setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS)); // same parallelism as write task to avoid shuffle
if (conf.getBoolean(FlinkOptions.WRITE_BULK_INSERT_SORT_INPUT)) {
SortOperatorGen sortOperatorGen = BucketBulkInsertWriterHelper.getFileIdSorterGen(rowTypeWithFileId);
dataStream = dataStream.transform("file_sorter", typeInfo, sortOperatorGen.createSortOperator())
.setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS)); // same parallelism as write task to avoid shuffle
ExecNodeUtil.setManagedMemoryWeight(dataStream.getTransformation(),
conf.getInteger(FlinkOptions.WRITE_SORT_MEMORY) * 1024L * 1024L);
}
return dataStream
.transform("bucket_bulk_insert", TypeInformation.of(Object.class), operatorFactory)
.uid("uid_bucket_bulk_insert" + conf.getString(FlinkOptions.TABLE_NAME))
.setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS))
.addSink(DummySink.INSTANCE)
.name("dummy");
}
final String[] partitionFields = FilePathUtils.extractPartitionKeys(conf);
if (partitionFields.length > 0) {
RowDataKeyGen rowDataKeyGen = RowDataKeyGen.instance(conf, rowType);
if (conf.getBoolean(FlinkOptions.WRITE_BULK_INSERT_SHUFFLE_INPUT)) {
// shuffle by partition keys
// use #partitionCustom instead of #keyBy to avoid duplicate sort operations,
// see BatchExecutionUtils#applyBatchExecutionSettings for details.
Partitioner<String> partitioner = (key, channels) ->
KeyGroupRangeAssignment.assignKeyToParallelOperator(key, StreamGraphGenerator.DEFAULT_LOWER_BOUND_MAX_PARALLELISM, channels);
dataStream = dataStream.partitionCustom(partitioner, rowDataKeyGen::getPartitionPath);
}
if (conf.getBoolean(FlinkOptions.WRITE_BULK_INSERT_SORT_INPUT)) {
SortOperatorGen sortOperatorGen = new SortOperatorGen(rowType, partitionFields);
// sort by partition keys
dataStream = dataStream
.transform("partition_key_sorter",
TypeInformation.of(RowData.class),
sortOperatorGen.createSortOperator())
.setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS));
ExecNodeUtil.setManagedMemoryWeight(dataStream.getTransformation(),
conf.getInteger(FlinkOptions.WRITE_SORT_MEMORY) * 1024L * 1024L);
}
}
return dataStream
.transform("hoodie_bulk_insert_write",
TypeInformation.of(Object.class),
operatorFactory)
// follow the parallelism of upstream operators to avoid shuffle
.setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS))
.addSink(DummySink.INSTANCE)
.name("dummy");
}
/**
* Insert the dataset with append mode(no upsert or deduplication).
*
* <p>The input dataset would be rebalanced among the write tasks:
*
* <pre>
* | input1 | ===\ /=== | task1 | (p1, p2, p3, p4)
* shuffle
* | input2 | ===/ \=== | task2 | (p1, p2, p3, p4)
*
* Note: Both input1 and input2's dataset come from partitions: p1, p2, p3, p4
* </pre>
*
* <p>The write task switches to new file handle each time it receives a record
* from the different partition path, so there may be many small files.
*
* @param conf The configuration
* @param rowType The input row type
* @param dataStream The input data stream
* @param bounded Whether the input stream is bounded
* @return the appending data stream sink
*/
public static DataStreamSink<Object> append(
Configuration conf,
RowType rowType,
DataStream<RowData> dataStream,
boolean bounded) {
if (!bounded) {
// In principle, the config should be immutable, but the boundedness
// is only visible when creating the sink pipeline.
conf.setBoolean(FlinkOptions.WRITE_BULK_INSERT_SORT_INPUT, false);
}
WriteOperatorFactory<RowData> operatorFactory = AppendWriteOperator.getFactory(conf, rowType);
return dataStream
.transform("hoodie_append_write", TypeInformation.of(Object.class), operatorFactory)
.uid("uid_hoodie_stream_write" + conf.getString(FlinkOptions.TABLE_NAME))
.setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS))
.addSink(DummySink.INSTANCE)
.name("dummy");
}
/**
* Constructs bootstrap pipeline as streaming.
* The bootstrap operator loads the existing data index (primary key to file id mapping),
* then sends the indexing data set to subsequent operator(usually the bucket assign operator).
*/
public static DataStream<HoodieRecord> bootstrap(
Configuration conf,
RowType rowType,
int defaultParallelism,
DataStream<RowData> dataStream) {
return bootstrap(conf, rowType, defaultParallelism, dataStream, false, false);
}
/**
* Constructs bootstrap pipeline.
* The bootstrap operator loads the existing data index (primary key to file id mapping),
* then send the indexing data set to subsequent operator(usually the bucket assign operator).
*
* @param conf The configuration
* @param rowType The row type
* @param defaultParallelism The default parallelism
* @param dataStream The data stream
* @param bounded Whether the source is bounded
* @param overwrite Whether it is insert overwrite
*/
public static DataStream<HoodieRecord> bootstrap(
Configuration conf,
RowType rowType,
int defaultParallelism,
DataStream<RowData> dataStream,
boolean bounded,
boolean overwrite) {
final boolean globalIndex = conf.getBoolean(FlinkOptions.INDEX_GLOBAL_ENABLED);
if (overwrite || OptionsResolver.isBucketIndexType(conf)) {
return rowDataToHoodieRecord(conf, rowType, dataStream);
} else if (bounded && !globalIndex && OptionsResolver.isPartitionedTable(conf)) {
return boundedBootstrap(conf, rowType, defaultParallelism, dataStream);
} else {
return streamBootstrap(conf, rowType, defaultParallelism, dataStream, bounded);
}
}
private static DataStream<HoodieRecord> streamBootstrap(
Configuration conf,
RowType rowType,
int defaultParallelism,
DataStream<RowData> dataStream,
boolean bounded) {
DataStream<HoodieRecord> dataStream1 = rowDataToHoodieRecord(conf, rowType, dataStream);
if (conf.getBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED) || bounded) {
dataStream1 = dataStream1
.transform(
"index_bootstrap",
TypeInformation.of(HoodieRecord.class),
new BootstrapOperator<>(conf))
.setParallelism(conf.getOptional(FlinkOptions.INDEX_BOOTSTRAP_TASKS).orElse(defaultParallelism))
.uid("uid_index_bootstrap_" + conf.getString(FlinkOptions.TABLE_NAME));
}
return dataStream1;
}
/**
* Constructs bootstrap pipeline for batch execution mode.
* The indexing data set is loaded before the actual data write
* in order to support batch UPSERT.
*/
private static DataStream<HoodieRecord> boundedBootstrap(
Configuration conf,
RowType rowType,
int defaultParallelism,
DataStream<RowData> dataStream) {
final RowDataKeyGen rowDataKeyGen = RowDataKeyGen.instance(conf, rowType);
// shuffle by partition keys
dataStream = dataStream
.keyBy(rowDataKeyGen::getPartitionPath);
return rowDataToHoodieRecord(conf, rowType, dataStream)
.transform(
"batch_index_bootstrap",
TypeInformation.of(HoodieRecord.class),
new BatchBootstrapOperator<>(conf))
.setParallelism(conf.getOptional(FlinkOptions.INDEX_BOOTSTRAP_TASKS).orElse(defaultParallelism))
.uid("uid_batch_index_bootstrap_" + conf.getString(FlinkOptions.TABLE_NAME));
}
/**
* Transforms the row data to hoodie records.
*/
public static DataStream<HoodieRecord> rowDataToHoodieRecord(Configuration conf, RowType rowType, DataStream<RowData> dataStream) {
return dataStream.map(RowDataToHoodieFunctions.create(rowType, conf), TypeInformation.of(HoodieRecord.class))
.setParallelism(dataStream.getParallelism()).name("row_data_to_hoodie_record");
}
/**
* The streaming write pipeline.
*
* <p>The input dataset shuffles by the primary key first then
* shuffles by the file group ID before passing around to the write function.
* The whole pipeline looks like the following:
*
* <pre>
* | input1 | ===\ /=== | bucket assigner | ===\ /=== | task1 |
* shuffle(by PK) shuffle(by bucket ID)
* | input2 | ===/ \=== | bucket assigner | ===/ \=== | task2 |
*
* Note: a file group must be handled by one write task to avoid write conflict.
* </pre>
*
* <p>The bucket assigner assigns the inputs to suitable file groups, the write task caches
* and flushes the data set to disk.
*
* @param conf The configuration
* @param defaultParallelism The default parallelism
* @param dataStream The input data stream
* @return the stream write data stream pipeline
*/
public static DataStream<Object> hoodieStreamWrite(Configuration conf, int defaultParallelism, DataStream<HoodieRecord> dataStream) {
if (OptionsResolver.isBucketIndexType(conf)) {
WriteOperatorFactory<HoodieRecord> operatorFactory = BucketStreamWriteOperator.getFactory(conf);
int bucketNum = conf.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS);
String indexKeyFields = conf.getString(FlinkOptions.INDEX_KEY_FIELD);
BucketIndexPartitioner<String> partitioner = new BucketIndexPartitioner<>(bucketNum, indexKeyFields);
return dataStream.partitionCustom(partitioner, HoodieRecord::getRecordKey)
.transform("bucket_write", TypeInformation.of(Object.class), operatorFactory)
.uid("uid_bucket_write" + conf.getString(FlinkOptions.TABLE_NAME))
.setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS));
} else {
WriteOperatorFactory<HoodieRecord> operatorFactory = StreamWriteOperator.getFactory(conf);
return dataStream
// Key-by record key, to avoid multiple subtasks write to a bucket at the same time
.keyBy(HoodieRecord::getRecordKey)
.transform(
"bucket_assigner",
TypeInformation.of(HoodieRecord.class),
new KeyedProcessOperator<>(new BucketAssignFunction<>(conf)))
.uid("uid_bucket_assigner_" + conf.getString(FlinkOptions.TABLE_NAME))
.setParallelism(conf.getOptional(FlinkOptions.BUCKET_ASSIGN_TASKS).orElse(defaultParallelism))
// shuffle by fileId(bucket id)
.keyBy(record -> record.getCurrentLocation().getFileId())
.transform("stream_write", TypeInformation.of(Object.class), operatorFactory)
.uid("uid_stream_write" + conf.getString(FlinkOptions.TABLE_NAME))
.setParallelism(conf.getInteger(FlinkOptions.WRITE_TASKS));
}
}
/**
* The compaction tasks pipeline.
*
* <p>The compaction plan operator monitors the new compaction plan on the timeline
* then distributes the sub-plans to the compaction tasks. The compaction task then
* handle over the metadata to commit task for compaction transaction commit.
* The whole pipeline looks like the following:
*
* <pre>
* /=== | task1 | ===\
* | plan generation | ===> re-balance | commit |
* \=== | task2 | ===/
*
* Note: both the compaction plan generation task and commission task are singleton.
* </pre>
*
* @param conf The configuration
* @param dataStream The input data stream
* @return the compaction pipeline
*/
public static DataStreamSink<CompactionCommitEvent> compact(Configuration conf, DataStream<Object> dataStream) {
return dataStream.transform("compact_plan_generate",
TypeInformation.of(CompactionPlanEvent.class),
new CompactionPlanOperator(conf))
.setParallelism(1) // plan generate must be singleton
.rebalance()
.transform("compact_task",
TypeInformation.of(CompactionCommitEvent.class),
new ProcessOperator<>(new CompactFunction(conf)))
.setParallelism(conf.getInteger(FlinkOptions.COMPACTION_TASKS))
.addSink(new CompactionCommitSink(conf))
.name("compact_commit")
.setParallelism(1); // compaction commit should be singleton
}
public static DataStreamSink<Object> clean(Configuration conf, DataStream<Object> dataStream) {
return dataStream.addSink(new CleanFunction<>(conf))
.setParallelism(1)
.name("clean_commits");
}
/**
* Dummy sink that does nothing.
*/
public static class DummySink implements SinkFunction<Object> {
private static final long serialVersionUID = 1L;
public static DummySink INSTANCE = new DummySink();
}
}