1
0

add BootstrapFunction to support index bootstrap (#3024)

Co-authored-by: 喻兆靖 <yuzhaojing@bilibili.com>
This commit is contained in:
yuzhaojing
2021-06-08 13:55:25 +08:00
committed by GitHub
parent 57611d10b5
commit cf83f10f5b
11 changed files with 507 additions and 113 deletions

View File

@@ -21,6 +21,7 @@ package org.apache.hudi.sink;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.sink.bootstrap.BootstrapFunction;
import org.apache.hudi.sink.compact.CompactFunction;
import org.apache.hudi.sink.compact.CompactionCommitEvent;
import org.apache.hudi.sink.compact.CompactionCommitSink;
@@ -49,6 +50,7 @@ import org.apache.flink.streaming.api.CheckpointingMode;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.source.FileProcessingMode;
import org.apache.flink.streaming.api.operators.ProcessOperator;
import org.apache.flink.streaming.api.operators.KeyedProcessOperator;
import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
import org.apache.flink.table.types.logical.RowType;
@@ -109,14 +111,22 @@ public class StreamWriteITCase extends TestLogger {
String sourcePath = Objects.requireNonNull(Thread.currentThread()
.getContextClassLoader().getResource("test_source.data")).toString();
DataStream<Object> dataStream = execEnv
DataStream<HoodieRecord> hoodieDataStream = execEnv
// use continuous file source to trigger checkpoint
.addSource(new ContinuousFileSource.BoundedSourceFunction(new Path(sourcePath), 2))
.name("continuous_file_source")
.setParallelism(1)
.map(record -> deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8)))
.setParallelism(4)
.map(new RowDataToHoodieFunction<>(rowType, conf), TypeInformation.of(HoodieRecord.class))
.map(new RowDataToHoodieFunction<>(rowType, conf), TypeInformation.of(HoodieRecord.class));
if (conf.getBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED)) {
hoodieDataStream = hoodieDataStream.transform("index_bootstrap",
TypeInformation.of(HoodieRecord.class),
new ProcessOperator<>(new BootstrapFunction<>(conf)));
}
DataStream<Object> pipeline = hoodieDataStream
// Key-by record key, to avoid multiple subtasks write to a bucket at the same time
.keyBy(HoodieRecord::getRecordKey)
.transform(
@@ -128,7 +138,7 @@ public class StreamWriteITCase extends TestLogger {
.keyBy(record -> record.getCurrentLocation().getFileId())
.transform("hoodie_stream_write", TypeInformation.of(Object.class), operatorFactory)
.uid("uid_hoodie_stream_write");
execEnv.addOperator(dataStream.getTransformation());
execEnv.addOperator(pipeline.getTransformation());
JobClient client = execEnv.executeAsync(execEnv.getStreamGraph(conf.getString(FlinkOptions.TABLE_NAME)));
// wait for the streaming job to finish
@@ -171,12 +181,20 @@ public class StreamWriteITCase extends TestLogger {
TypeInformation<String> typeInfo = BasicTypeInfo.STRING_TYPE_INFO;
format.setCharsetName("UTF-8");
DataStream<Object> pipeline = execEnv
DataStream<HoodieRecord> hoodieDataStream = execEnv
// use PROCESS_CONTINUOUSLY mode to trigger checkpoint
.readFile(format, sourcePath, FileProcessingMode.PROCESS_CONTINUOUSLY, 1000, typeInfo)
.map(record -> deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8)))
.setParallelism(4)
.map(new RowDataToHoodieFunction<>(rowType, conf), TypeInformation.of(HoodieRecord.class))
.map(new RowDataToHoodieFunction<>(rowType, conf), TypeInformation.of(HoodieRecord.class));
if (conf.getBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED)) {
hoodieDataStream = hoodieDataStream.transform("index_bootstrap",
TypeInformation.of(HoodieRecord.class),
new ProcessOperator<>(new BootstrapFunction<>(conf)));
}
DataStream<Object> pipeline = hoodieDataStream
// Key-by record key, to avoid multiple subtasks write to a bucket at the same time
.keyBy(HoodieRecord::getRecordKey)
.transform(

View File

@@ -576,10 +576,6 @@ public class TestWriteCopyOnWrite {
@Test
public void testIndexStateBootstrap() throws Exception {
// reset the config option
conf.setBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED, true);
funcWrapper = new StreamWriteFunctionWrapper<>(tempFile.getAbsolutePath(), conf);
// open the function and ingest data
funcWrapper.openFunction();
for (RowData rowData : TestData.DATA_SET_INSERT) {
@@ -598,13 +594,21 @@ public class TestWriteCopyOnWrite {
funcWrapper.checkpointComplete(1);
// Mark the index state as not fully loaded to trigger re-load from the filesystem.
funcWrapper.clearIndexState();
// the data is not flushed yet
checkWrittenData(tempFile, EXPECTED1);
// reset the config option
conf.setBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED, true);
funcWrapper = new StreamWriteFunctionWrapper<>(tempFile.getAbsolutePath(), conf);
// upsert another data buffer
funcWrapper.openFunction();
for (RowData rowData : TestData.DATA_SET_UPDATE_INSERT) {
funcWrapper.invoke(rowData);
}
assertTrue(funcWrapper.isAlreadyBootstrap());
checkIndexLoaded(
new HoodieKey("id1", "par1"),
new HoodieKey("id2", "par1"),
@@ -613,11 +617,13 @@ public class TestWriteCopyOnWrite {
new HoodieKey("id5", "par3"),
new HoodieKey("id6", "par3"),
new HoodieKey("id7", "par4"),
new HoodieKey("id8", "par4"));
// the data is not flushed yet
checkWrittenData(tempFile, EXPECTED1);
new HoodieKey("id8", "par4"),
new HoodieKey("id9", "par3"),
new HoodieKey("id10", "par4"),
new HoodieKey("id11", "par4"));
// this triggers the data write and event send
funcWrapper.checkpointFunction(2);
funcWrapper.checkpointFunction(1);
String instant = funcWrapper.getWriteClient()
.getLastPendingInstant(getTableType());
@@ -631,7 +637,7 @@ public class TestWriteCopyOnWrite {
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.REQUESTED, instant);
funcWrapper.checkpointComplete(2);
funcWrapper.checkpointComplete(1);
// the coordinator checkpoint commits the inflight instant.
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.COMPLETED, instant);
checkWrittenData(tempFile, EXPECTED2);

View File

@@ -23,6 +23,8 @@ import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.sink.bootstrap.BootstrapFunction;
import org.apache.hudi.sink.bootstrap.BootstrapRecord;
import org.apache.hudi.sink.StreamWriteFunction;
import org.apache.hudi.sink.StreamWriteOperatorCoordinator;
import org.apache.hudi.sink.event.BatchWriteSuccessEvent;
@@ -46,6 +48,7 @@ import org.apache.flink.table.data.RowData;
import org.apache.flink.util.Collector;
import java.util.HashSet;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Set;
@@ -68,6 +71,8 @@ public class StreamWriteFunctionWrapper<I> {
/** Function that converts row data to HoodieRecord. */
private RowDataToHoodieFunction<RowData, HoodieRecord<?>> toHoodieFunction;
/** Function that load index in state. */
private BootstrapFunction<HoodieRecord<?>, HoodieRecord<?>> bootstrapFunction;
/** Function that assigns bucket ID. */
private BucketAssignFunction<String, HoodieRecord<?>, HoodieRecord<?>> bucketAssignerFunction;
/** BucketAssignOperator context. **/
@@ -94,6 +99,8 @@ public class StreamWriteFunctionWrapper<I> {
// one function
this.coordinatorContext = new MockOperatorCoordinatorContext(new OperatorID(), 1);
this.coordinator = new StreamWriteOperatorCoordinator(conf, this.coordinatorContext);
this.compactFunctionWrapper = new CompactFunctionWrapper(this.conf);
this.bucketAssignOperatorContext = new MockBucketAssignOperatorContext();
this.functionInitializationContext = new MockFunctionInitializationContext();
this.compactFunctionWrapper = new CompactFunctionWrapper(this.conf);
this.bucketAssignOperatorContext = new MockBucketAssignOperatorContext();
@@ -106,6 +113,13 @@ public class StreamWriteFunctionWrapper<I> {
toHoodieFunction.setRuntimeContext(runtimeContext);
toHoodieFunction.open(conf);
if (conf.getBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED)) {
bootstrapFunction = new BootstrapFunction<>(conf);
bootstrapFunction.setRuntimeContext(runtimeContext);
bootstrapFunction.initializeState(this.functionInitializationContext);
bootstrapFunction.open(conf);
}
bucketAssignerFunction = new BucketAssignFunction<>(conf);
bucketAssignerFunction.setRuntimeContext(runtimeContext);
bucketAssignerFunction.open(conf);
@@ -136,6 +150,32 @@ public class StreamWriteFunctionWrapper<I> {
}
};
if (conf.getBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED)) {
List<HoodieRecord<?>> bootstrapRecords = new ArrayList<>();
Collector<HoodieRecord<?>> bootstrapCollector = new Collector<HoodieRecord<?>>() {
@Override
public void collect(HoodieRecord<?> record) {
if (record instanceof BootstrapRecord) {
bootstrapRecords.add(record);
}
}
@Override
public void close() {
}
};
bootstrapFunction.processElement(hoodieRecord, null, bootstrapCollector);
for (HoodieRecord bootstrapRecord : bootstrapRecords) {
bucketAssignerFunction.processElement(bootstrapRecord, null, collector);
}
this.bucketAssignOperatorContext.setCurrentKey(hoodieRecord.getRecordKey());
}
bucketAssignerFunction.processElement(hoodieRecord, null, collector);
writeFunction.processElement(hoodieRecords[0], null, null);
}
@@ -210,6 +250,10 @@ public class StreamWriteFunctionWrapper<I> {
return this.writeFunction.isConfirming();
}
public boolean isAlreadyBootstrap() {
return this.bootstrapFunction.isAlreadyBootstrap();
}
// -------------------------------------------------------------------------
// Inner Class
// -------------------------------------------------------------------------