[HUDI-2578] Support merging small files for flink insert operation (#3822)
This commit is contained in:
@@ -46,6 +46,7 @@ import org.junit.jupiter.api.io.TempDir;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
@@ -528,7 +529,7 @@ public class TestWriteCopyOnWrite {
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInsertAllowsDuplication() throws Exception {
|
||||
public void testInsertAppendMode() throws Exception {
|
||||
InsertFunctionWrapper<RowData> funcWrapper = new InsertFunctionWrapper<>(tempFile.getAbsolutePath(), conf);
|
||||
|
||||
// open the function and ingest data
|
||||
@@ -593,6 +594,95 @@ public class TestWriteCopyOnWrite {
|
||||
TestData.checkWrittenAllData(tempFile, expected, 1);
|
||||
}
|
||||
|
||||
/**
|
||||
* The test is almost same with {@link #testInsertWithSmallBufferSize} except that
|
||||
* it is with insert clustering mode.
|
||||
*/
|
||||
@Test
|
||||
public void testInsertClustering() throws Exception {
|
||||
// reset the config option
|
||||
conf.setString(FlinkOptions.OPERATION, "insert");
|
||||
conf.setBoolean(FlinkOptions.INSERT_CLUSTER, true);
|
||||
conf.setDouble(FlinkOptions.WRITE_TASK_MAX_SIZE, 200.0008); // 839 bytes buffer size
|
||||
funcWrapper = new StreamWriteFunctionWrapper<>(tempFile.getAbsolutePath(), conf);
|
||||
|
||||
// open the function and ingest data
|
||||
funcWrapper.openFunction();
|
||||
// record (operation: 'I') is 304 bytes and record (operation: 'U') is 352 bytes.
|
||||
// so 3 records expect to trigger a mini-batch write
|
||||
// flush the max size bucket once at a time.
|
||||
for (RowData rowData : TestData.DATA_SET_INSERT_SAME_KEY) {
|
||||
funcWrapper.invoke(rowData);
|
||||
}
|
||||
|
||||
Map<String, List<HoodieRecord>> dataBuffer = funcWrapper.getDataBuffer();
|
||||
assertThat("Should have 1 data bucket", dataBuffer.size(), is(1));
|
||||
assertThat("2 records expect to flush out as a mini-batch",
|
||||
dataBuffer.values().stream().findFirst().map(List::size).orElse(-1),
|
||||
is(2));
|
||||
|
||||
// this triggers the data write and event send
|
||||
funcWrapper.checkpointFunction(1);
|
||||
dataBuffer = funcWrapper.getDataBuffer();
|
||||
assertThat("All data should be flushed out", dataBuffer.size(), is(0));
|
||||
|
||||
for (int i = 0; i < 2; i++) {
|
||||
final OperatorEvent event = funcWrapper.getNextEvent(); // remove the first event first
|
||||
assertThat("The operator expect to send an event", event, instanceOf(WriteMetadataEvent.class));
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, event);
|
||||
}
|
||||
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
|
||||
|
||||
String instant = lastPendingInstant();
|
||||
|
||||
funcWrapper.checkpointComplete(1);
|
||||
|
||||
Map<String, String> expected = new HashMap<>();
|
||||
|
||||
expected.put("par1", "["
|
||||
+ "id1,par1,id1,Danny,23,0,par1, "
|
||||
+ "id1,par1,id1,Danny,23,1,par1, "
|
||||
+ "id1,par1,id1,Danny,23,2,par1, "
|
||||
+ "id1,par1,id1,Danny,23,3,par1, "
|
||||
+ "id1,par1,id1,Danny,23,4,par1]");
|
||||
TestData.checkWrittenData(tempFile, expected, 1);
|
||||
|
||||
// started a new instant already
|
||||
checkInflightInstant();
|
||||
checkInstantState(HoodieInstant.State.COMPLETED, instant);
|
||||
|
||||
// insert duplicates again
|
||||
for (RowData rowData : TestData.DATA_SET_INSERT_SAME_KEY) {
|
||||
funcWrapper.invoke(rowData);
|
||||
}
|
||||
|
||||
funcWrapper.checkpointFunction(2);
|
||||
|
||||
for (int i = 0; i < 2; i++) {
|
||||
final OperatorEvent event = funcWrapper.getNextEvent(); // remove the first event first
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, event);
|
||||
}
|
||||
|
||||
funcWrapper.checkpointComplete(2);
|
||||
|
||||
// same with the original base file content.
|
||||
Map<String, List<String>> expected2 = new HashMap<>();
|
||||
expected2.put("par1", Arrays.asList(
|
||||
"id1,par1,id1,Danny,23,0,par1",
|
||||
"id1,par1,id1,Danny,23,0,par1",
|
||||
"id1,par1,id1,Danny,23,1,par1",
|
||||
"id1,par1,id1,Danny,23,1,par1",
|
||||
"id1,par1,id1,Danny,23,2,par1",
|
||||
"id1,par1,id1,Danny,23,2,par1",
|
||||
"id1,par1,id1,Danny,23,3,par1",
|
||||
"id1,par1,id1,Danny,23,3,par1",
|
||||
"id1,par1,id1,Danny,23,4,par1",
|
||||
"id1,par1,id1,Danny,23,4,par1"));
|
||||
|
||||
// Same the original base file content.
|
||||
TestData.checkWrittenFullData(tempFile, expected2);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInsertWithSmallBufferSize() throws Exception {
|
||||
// reset the config option
|
||||
|
||||
@@ -63,6 +63,11 @@ public class TestWriteMergeOnRead extends TestWriteCopyOnWrite {
|
||||
conf.setBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void testInsertClustering() {
|
||||
// insert clustering is only valid for cow table.
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void checkWrittenData(File baseFile, Map<String, String> expected, int partitions) throws Exception {
|
||||
HoodieTableMetaClient metaClient = HoodieFlinkTable.create(writeConfig, context).getMetaClient();
|
||||
|
||||
@@ -38,6 +38,11 @@ public class TestWriteMergeOnReadWithCompact extends TestWriteCopyOnWrite {
|
||||
conf.setInteger(FlinkOptions.COMPACTION_DELTA_COMMITS, 1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void testInsertClustering() {
|
||||
// insert clustering is only valid for cow table.
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Map<String, String> getExpectedBeforeCheckpointComplete() {
|
||||
return EXPECTED1;
|
||||
|
||||
@@ -35,6 +35,7 @@ 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.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.table.data.RowData;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
@@ -101,7 +102,7 @@ public class InsertFunctionWrapper<I> {
|
||||
// checkpoint the coordinator first
|
||||
this.coordinator.checkpointCoordinator(checkpointId, new CompletableFuture<>());
|
||||
|
||||
writeFunction.snapshotState(null);
|
||||
writeFunction.snapshotState(new MockFunctionSnapshotContext(checkpointId));
|
||||
stateInitializationContext.getOperatorStateStore().checkpointBegin(checkpointId);
|
||||
}
|
||||
|
||||
|
||||
@@ -45,6 +45,7 @@ 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.MockFunctionSnapshotContext;
|
||||
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;
|
||||
@@ -218,7 +219,7 @@ public class StreamWriteFunctionWrapper<I> {
|
||||
}
|
||||
bucketAssignerFunction.snapshotState(null);
|
||||
|
||||
writeFunction.snapshotState(null);
|
||||
writeFunction.snapshotState(new MockFunctionSnapshotContext(checkpointId));
|
||||
stateInitializationContext.getOperatorStateStore().checkpointBegin(checkpointId);
|
||||
}
|
||||
|
||||
|
||||
@@ -850,9 +850,10 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
|
||||
+ "+I[id1, Sophia, 18, 1970-01-01T00:00:05, par5]]", 3);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testAppendWrite() {
|
||||
TableEnvironment tableEnv = batchTableEnv;
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
void testAppendWrite(boolean clustering) {
|
||||
TableEnvironment tableEnv = streamTableEnv;
|
||||
// csv source
|
||||
String csvSourceDDL = TestConfigurations.getCsvSourceDDL("csv_source", "test_source_5.data");
|
||||
tableEnv.executeSql(csvSourceDDL);
|
||||
@@ -860,7 +861,7 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
|
||||
String hoodieTableDDL = sql("hoodie_sink")
|
||||
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
|
||||
.option(FlinkOptions.OPERATION, "insert")
|
||||
.option(FlinkOptions.INSERT_DEDUP, false)
|
||||
.option(FlinkOptions.INSERT_CLUSTER, clustering)
|
||||
.end();
|
||||
tableEnv.executeSql(hoodieTableDDL);
|
||||
|
||||
|
||||
Reference in New Issue
Block a user