1
0

[HUDI-2087] Support Append only in Flink stream (#3252)

Co-authored-by: 喻兆靖 <yuzhaojing@bilibili.com>
This commit is contained in:
yuzhaojing
2021-07-10 14:49:35 +08:00
committed by GitHub
parent 7c6eebf98c
commit 783c9cb369
14 changed files with 243 additions and 29 deletions

View File

@@ -23,6 +23,7 @@ import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.table.view.FileSystemViewStorageType;
@@ -44,6 +45,7 @@ import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
import java.io.File;
import java.util.Arrays;
import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
@@ -387,12 +389,12 @@ public class TestWriteCopyOnWrite {
@Test
public void testInsertWithMiniBatches() throws Exception {
// reset the config option
conf.setDouble(FlinkOptions.WRITE_BATCH_SIZE, 0.0006); // 630 bytes batch size
conf.setDouble(FlinkOptions.WRITE_BATCH_SIZE, 0.0007); // 734 bytes batch size
funcWrapper = new StreamWriteFunctionWrapper<>(tempFile.getAbsolutePath(), conf);
// open the function and ingest data
funcWrapper.openFunction();
// Each record is 208 bytes. so 4 records expect to trigger a mini-batch write
// Each record is 216 bytes. so 4 records expect to trigger a mini-batch write
for (RowData rowData : TestData.DATA_SET_INSERT_DUPLICATES) {
funcWrapper.invoke(rowData);
}
@@ -448,13 +450,13 @@ public class TestWriteCopyOnWrite {
@Test
public void testInsertWithDeduplication() throws Exception {
// reset the config option
conf.setDouble(FlinkOptions.WRITE_BATCH_SIZE, 0.0006); // 630 bytes batch size
conf.setDouble(FlinkOptions.WRITE_BATCH_SIZE, 0.0007); // 734 bytes batch size
conf.setBoolean(FlinkOptions.INSERT_DROP_DUPS, true);
funcWrapper = new StreamWriteFunctionWrapper<>(tempFile.getAbsolutePath(), conf);
// open the function and ingest data
funcWrapper.openFunction();
// Each record is 208 bytes. so 4 records expect to trigger a mini-batch write
// Each record is 216 bytes. so 4 records expect to trigger a mini-batch write
for (RowData rowData : TestData.DATA_SET_INSERT_SAME_KEY) {
funcWrapper.invoke(rowData);
}
@@ -510,14 +512,91 @@ public class TestWriteCopyOnWrite {
}
@Test
public void testInsertWithSmallBufferSize() throws Exception {
public void testAppendOnly() throws Exception {
// reset the config option
conf.setDouble(FlinkOptions.WRITE_TASK_MAX_SIZE, 200.0006); // 630 bytes buffer size
conf.setDouble(FlinkOptions.WRITE_BATCH_SIZE, 0.0007); // 734 bytes batch size
conf.setBoolean(FlinkOptions.INSERT_DROP_DUPS, false);
conf.setBoolean(FlinkOptions.APPEND_ONLY_ENABLE, true);
conf.setString(FlinkOptions.OPERATION, WriteOperationType.INSERT.value());
funcWrapper = new StreamWriteFunctionWrapper<>(tempFile.getAbsolutePath(), conf);
// open the function and ingest data
funcWrapper.openFunction();
// each record is 208 bytes. so 4 records expect to trigger buffer flush:
// Each record is 216 bytes. so 4 records expect to trigger a mini-batch write
for (RowData rowData : TestData.DATA_SET_INSERT_SAME_KEY) {
funcWrapper.invoke(rowData);
}
// this triggers the data write and event send
funcWrapper.checkpointFunction(1);
Map<String, List<HoodieRecord>> dataBuffer = funcWrapper.getDataBuffer();
assertThat("All data should be flushed out", dataBuffer.size(), is(0));
final OperatorEvent event1 = funcWrapper.getNextEvent(); // remove the first event first
final OperatorEvent event2 = funcWrapper.getNextEvent();
assertThat("The operator expect to send an event", event2, instanceOf(WriteMetadataEvent.class));
funcWrapper.getCoordinator().handleEventFromOperator(0, event1);
funcWrapper.getCoordinator().handleEventFromOperator(0, event2);
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
String instant = funcWrapper.getWriteClient()
.getLastPendingInstant(getTableType());
funcWrapper.checkpointComplete(1);
Map<String, List<String>> expected = new HashMap<>();
expected.put("par1", Arrays.asList(
"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.checkWrittenAllData(tempFile, expected, 1);
// started a new instant already
checkInflightInstant(funcWrapper.getWriteClient());
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.COMPLETED, instant);
// insert duplicates again
for (RowData rowData : TestData.DATA_SET_INSERT_SAME_KEY) {
funcWrapper.invoke(rowData);
}
funcWrapper.checkpointFunction(2);
final OperatorEvent event3 = funcWrapper.getNextEvent(); // remove the first event first
final OperatorEvent event4 = funcWrapper.getNextEvent();
funcWrapper.getCoordinator().handleEventFromOperator(0, event3);
funcWrapper.getCoordinator().handleEventFromOperator(0, event4);
funcWrapper.checkpointComplete(2);
// Same the original base file content.
expected.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"));
TestData.checkWrittenAllData(tempFile, expected, 1);
}
@Test
public void testInsertWithSmallBufferSize() throws Exception {
// reset the config option
conf.setDouble(FlinkOptions.WRITE_TASK_MAX_SIZE, 200.0007); // 734 bytes buffer size
funcWrapper = new StreamWriteFunctionWrapper<>(tempFile.getAbsolutePath(), conf);
// open the function and ingest data
funcWrapper.openFunction();
// each record is 216 bytes. so 4 records expect to trigger buffer flush:
// flush the max size bucket once at a time.
for (RowData rowData : TestData.DATA_SET_INSERT_DUPLICATES) {
funcWrapper.invoke(rowData);
@@ -660,7 +739,7 @@ public class TestWriteCopyOnWrite {
public void testWriteExactlyOnce() throws Exception {
// reset the config option
conf.setLong(FlinkOptions.WRITE_COMMIT_ACK_TIMEOUT, 3);
conf.setDouble(FlinkOptions.WRITE_TASK_MAX_SIZE, 200.0006); // 630 bytes buffer size
conf.setDouble(FlinkOptions.WRITE_TASK_MAX_SIZE, 200.0007); // 734 bytes buffer size
funcWrapper = new StreamWriteFunctionWrapper<>(tempFile.getAbsolutePath(), conf);
// open the function and ingest data

View File

@@ -23,12 +23,14 @@ import org.apache.hudi.client.common.HoodieFlinkEngineContext;
import org.apache.hudi.common.config.SerializableConfiguration;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.TableSchemaResolver;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.sink.utils.StreamWriteFunctionWrapper;
import org.apache.hudi.table.HoodieFlinkTable;
import org.apache.hudi.util.StreamerUtil;
import org.apache.hudi.utils.TestData;
@@ -37,6 +39,7 @@ import org.apache.avro.Schema;
import org.apache.flink.configuration.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.io.File;
import java.util.Comparator;
@@ -44,6 +47,8 @@ import java.util.HashMap;
import java.util.Map;
import java.util.stream.Collectors;
import static org.junit.jupiter.api.Assertions.assertThrows;
/**
* Test cases for delta stream write.
*/
@@ -86,6 +91,16 @@ public class TestWriteMergeOnRead extends TestWriteCopyOnWrite {
return EXPECTED1;
}
@Test
public void testAppendOnly() throws Exception {
conf.setBoolean(FlinkOptions.APPEND_ONLY_ENABLE, true);
conf.setString(FlinkOptions.OPERATION, WriteOperationType.INSERT.value());
funcWrapper = new StreamWriteFunctionWrapper<>(tempFile.getAbsolutePath(), conf);
assertThrows(IllegalArgumentException.class, () -> {
funcWrapper.openFunction();
}, "APPEND_ONLY mode only support in COPY_ON_WRITE table");
}
protected Map<String, String> getMiniBatchExpected() {
Map<String, String> expected = new HashMap<>();
// MOR mode merges the messages with the same key.

View File

@@ -19,15 +19,18 @@
package org.apache.hudi.sink;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.sink.utils.StreamWriteFunctionWrapper;
import org.apache.flink.configuration.Configuration;
import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Test;
import java.util.HashMap;
import java.util.Map;
import static org.junit.jupiter.api.Assertions.assertThrows;
/**
* Test cases for delta stream write with compaction.
*/
@@ -39,10 +42,19 @@ public class TestWriteMergeOnReadWithCompact extends TestWriteCopyOnWrite {
conf.setInteger(FlinkOptions.COMPACTION_DELTA_COMMITS, 1);
}
@Disabled
@Override
protected Map<String, String> getExpectedBeforeCheckpointComplete() {
return EXPECTED1;
}
@Test
public void testIndexStateBootstrap() {
// Ignore the index bootstrap because we only support parquet load now.
public void testAppendOnly() throws Exception {
conf.setBoolean(FlinkOptions.APPEND_ONLY_ENABLE, true);
conf.setString(FlinkOptions.OPERATION, WriteOperationType.INSERT.value());
funcWrapper = new StreamWriteFunctionWrapper<>(tempFile.getAbsolutePath(), conf);
assertThrows(IllegalArgumentException.class, () -> {
funcWrapper.openFunction();
}, "APPEND_ONLY mode only support in COPY_ON_WRITE table");
}
protected Map<String, String> getMiniBatchExpected() {

View File

@@ -360,8 +360,10 @@ public class TestData {
assert baseFile.isDirectory();
FileFilter filter = file -> !file.getName().startsWith(".");
File[] partitionDirs = baseFile.listFiles(filter);
assertNotNull(partitionDirs);
assertThat(partitionDirs.length, is(partitions));
for (File partitionDir : partitionDirs) {
File[] dataFiles = partitionDir.listFiles(filter);
assertNotNull(dataFiles);
@@ -381,6 +383,37 @@ public class TestData {
}
}
public static void checkWrittenAllData(
File baseFile,
Map<String, List<String>> expected,
int partitions) throws IOException {
assert baseFile.isDirectory();
FileFilter filter = file -> !file.getName().startsWith(".");
File[] partitionDirs = baseFile.listFiles(filter);
assertNotNull(partitionDirs);
assertThat(partitionDirs.length, is(partitions));
for (File partitionDir : partitionDirs) {
File[] dataFiles = partitionDir.listFiles(filter);
assertNotNull(dataFiles);
List<String> readBuffer = new ArrayList<>();
for (File dataFile : dataFiles) {
ParquetReader<GenericRecord> reader = AvroParquetReader
.<GenericRecord>builder(new Path(dataFile.getAbsolutePath())).build();
GenericRecord nextRecord = reader.read();
while (nextRecord != null) {
readBuffer.add(filterOutVariables(nextRecord));
nextRecord = reader.read();
}
readBuffer.sort(Comparator.naturalOrder());
}
assertThat(readBuffer, is(expected.get(partitionDir.getName())));
}
}
/**
* Checks the source data are written as expected.
*