[HUDI-2342] Optimize Bootstrap operator (#3516)
Co-authored-by: 喻兆靖 <yuzhaojing@bilibili.com>
This commit is contained in:
@@ -28,7 +28,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.sink.bootstrap.BootstrapFunction;
|
||||
import org.apache.hudi.sink.bootstrap.BootstrapOperator;
|
||||
import org.apache.hudi.sink.compact.CompactFunction;
|
||||
import org.apache.hudi.sink.compact.CompactionCommitEvent;
|
||||
import org.apache.hudi.sink.compact.CompactionCommitSink;
|
||||
@@ -277,7 +277,7 @@ public class StreamWriteITCase extends TestLogger {
|
||||
if (conf.getBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED)) {
|
||||
hoodieDataStream = hoodieDataStream.transform("index_bootstrap",
|
||||
TypeInformation.of(HoodieRecord.class),
|
||||
new ProcessOperator<>(new BootstrapFunction<>(conf)));
|
||||
new BootstrapOperator<>(conf));
|
||||
}
|
||||
|
||||
DataStream<Object> pipeline = hoodieDataStream
|
||||
@@ -370,7 +370,7 @@ public class StreamWriteITCase extends TestLogger {
|
||||
if (conf.getBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED)) {
|
||||
hoodieDataStream = hoodieDataStream.transform("index_bootstrap",
|
||||
TypeInformation.of(HoodieRecord.class),
|
||||
new ProcessOperator<>(new BootstrapFunction<>(conf)));
|
||||
new BootstrapOperator<>(conf));
|
||||
}
|
||||
|
||||
DataStream<Object> pipeline = hoodieDataStream
|
||||
|
||||
@@ -725,8 +725,6 @@ public class TestWriteCopyOnWrite {
|
||||
funcWrapper.invoke(rowData);
|
||||
}
|
||||
|
||||
assertTrue(funcWrapper.isAlreadyBootstrap());
|
||||
|
||||
checkIndexLoaded(
|
||||
new HoodieKey("id1", "par1"),
|
||||
new HoodieKey("id2", "par1"),
|
||||
@@ -743,6 +741,8 @@ public class TestWriteCopyOnWrite {
|
||||
// this triggers the data write and event send
|
||||
funcWrapper.checkpointFunction(1);
|
||||
|
||||
assertTrue(funcWrapper.isAlreadyBootstrap());
|
||||
|
||||
String instant = funcWrapper.getWriteClient()
|
||||
.getLastPendingInstant(getTableType());
|
||||
|
||||
|
||||
@@ -0,0 +1,78 @@
|
||||
/*
|
||||
* 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.flink.streaming.api.operators.Output;
|
||||
import org.apache.flink.streaming.api.watermark.Watermark;
|
||||
import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker;
|
||||
import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
|
||||
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
|
||||
import org.apache.flink.util.InstantiationUtil;
|
||||
import org.apache.flink.util.OutputTag;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Collecting {@link Output} for {@link StreamRecord}.
|
||||
*/
|
||||
public class CollectorOutput<T> implements Output<StreamRecord<T>> {
|
||||
|
||||
private final List<StreamElement> list;
|
||||
|
||||
public CollectorOutput(List<StreamElement> list) {
|
||||
this.list = list;
|
||||
}
|
||||
|
||||
public List<StreamElement> getList() {
|
||||
return list;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void emitWatermark(Watermark mark) {
|
||||
list.add(mark);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void emitLatencyMarker(LatencyMarker latencyMarker) {
|
||||
list.add(latencyMarker);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collect(StreamRecord<T> record) {
|
||||
try {
|
||||
ClassLoader cl = record.getClass().getClassLoader();
|
||||
T copied =
|
||||
InstantiationUtil.deserializeObject(
|
||||
InstantiationUtil.serializeObject(record.getValue()), cl);
|
||||
list.add(record.copy(copied));
|
||||
} catch (IOException | ClassNotFoundException ex) {
|
||||
throw new RuntimeException("Unable to deserialize record: " + record, ex);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public <X> void collect(OutputTag<X> outputTag, StreamRecord<X> record) {
|
||||
throw new UnsupportedOperationException("Side output not supported for CollectorOutput");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
}
|
||||
}
|
||||
@@ -52,7 +52,6 @@ public class CompactFunctionWrapper {
|
||||
|
||||
private final IOManager ioManager;
|
||||
private final StreamingRuntimeContext runtimeContext;
|
||||
private final MockFunctionInitializationContext functionInitializationContext;
|
||||
|
||||
/** Function that generates the {@link HoodieCompactionPlan}. */
|
||||
private CompactionPlanOperator compactionPlanOperator;
|
||||
@@ -70,7 +69,6 @@ public class CompactFunctionWrapper {
|
||||
.build();
|
||||
this.runtimeContext = new MockStreamingRuntimeContext(false, 1, 0, environment);
|
||||
this.conf = conf;
|
||||
this.functionInitializationContext = new MockFunctionInitializationContext();
|
||||
}
|
||||
|
||||
public void openFunction() throws Exception {
|
||||
|
||||
@@ -19,15 +19,18 @@ package org.apache.hudi.sink.utils;
|
||||
|
||||
import org.apache.flink.api.common.state.KeyedStateStore;
|
||||
import org.apache.flink.runtime.state.FunctionInitializationContext;
|
||||
import org.apache.flink.runtime.state.KeyGroupStatePartitionStreamProvider;
|
||||
import org.apache.flink.runtime.state.StateInitializationContext;
|
||||
import org.apache.flink.runtime.state.StatePartitionStreamProvider;
|
||||
|
||||
/**
|
||||
* A {@link FunctionInitializationContext} for testing purpose.
|
||||
*/
|
||||
public class MockFunctionInitializationContext implements FunctionInitializationContext {
|
||||
public class MockStateInitializationContext implements StateInitializationContext {
|
||||
|
||||
private final MockOperatorStateStore operatorStateStore;
|
||||
|
||||
public MockFunctionInitializationContext() {
|
||||
public MockStateInitializationContext() {
|
||||
operatorStateStore = new MockOperatorStateStore();
|
||||
}
|
||||
|
||||
@@ -45,4 +48,14 @@ public class MockFunctionInitializationContext implements FunctionInitialization
|
||||
public KeyedStateStore getKeyedStateStore() {
|
||||
return operatorStateStore;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterable<StatePartitionStreamProvider> getRawOperatorStateInputs() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterable<KeyGroupStatePartitionStreamProvider> getRawKeyedStateInputs() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
@@ -25,8 +25,7 @@ import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.sink.StreamWriteFunction;
|
||||
import org.apache.hudi.sink.StreamWriteOperatorCoordinator;
|
||||
import org.apache.hudi.sink.bootstrap.BootstrapFunction;
|
||||
import org.apache.hudi.sink.bootstrap.IndexRecord;
|
||||
import org.apache.hudi.sink.bootstrap.BootstrapOperator;
|
||||
import org.apache.hudi.sink.event.WriteMetadataEvent;
|
||||
import org.apache.hudi.sink.partitioner.BucketAssignFunction;
|
||||
import org.apache.hudi.sink.partitioner.BucketAssignOperator;
|
||||
@@ -34,6 +33,7 @@ import org.apache.hudi.sink.transform.RowDataToHoodieFunction;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
import org.apache.hudi.utils.TestConfigurations;
|
||||
|
||||
import org.apache.flink.api.common.ExecutionConfig;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
|
||||
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
|
||||
@@ -43,8 +43,14 @@ import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorCo
|
||||
import org.apache.flink.runtime.operators.coordination.OperatorEvent;
|
||||
import org.apache.flink.runtime.operators.testutils.MockEnvironment;
|
||||
import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder;
|
||||
import org.apache.flink.streaming.api.graph.StreamConfig;
|
||||
import org.apache.flink.streaming.api.operators.Output;
|
||||
import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
|
||||
import org.apache.flink.streaming.api.operators.collect.utils.MockOperatorEventGateway;
|
||||
import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
|
||||
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
|
||||
import org.apache.flink.streaming.util.MockStreamTask;
|
||||
import org.apache.flink.streaming.util.MockStreamTaskBuilder;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.util.Collector;
|
||||
|
||||
@@ -68,7 +74,7 @@ public class StreamWriteFunctionWrapper<I> {
|
||||
private final MockOperatorEventGateway gateway;
|
||||
private final MockOperatorCoordinatorContext coordinatorContext;
|
||||
private final StreamWriteOperatorCoordinator coordinator;
|
||||
private final MockFunctionInitializationContext functionInitializationContext;
|
||||
private final MockStateInitializationContext stateInitializationContext;
|
||||
|
||||
/**
|
||||
* Function that converts row data to HoodieRecord.
|
||||
@@ -77,7 +83,7 @@ public class StreamWriteFunctionWrapper<I> {
|
||||
/**
|
||||
* Function that load index in state.
|
||||
*/
|
||||
private BootstrapFunction<HoodieRecord<?>, HoodieRecord<?>> bootstrapFunction;
|
||||
private BootstrapOperator<HoodieRecord<?>, HoodieRecord<?>> bootstrapOperator;
|
||||
/**
|
||||
* Function that assigns bucket ID.
|
||||
*/
|
||||
@@ -93,6 +99,12 @@ public class StreamWriteFunctionWrapper<I> {
|
||||
|
||||
private CompactFunctionWrapper compactFunctionWrapper;
|
||||
|
||||
private final Output<StreamRecord<HoodieRecord<?>>> output;
|
||||
|
||||
private final MockStreamTask streamTask;
|
||||
|
||||
private final StreamConfig streamConfig;
|
||||
|
||||
private final boolean asyncCompaction;
|
||||
|
||||
public StreamWriteFunctionWrapper(String tablePath) throws Exception {
|
||||
@@ -114,10 +126,17 @@ public class StreamWriteFunctionWrapper<I> {
|
||||
this.coordinator = new StreamWriteOperatorCoordinator(conf, this.coordinatorContext);
|
||||
this.compactFunctionWrapper = new CompactFunctionWrapper(this.conf);
|
||||
this.bucketAssignOperatorContext = new MockBucketAssignOperatorContext();
|
||||
this.functionInitializationContext = new MockFunctionInitializationContext();
|
||||
this.stateInitializationContext = new MockStateInitializationContext();
|
||||
this.compactFunctionWrapper = new CompactFunctionWrapper(this.conf);
|
||||
this.asyncCompaction = StreamerUtil.needsAsyncCompaction(conf);
|
||||
this.bucketAssignOperatorContext = new MockBucketAssignOperatorContext();
|
||||
this.output = new CollectorOutput<>(new ArrayList<>());
|
||||
this.streamConfig = new StreamConfig(conf);
|
||||
streamConfig.setOperatorID(new OperatorID());
|
||||
this.streamTask = new MockStreamTaskBuilder(environment)
|
||||
.setConfig(new StreamConfig(conf))
|
||||
.setExecutionConfig(new ExecutionConfig().enableObjectReuse())
|
||||
.build();
|
||||
}
|
||||
|
||||
public void openFunction() throws Exception {
|
||||
@@ -128,16 +147,16 @@ public class StreamWriteFunctionWrapper<I> {
|
||||
toHoodieFunction.open(conf);
|
||||
|
||||
if (conf.getBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED)) {
|
||||
bootstrapFunction = new BootstrapFunction<>(conf);
|
||||
bootstrapFunction.setRuntimeContext(runtimeContext);
|
||||
bootstrapFunction.open(conf);
|
||||
bootstrapOperator = new BootstrapOperator<>(conf);
|
||||
bootstrapOperator.setup(streamTask, streamConfig, output);
|
||||
bootstrapOperator.initializeState(this.stateInitializationContext);
|
||||
}
|
||||
|
||||
bucketAssignerFunction = new BucketAssignFunction<>(conf);
|
||||
bucketAssignerFunction.setRuntimeContext(runtimeContext);
|
||||
bucketAssignerFunction.open(conf);
|
||||
bucketAssignerFunction.setContext(bucketAssignOperatorContext);
|
||||
bucketAssignerFunction.initializeState(this.functionInitializationContext);
|
||||
bucketAssignerFunction.initializeState(this.stateInitializationContext);
|
||||
|
||||
setupWriteFunction();
|
||||
|
||||
@@ -146,6 +165,7 @@ public class StreamWriteFunctionWrapper<I> {
|
||||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void invoke(I record) throws Exception {
|
||||
HoodieRecord<?> hoodieRecord = toHoodieFunction.map((RowData) record);
|
||||
HoodieRecord<?>[] hoodieRecords = new HoodieRecord[1];
|
||||
@@ -162,27 +182,16 @@ 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 IndexRecord) {
|
||||
bootstrapRecords.add(record);
|
||||
}
|
||||
List<StreamElement> list = ((CollectorOutput) output).getList();
|
||||
for (StreamElement streamElement : list) {
|
||||
if (streamElement.isRecord()) {
|
||||
HoodieRecord<?> bootstrapRecord = (HoodieRecord<?>) streamElement.asRecord().getValue();
|
||||
bucketAssignerFunction.processElement(bootstrapRecord, null, collector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
|
||||
}
|
||||
};
|
||||
|
||||
bootstrapFunction.processElement(hoodieRecord, null, bootstrapCollector);
|
||||
for (HoodieRecord bootstrapRecord : bootstrapRecords) {
|
||||
bucketAssignerFunction.processElement(bootstrapRecord, null, collector);
|
||||
}
|
||||
|
||||
bootstrapOperator.processElement(new StreamRecord<>(hoodieRecord));
|
||||
list.clear();
|
||||
this.bucketAssignOperatorContext.setCurrentKey(hoodieRecord.getRecordKey());
|
||||
}
|
||||
|
||||
@@ -210,14 +219,17 @@ public class StreamWriteFunctionWrapper<I> {
|
||||
public void checkpointFunction(long checkpointId) throws Exception {
|
||||
// checkpoint the coordinator first
|
||||
this.coordinator.checkpointCoordinator(checkpointId, new CompletableFuture<>());
|
||||
if (conf.getBoolean(FlinkOptions.INDEX_BOOTSTRAP_ENABLED)) {
|
||||
bootstrapOperator.snapshotState(null);
|
||||
}
|
||||
bucketAssignerFunction.snapshotState(null);
|
||||
|
||||
writeFunction.snapshotState(null);
|
||||
functionInitializationContext.getOperatorStateStore().checkpointBegin(checkpointId);
|
||||
stateInitializationContext.getOperatorStateStore().checkpointBegin(checkpointId);
|
||||
}
|
||||
|
||||
public void checkpointComplete(long checkpointId) {
|
||||
functionInitializationContext.getOperatorStateStore().checkpointSuccess(checkpointId);
|
||||
stateInitializationContext.getOperatorStateStore().checkpointSuccess(checkpointId);
|
||||
coordinator.notifyCheckpointComplete(checkpointId);
|
||||
this.bucketAssignerFunction.notifyCheckpointComplete(checkpointId);
|
||||
if (asyncCompaction) {
|
||||
@@ -264,8 +276,8 @@ public class StreamWriteFunctionWrapper<I> {
|
||||
return this.writeFunction.isConfirming();
|
||||
}
|
||||
|
||||
public boolean isAlreadyBootstrap() {
|
||||
return this.bootstrapFunction.isAlreadyBootstrap();
|
||||
public boolean isAlreadyBootstrap() throws Exception {
|
||||
return this.bootstrapOperator.isAlreadyBootstrap();
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
@@ -276,7 +288,7 @@ public class StreamWriteFunctionWrapper<I> {
|
||||
writeFunction = new StreamWriteFunction<>(conf);
|
||||
writeFunction.setRuntimeContext(runtimeContext);
|
||||
writeFunction.setOperatorEventGateway(gateway);
|
||||
writeFunction.initializeState(this.functionInitializationContext);
|
||||
writeFunction.initializeState(this.stateInitializationContext);
|
||||
writeFunction.open(conf);
|
||||
|
||||
// handle the bootstrap event
|
||||
|
||||
Reference in New Issue
Block a user