1
0

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

Co-authored-by: 喻兆靖 <yuzhaojing@bilibili.com>
This commit is contained in:
yuzhaojing
2021-08-04 17:53:20 +08:00
committed by GitHub
parent 02331fc223
commit b8b9d6db83
14 changed files with 213 additions and 32 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;
@@ -532,6 +533,81 @@ public class TestWriteCopyOnWrite {
checkWrittenData(tempFile, expected, 1);
}
@Test
public void testInsertAllowsDuplication() throws Exception {
// reset the config option
conf.setDouble(FlinkOptions.WRITE_BATCH_SIZE, 0.0006); // 630 bytes batch size
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 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, 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.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 with the original base file content.
expected.put("par1", "["
+ "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

View File

@@ -37,6 +37,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;
@@ -67,6 +68,11 @@ public class TestWriteMergeOnRead extends TestWriteCopyOnWrite {
conf.setBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED, false);
}
@Test
public void testInsertAllowsDuplication() {
// ignore the test because only COW table supports INSERT duplication
}
@Override
protected void checkWrittenData(File baseFile, Map<String, String> expected, int partitions) throws Exception {
HoodieTableMetaClient metaClient = HoodieFlinkTable.create(writeConfig, context).getMetaClient();

View File

@@ -22,7 +22,6 @@ import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.flink.configuration.Configuration;
import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Test;
import java.util.HashMap;
@@ -39,10 +38,14 @@ public class TestWriteMergeOnReadWithCompact extends TestWriteCopyOnWrite {
conf.setInteger(FlinkOptions.COMPACTION_DELTA_COMMITS, 1);
}
@Disabled
@Test
public void testIndexStateBootstrap() {
// Ignore the index bootstrap because we only support parquet load now.
public void testInsertAllowsDuplication() {
// ignore the test because only COW table supports INSERT duplication
}
@Override
protected Map<String, String> getExpectedBeforeCheckpointComplete() {
return EXPECTED1;
}
protected Map<String, String> getMiniBatchExpected() {

View File

@@ -340,6 +340,24 @@ public class TestHoodieTableFactory {
assertThat(conf2.getInteger(FlinkOptions.ARCHIVE_MAX_COMMITS), is(45));
}
@Test
void testMorTableInsertAllowDuplication() {
TableSchema schema = TableSchema.builder()
.field("f0", DataTypes.INT().notNull())
.field("f1", DataTypes.VARCHAR(20))
.field("f2", DataTypes.TIMESTAMP(3))
.field("ts", DataTypes.TIMESTAMP(3))
.primaryKey("f0")
.build();
// overwrite the operation
this.conf.setString(FlinkOptions.OPERATION.key(), "insert");
this.conf.setString(FlinkOptions.TABLE_TYPE.key(), FlinkOptions.TABLE_TYPE_MERGE_ON_READ);
final MockContext sinkContext = MockContext.getInstance(this.conf, schema, "f2");
assertThrows(ValidationException.class, () -> new HoodieTableFactory().createDynamicTableSink(sinkContext),
"Option 'write.insert.allow_dup' is only allowed for COPY_ON_WRITE table.");
}
// -------------------------------------------------------------------------
// Inner Class
// -------------------------------------------------------------------------

View File

@@ -394,6 +394,48 @@ public class TestData {
}
}
/**
* Checks the source data set are written as expected.
* Different with {@link #checkWrittenData}, it reads all the data files.
*
* <p>Note: Replace it with the Flink reader when it is supported.
*
* @param baseFile The file base to check, should be a directory
* @param expected The expected results mapping, the key should be the partition path
* and value should be values list with the key partition
* @param partitions The expected partition number
*/
public static void checkWrittenAllData(
File baseFile,
Map<String, 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.toString(), is(expected.get(partitionDir.getName())));
}
}
/**
* Checks the source data are written as expected.
*