[HUDI-4273] Support inline schedule clustering for Flink stream (#5890)
* [HUDI-4273] Support inline schedule clustering for Flink stream * delete deprecated clustering plan strategy and add clustering ITTest
This commit is contained in:
@@ -152,6 +152,17 @@ public class ITTestDataStreamWrite extends TestLogger {
|
||||
testWriteToHoodie(conf, "mor_write_with_compact", 1, EXPECTED);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWriteMergeOnReadWithClustering() throws Exception {
|
||||
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||
conf.setBoolean(FlinkOptions.CLUSTERING_SCHEDULE_ENABLED, true);
|
||||
conf.setInteger(FlinkOptions.CLUSTERING_DELTA_COMMITS, 1);
|
||||
conf.setString(FlinkOptions.OPERATION, "insert");
|
||||
conf.setString(FlinkOptions.TABLE_TYPE, HoodieTableType.COPY_ON_WRITE.name());
|
||||
|
||||
testWriteToHoodieWithCluster(conf, "cow_write_with_cluster", 1, EXPECTED);
|
||||
}
|
||||
|
||||
private void testWriteToHoodie(
|
||||
Transformer transformer,
|
||||
String jobName,
|
||||
@@ -250,6 +261,69 @@ public class ITTestDataStreamWrite extends TestLogger {
|
||||
}
|
||||
|
||||
TestData.checkWrittenFullData(tempFile, expected);
|
||||
}
|
||||
|
||||
private void testWriteToHoodieWithCluster(
|
||||
Configuration conf,
|
||||
String jobName,
|
||||
int checkpoints,
|
||||
Map<String, List<String>> expected) throws Exception {
|
||||
|
||||
StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment();
|
||||
execEnv.getConfig().disableObjectReuse();
|
||||
execEnv.setParallelism(4);
|
||||
// set up checkpoint interval
|
||||
execEnv.enableCheckpointing(4000, CheckpointingMode.EXACTLY_ONCE);
|
||||
execEnv.getCheckpointConfig().setMaxConcurrentCheckpoints(1);
|
||||
|
||||
// Read from file source
|
||||
RowType rowType =
|
||||
(RowType) AvroSchemaConverter.convertToDataType(StreamerUtil.getSourceSchema(conf))
|
||||
.getLogicalType();
|
||||
|
||||
JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema(
|
||||
rowType,
|
||||
InternalTypeInfo.of(rowType),
|
||||
false,
|
||||
true,
|
||||
TimestampFormat.ISO_8601
|
||||
);
|
||||
String sourcePath = Objects.requireNonNull(Thread.currentThread()
|
||||
.getContextClassLoader().getResource("test_source.data")).toString();
|
||||
|
||||
boolean isMor = conf.getString(FlinkOptions.TABLE_TYPE).equals(HoodieTableType.MERGE_ON_READ.name());
|
||||
|
||||
DataStream<RowData> dataStream;
|
||||
if (isMor) {
|
||||
TextInputFormat format = new TextInputFormat(new Path(sourcePath));
|
||||
format.setFilesFilter(FilePathFilter.createDefaultFilter());
|
||||
TypeInformation<String> typeInfo = BasicTypeInfo.STRING_TYPE_INFO;
|
||||
format.setCharsetName("UTF-8");
|
||||
|
||||
dataStream = 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(1);
|
||||
} else {
|
||||
dataStream = execEnv
|
||||
// use continuous file source to trigger checkpoint
|
||||
.addSource(new ContinuousFileSource.BoundedSourceFunction(new Path(sourcePath), checkpoints))
|
||||
.name("continuous_file_source")
|
||||
.setParallelism(1)
|
||||
.map(record -> deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8)))
|
||||
.setParallelism(4);
|
||||
}
|
||||
|
||||
DataStream<Object> pipeline = Pipelines.append(conf, rowType, dataStream, true);
|
||||
execEnv.addOperator(pipeline.getTransformation());
|
||||
|
||||
Pipelines.cluster(conf, rowType, pipeline);
|
||||
JobClient client = execEnv.executeAsync(jobName);
|
||||
|
||||
// wait for the streaming job to finish
|
||||
client.getJobExecutionResult().get();
|
||||
|
||||
TestData.checkWrittenFullData(tempFile, expected);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -286,6 +286,29 @@ public class TestWriteCopyOnWrite extends TestWriteBase {
|
||||
.end();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInsertAsyncClustering() throws Exception {
|
||||
// reset the config option
|
||||
conf.setString(FlinkOptions.OPERATION, "insert");
|
||||
conf.setBoolean(FlinkOptions.CLUSTERING_SCHEDULE_ENABLED, true);
|
||||
conf.setBoolean(FlinkOptions.CLUSTERING_ASYNC_ENABLED, true);
|
||||
conf.setInteger(FlinkOptions.CLUSTERING_DELTA_COMMITS, 1);
|
||||
|
||||
prepareInsertPipeline(conf)
|
||||
.consume(TestData.DATA_SET_INSERT_SAME_KEY)
|
||||
.checkpoint(1)
|
||||
.handleEvents(1)
|
||||
.checkpointComplete(1)
|
||||
.checkWrittenData(EXPECTED4, 1)
|
||||
// insert duplicates again
|
||||
.consume(TestData.DATA_SET_INSERT_SAME_KEY)
|
||||
.checkpoint(2)
|
||||
.handleEvents(1)
|
||||
.checkpointComplete(2)
|
||||
.checkWrittenFullData(EXPECTED5)
|
||||
.end();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInsertWithSmallBufferSize() throws Exception {
|
||||
// reset the config option
|
||||
@@ -419,7 +442,7 @@ public class TestWriteCopyOnWrite extends TestWriteBase {
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
private TestHarness preparePipeline() throws Exception {
|
||||
return TestHarness.instance().preparePipeline(tempFile, conf);
|
||||
return preparePipeline(conf);
|
||||
}
|
||||
|
||||
protected TestHarness preparePipeline(Configuration conf) throws Exception {
|
||||
@@ -427,6 +450,10 @@ public class TestWriteCopyOnWrite extends TestWriteBase {
|
||||
}
|
||||
|
||||
protected TestHarness prepareInsertPipeline() throws Exception {
|
||||
return prepareInsertPipeline(conf);
|
||||
}
|
||||
|
||||
protected TestHarness prepareInsertPipeline(Configuration conf) throws Exception {
|
||||
return TestHarness.instance().preparePipeline(tempFile, conf, true);
|
||||
}
|
||||
|
||||
|
||||
@@ -115,7 +115,6 @@ public class ITTestHoodieFlinkClustering {
|
||||
|
||||
// set the table name
|
||||
conf.setString(FlinkOptions.TABLE_NAME, metaClient.getTableConfig().getTableName());
|
||||
conf.setString(FlinkOptions.TABLE_TYPE, metaClient.getTableConfig().getTableType().name());
|
||||
|
||||
// set record key field
|
||||
conf.setString(FlinkOptions.RECORD_KEY_FIELD, metaClient.getTableConfig().getRecordKeyFieldProp());
|
||||
@@ -133,7 +132,7 @@ public class ITTestHoodieFlinkClustering {
|
||||
// To compute the clustering instant time and do clustering.
|
||||
String clusteringInstantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
|
||||
HoodieFlinkWriteClient writeClient = StreamerUtil.createWriteClient(conf, null);
|
||||
HoodieFlinkWriteClient writeClient = StreamerUtil.createWriteClient(conf);
|
||||
HoodieFlinkTable<?> table = writeClient.getHoodieTable();
|
||||
|
||||
boolean scheduled = writeClient.scheduleClusteringAtInstant(clusteringInstantTime, Option.empty());
|
||||
@@ -160,7 +159,7 @@ public class ITTestHoodieFlinkClustering {
|
||||
final DataType rowDataType = AvroSchemaConverter.convertToDataType(tableAvroSchema);
|
||||
final RowType rowType = (RowType) rowDataType.getLogicalType();
|
||||
|
||||
DataStream<ClusteringCommitEvent> dataStream = env.addSource(new ClusteringPlanSourceFunction(timeline.lastInstant().get(), clusteringPlan))
|
||||
DataStream<ClusteringCommitEvent> dataStream = env.addSource(new ClusteringPlanSourceFunction(clusteringInstantTime, clusteringPlan))
|
||||
.name("clustering_source")
|
||||
.uid("uid_clustering_source")
|
||||
.rebalance()
|
||||
@@ -181,4 +180,44 @@ public class ITTestHoodieFlinkClustering {
|
||||
env.execute("flink_hudi_clustering");
|
||||
TestData.checkWrittenData(tempFile, EXPECTED, 4);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHoodieFlinkClusteringService() throws Exception {
|
||||
// Create hoodie table and insert into data.
|
||||
EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build();
|
||||
TableEnvironment tableEnv = TableEnvironmentImpl.create(settings);
|
||||
tableEnv.getConfig().getConfiguration()
|
||||
.setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1);
|
||||
Map<String, String> options = new HashMap<>();
|
||||
options.put(FlinkOptions.PATH.key(), tempFile.getAbsolutePath());
|
||||
|
||||
// use append mode
|
||||
options.put(FlinkOptions.OPERATION.key(), WriteOperationType.INSERT.value());
|
||||
options.put(FlinkOptions.INSERT_CLUSTER.key(), "false");
|
||||
|
||||
String hoodieTableDDL = TestConfigurations.getCreateHoodieTableDDL("t1", options);
|
||||
tableEnv.executeSql(hoodieTableDDL);
|
||||
tableEnv.executeSql(TestSQL.INSERT_T1).await();
|
||||
|
||||
// wait for the asynchronous commit to finish
|
||||
TimeUnit.SECONDS.sleep(3);
|
||||
|
||||
// Make configuration and setAvroSchema.
|
||||
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
|
||||
FlinkClusteringConfig cfg = new FlinkClusteringConfig();
|
||||
cfg.path = tempFile.getAbsolutePath();
|
||||
cfg.minClusteringIntervalSeconds = 3;
|
||||
cfg.schedule = true;
|
||||
Configuration conf = FlinkClusteringConfig.toFlinkConfig(cfg);
|
||||
|
||||
HoodieFlinkClusteringJob.AsyncClusteringService asyncClusteringService = new HoodieFlinkClusteringJob.AsyncClusteringService(cfg, conf, env);
|
||||
asyncClusteringService.start(null);
|
||||
|
||||
// wait for the asynchronous commit to finish
|
||||
TimeUnit.SECONDS.sleep(5);
|
||||
|
||||
asyncClusteringService.shutDown();
|
||||
|
||||
TestData.checkWrittenData(tempFile, EXPECTED, 4);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,121 @@
|
||||
/*
|
||||
* 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.sink.clustering.ClusteringCommitEvent;
|
||||
import org.apache.hudi.sink.clustering.ClusteringCommitSink;
|
||||
import org.apache.hudi.sink.clustering.ClusteringOperator;
|
||||
import org.apache.hudi.sink.clustering.ClusteringPlanEvent;
|
||||
import org.apache.hudi.sink.clustering.ClusteringPlanOperator;
|
||||
import org.apache.hudi.utils.TestConfigurations;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
|
||||
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
|
||||
import org.apache.flink.runtime.jobgraph.OperatorID;
|
||||
import org.apache.flink.runtime.memory.MemoryManager;
|
||||
import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext;
|
||||
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.StreamingRuntimeContext;
|
||||
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
|
||||
import org.apache.flink.streaming.runtime.tasks.StreamTask;
|
||||
import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
|
||||
|
||||
/**
|
||||
* A wrapper class to manipulate the {@link ClusteringOperator} instance for testing.
|
||||
*/
|
||||
public class ClusteringFunctionWrapper {
|
||||
private final Configuration conf;
|
||||
|
||||
private final IOManager ioManager;
|
||||
private final StreamingRuntimeContext runtimeContext;
|
||||
|
||||
private final StreamTask<?, ?> streamTask;
|
||||
private final StreamConfig streamConfig;
|
||||
|
||||
/**
|
||||
* Function that generates the {@code HoodieClusteringPlan}.
|
||||
*/
|
||||
private ClusteringPlanOperator clusteringPlanOperator;
|
||||
/**
|
||||
* Output to collect the clustering commit events.
|
||||
*/
|
||||
private CollectorOutput<ClusteringCommitEvent> commitEventOutput;
|
||||
/**
|
||||
* Function that executes the clustering task.
|
||||
*/
|
||||
private ClusteringOperator clusteringOperator;
|
||||
/**
|
||||
* Stream sink to handle clustering commits.
|
||||
*/
|
||||
private ClusteringCommitSink commitSink;
|
||||
|
||||
public ClusteringFunctionWrapper(Configuration conf, StreamTask<?, ?> streamTask, StreamConfig streamConfig) {
|
||||
this.ioManager = new IOManagerAsync();
|
||||
MockEnvironment environment = new MockEnvironmentBuilder()
|
||||
.setTaskName("mockTask")
|
||||
.setManagedMemorySize(4 * MemoryManager.DEFAULT_PAGE_SIZE)
|
||||
.setIOManager(ioManager)
|
||||
.build();
|
||||
this.runtimeContext = new MockStreamingRuntimeContext(false, 1, 0, environment);
|
||||
this.conf = conf;
|
||||
this.streamTask = streamTask;
|
||||
this.streamConfig = streamConfig;
|
||||
}
|
||||
|
||||
public void openFunction() throws Exception {
|
||||
clusteringPlanOperator = new ClusteringPlanOperator(conf);
|
||||
clusteringPlanOperator.open();
|
||||
|
||||
clusteringOperator = new ClusteringOperator(conf, TestConfigurations.ROW_TYPE);
|
||||
// CAUTION: deprecated API used.
|
||||
clusteringOperator.setProcessingTimeService(new TestProcessingTimeService());
|
||||
commitEventOutput = new CollectorOutput<>();
|
||||
clusteringOperator.setup(streamTask, streamConfig, commitEventOutput);
|
||||
clusteringOperator.open();
|
||||
final NonThrownExecutor syncExecutor = new MockCoordinatorExecutor(
|
||||
new MockOperatorCoordinatorContext(new OperatorID(), 1));
|
||||
clusteringOperator.setExecutor(syncExecutor);
|
||||
|
||||
commitSink = new ClusteringCommitSink(conf);
|
||||
commitSink.setRuntimeContext(runtimeContext);
|
||||
commitSink.open(conf);
|
||||
}
|
||||
|
||||
public void cluster(long checkpointID) throws Exception {
|
||||
// collect the ClusteringPlanEvents.
|
||||
CollectorOutput<ClusteringPlanEvent> planOutput = new CollectorOutput<>();
|
||||
clusteringPlanOperator.setOutput(planOutput);
|
||||
clusteringPlanOperator.notifyCheckpointComplete(checkpointID);
|
||||
// collect the ClusteringCommitEvents
|
||||
for (ClusteringPlanEvent event : planOutput.getRecords()) {
|
||||
clusteringOperator.processElement(new StreamRecord<>(event));
|
||||
}
|
||||
// handle and commit the clustering
|
||||
for (ClusteringCommitEvent event : commitEventOutput.getRecords()) {
|
||||
commitSink.invoke(event, null);
|
||||
}
|
||||
}
|
||||
|
||||
public void close() throws Exception {
|
||||
ioManager.close();
|
||||
}
|
||||
}
|
||||
@@ -18,6 +18,8 @@
|
||||
|
||||
package org.apache.hudi.sink.utils;
|
||||
|
||||
import org.apache.hudi.configuration.OptionsResolver;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.sink.StreamWriteOperatorCoordinator;
|
||||
import org.apache.hudi.sink.append.AppendWriteFunction;
|
||||
import org.apache.hudi.sink.bulk.BulkInsertWriterHelper;
|
||||
@@ -25,6 +27,7 @@ import org.apache.hudi.sink.event.WriteMetadataEvent;
|
||||
import org.apache.hudi.util.AvroSchemaConverter;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
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;
|
||||
@@ -34,9 +37,12 @@ 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.StreamingRuntimeContext;
|
||||
import org.apache.flink.streaming.api.operators.collect.utils.MockFunctionSnapshotContext;
|
||||
import org.apache.flink.streaming.api.operators.collect.utils.MockOperatorEventGateway;
|
||||
import org.apache.flink.streaming.runtime.tasks.StreamTask;
|
||||
import org.apache.flink.streaming.util.MockStreamTaskBuilder;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
|
||||
@@ -57,12 +63,15 @@ public class InsertFunctionWrapper<I> implements TestFunctionWrapper<I> {
|
||||
private final StreamWriteOperatorCoordinator coordinator;
|
||||
private final MockStateInitializationContext stateInitializationContext;
|
||||
|
||||
private final boolean asyncClustering;
|
||||
private ClusteringFunctionWrapper clusteringFunctionWrapper;
|
||||
|
||||
/**
|
||||
* Append write function.
|
||||
*/
|
||||
private AppendWriteFunction<RowData> writeFunction;
|
||||
|
||||
public InsertFunctionWrapper(String tablePath, Configuration conf) {
|
||||
public InsertFunctionWrapper(String tablePath, Configuration conf) throws Exception {
|
||||
IOManager ioManager = new IOManagerAsync();
|
||||
MockEnvironment environment = new MockEnvironmentBuilder()
|
||||
.setTaskName("mockTask")
|
||||
@@ -77,6 +86,15 @@ public class InsertFunctionWrapper<I> implements TestFunctionWrapper<I> {
|
||||
this.coordinatorContext = new MockOperatorCoordinatorContext(new OperatorID(), 1);
|
||||
this.coordinator = new StreamWriteOperatorCoordinator(conf, this.coordinatorContext);
|
||||
this.stateInitializationContext = new MockStateInitializationContext();
|
||||
|
||||
this.asyncClustering = OptionsResolver.needsAsyncClustering(conf);
|
||||
StreamConfig streamConfig = new StreamConfig(conf);
|
||||
streamConfig.setOperatorID(new OperatorID());
|
||||
StreamTask<?, ?> streamTask = new MockStreamTaskBuilder(environment)
|
||||
.setConfig(new StreamConfig(conf))
|
||||
.setExecutionConfig(new ExecutionConfig().enableObjectReuse())
|
||||
.build();
|
||||
this.clusteringFunctionWrapper = new ClusteringFunctionWrapper(this.conf, streamTask, streamConfig);
|
||||
}
|
||||
|
||||
public void openFunction() throws Exception {
|
||||
@@ -84,6 +102,10 @@ public class InsertFunctionWrapper<I> implements TestFunctionWrapper<I> {
|
||||
this.coordinator.setExecutor(new MockCoordinatorExecutor(coordinatorContext));
|
||||
|
||||
setupWriteFunction();
|
||||
|
||||
if (asyncClustering) {
|
||||
clusteringFunctionWrapper.openFunction();
|
||||
}
|
||||
}
|
||||
|
||||
public void invoke(I record) throws Exception {
|
||||
@@ -109,6 +131,13 @@ public class InsertFunctionWrapper<I> implements TestFunctionWrapper<I> {
|
||||
public void checkpointComplete(long checkpointId) {
|
||||
stateInitializationContext.getOperatorStateStore().checkpointSuccess(checkpointId);
|
||||
coordinator.notifyCheckpointComplete(checkpointId);
|
||||
if (asyncClustering) {
|
||||
try {
|
||||
clusteringFunctionWrapper.cluster(checkpointId);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public StreamWriteOperatorCoordinator getCoordinator() {
|
||||
@@ -118,6 +147,9 @@ public class InsertFunctionWrapper<I> implements TestFunctionWrapper<I> {
|
||||
@Override
|
||||
public void close() throws Exception {
|
||||
this.coordinator.close();
|
||||
if (clusteringFunctionWrapper != null) {
|
||||
clusteringFunctionWrapper.close();
|
||||
}
|
||||
}
|
||||
|
||||
public BulkInsertWriterHelper getWriterHelper() {
|
||||
|
||||
@@ -21,6 +21,7 @@ package org.apache.hudi.sink.utils;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.configuration.OptionsResolver;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.sink.StreamWriteFunction;
|
||||
import org.apache.hudi.sink.StreamWriteOperatorCoordinator;
|
||||
@@ -28,7 +29,6 @@ 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.transform.RowDataToHoodieFunction;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
import org.apache.hudi.utils.TestConfigurations;
|
||||
|
||||
import org.apache.flink.api.common.ExecutionConfig;
|
||||
@@ -120,8 +120,7 @@ public class StreamWriteFunctionWrapper<I> implements TestFunctionWrapper<I> {
|
||||
this.compactFunctionWrapper = new CompactFunctionWrapper(this.conf);
|
||||
this.bucketAssignFunctionContext = new MockBucketAssignFunctionContext();
|
||||
this.stateInitializationContext = new MockStateInitializationContext();
|
||||
this.compactFunctionWrapper = new CompactFunctionWrapper(this.conf);
|
||||
this.asyncCompaction = StreamerUtil.needsAsyncCompaction(conf);
|
||||
this.asyncCompaction = OptionsResolver.needsAsyncCompaction(conf);
|
||||
this.streamConfig = new StreamConfig(conf);
|
||||
streamConfig.setOperatorID(new OperatorID());
|
||||
this.streamTask = new MockStreamTaskBuilder(environment)
|
||||
|
||||
@@ -25,6 +25,7 @@ import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.TableSchemaResolver;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.configuration.HadoopConfigurations;
|
||||
import org.apache.hudi.configuration.OptionsResolver;
|
||||
@@ -337,7 +338,9 @@ public class TestWriteBase {
|
||||
public TestHarness checkWrittenData(
|
||||
Map<String, String> expected,
|
||||
int partitions) throws Exception {
|
||||
if (OptionsResolver.isCowTable(conf) || conf.getBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED)) {
|
||||
if (OptionsResolver.isCowTable(conf)
|
||||
|| conf.getBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED)
|
||||
|| OptionsResolver.isAppendMode(conf)) {
|
||||
TestData.checkWrittenData(this.baseFile, expected, partitions);
|
||||
} else {
|
||||
checkWrittenDataMor(baseFile, expected, partitions);
|
||||
@@ -419,7 +422,7 @@ public class TestWriteBase {
|
||||
protected String lastCompleteInstant() {
|
||||
return OptionsResolver.isMorTable(conf)
|
||||
? TestUtils.getLastDeltaCompleteInstant(basePath)
|
||||
: TestUtils.getLastCompleteInstant(basePath);
|
||||
: TestUtils.getLastCompleteInstant(basePath, HoodieTimeline.COMMIT_ACTION);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -50,6 +50,16 @@ public class TestUtils {
|
||||
return StreamerUtil.getLastCompletedInstant(metaClient);
|
||||
}
|
||||
|
||||
public static String getLastCompleteInstant(String basePath, String commitAction) {
|
||||
final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
|
||||
.setConf(HadoopConfigurations.getHadoopConf(new Configuration())).setBasePath(basePath).build();
|
||||
return metaClient.getCommitsTimeline().filterCompletedInstants()
|
||||
.filter(instant -> commitAction.equals(instant.getAction()))
|
||||
.lastInstant()
|
||||
.map(HoodieInstant::getTimestamp)
|
||||
.orElse(null);
|
||||
}
|
||||
|
||||
public static String getLastDeltaCompleteInstant(String basePath) {
|
||||
final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
|
||||
.setConf(HadoopConfigurations.getHadoopConf(new Configuration())).setBasePath(basePath).build();
|
||||
|
||||
Reference in New Issue
Block a user