1
0

[HUDI-2449] Incremental read for Flink (#3686)

This commit is contained in:
Danny Chan
2021-09-19 09:06:46 +08:00
committed by GitHub
parent c7a5c8273b
commit 3354fac42f
16 changed files with 689 additions and 365 deletions

View File

@@ -88,4 +88,18 @@ public class TestFileIndex {
assertThat(fileStatuses.length, is(1));
assertTrue(fileStatuses[0].getPath().toString().endsWith(HoodieFileFormat.PARQUET.getFileExtension()));
}
@ParameterizedTest
@ValueSource(booleans = {true, false})
void testFileListingEmptyTable(boolean enableMetadata) {
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
conf.setBoolean(FlinkOptions.METADATA_ENABLED, enableMetadata);
FileIndex fileIndex = FileIndex.instance(new Path(tempFile.getAbsolutePath()), conf);
List<String> partitionKeys = Collections.singletonList("partition");
List<Map<String, String>> partitions = fileIndex.getPartitions(partitionKeys, "default", false);
assertThat(partitions.size(), is(0));
FileStatus[] fileStatuses = fileIndex.getFilesInPartitions();
assertThat(fileStatuses.length, is(0));
}
}

View File

@@ -144,7 +144,7 @@ public class TestStreamReadMonitoringFunction {
TestData.writeData(TestData.DATA_SET_INSERT, conf);
TestData.writeData(TestData.DATA_SET_UPDATE_INSERT, conf);
String specifiedCommit = TestUtils.getLatestCommit(tempFile.getAbsolutePath());
conf.setString(FlinkOptions.READ_STREAMING_START_COMMIT, specifiedCommit);
conf.setString(FlinkOptions.READ_START_COMMIT, specifiedCommit);
StreamReadMonitoringFunction function = TestUtils.getMonitorFunc(conf);
try (AbstractStreamOperatorTestHarness<MergeOnReadInputSplit> harness = createHarness(function)) {
harness.setup();
@@ -175,7 +175,7 @@ public class TestStreamReadMonitoringFunction {
TestData.writeData(TestData.DATA_SET_INSERT, conf);
TestData.writeData(TestData.DATA_SET_UPDATE_INSERT, conf);
String specifiedCommit = TestUtils.getLatestCommit(tempFile.getAbsolutePath());
conf.setString(FlinkOptions.READ_STREAMING_START_COMMIT, FlinkOptions.START_COMMIT_EARLIEST);
conf.setString(FlinkOptions.READ_START_COMMIT, FlinkOptions.START_COMMIT_EARLIEST);
StreamReadMonitoringFunction function = TestUtils.getMonitorFunc(conf);
try (AbstractStreamOperatorTestHarness<MergeOnReadInputSplit> harness = createHarness(function)) {
harness.setup();

View File

@@ -22,7 +22,6 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.TableSchemaResolver;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.table.format.FilePathUtils;
import org.apache.hudi.table.format.mor.MergeOnReadInputFormat;
import org.apache.hudi.table.format.mor.MergeOnReadInputSplit;
import org.apache.hudi.table.format.mor.MergeOnReadTableState;
@@ -45,7 +44,6 @@ import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.RowType;
import org.apache.hadoop.fs.Path;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
@@ -263,10 +261,8 @@ public class TestStreamReadOperator {
AvroSchemaConverter.convertToSchema(TestConfigurations.ROW_TYPE).toString(),
Collections.emptyList(),
new String[0]);
Path[] paths = FilePathUtils.getReadPaths(new Path(basePath), conf, hadoopConf, partitionKeys);
MergeOnReadInputFormat inputFormat = MergeOnReadInputFormat.builder()
.config(conf)
.paths(FilePathUtils.toFlinkPaths(paths))
.tableState(hoodieTableState)
.fieldTypes(rowDataType.getChildren())
.defaultPartName("default").limit(1000L)

View File

@@ -113,7 +113,7 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.READ_AS_STREAMING, true)
.option(FlinkOptions.TABLE_TYPE, tableType)
.option(FlinkOptions.READ_STREAMING_START_COMMIT, firstCommit)
.option(FlinkOptions.READ_START_COMMIT, firstCommit)
.end();
streamTableEnv.executeSql(hoodieTableDDL);
List<Row> rows = execSelectSql(streamTableEnv, "select * from t1", 10);
@@ -186,7 +186,7 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.READ_AS_STREAMING, true)
.option(FlinkOptions.TABLE_TYPE, tableType)
.option(FlinkOptions.READ_STREAMING_START_COMMIT, specifiedCommit)
.option(FlinkOptions.READ_START_COMMIT, specifiedCommit)
.end();
streamTableEnv.executeSql(createHoodieTable2);
List<Row> rows = execSelectSql(streamTableEnv, "select * from t2", 10);
@@ -289,7 +289,7 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
.option(FlinkOptions.TABLE_TYPE, FlinkOptions.TABLE_TYPE_MERGE_ON_READ)
.option(FlinkOptions.READ_AS_STREAMING, true)
.option(FlinkOptions.READ_STREAMING_CHECK_INTERVAL, 2)
.option(FlinkOptions.READ_STREAMING_START_COMMIT, latestCommit)
.option(FlinkOptions.READ_START_COMMIT, latestCommit)
.option(FlinkOptions.CHANGELOG_ENABLED, true)
.end();
streamTableEnv.executeSql(hoodieTableDDL);
@@ -343,7 +343,7 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.TABLE_TYPE, FlinkOptions.TABLE_TYPE_MERGE_ON_READ)
.option(FlinkOptions.READ_AS_STREAMING, true)
.option(FlinkOptions.READ_STREAMING_START_COMMIT, FlinkOptions.START_COMMIT_EARLIEST)
.option(FlinkOptions.READ_START_COMMIT, FlinkOptions.START_COMMIT_EARLIEST)
.option(FlinkOptions.READ_STREAMING_CHECK_INTERVAL, 2)
// close the async compaction
.option(FlinkOptions.COMPACTION_ASYNC_ENABLED, false)
@@ -879,6 +879,33 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
assertRowsEquals(result1, "[+I[1.23, 12345678.12, 12345.12, 123456789.123450000000000000]]");
}
@ParameterizedTest
@EnumSource(value = HoodieTableType.class)
void testIncrementalRead(HoodieTableType tableType) throws Exception {
TableEnvironment tableEnv = batchTableEnv;
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
conf.setString(FlinkOptions.TABLE_NAME, "t1");
conf.setString(FlinkOptions.TABLE_TYPE, tableType.name());
// write 3 batches of data set
TestData.writeData(TestData.dataSetInsert(1, 2), conf);
TestData.writeData(TestData.dataSetInsert(3, 4), conf);
TestData.writeData(TestData.dataSetInsert(5, 6), conf);
String latestCommit = TestUtils.getLatestCommit(tempFile.getAbsolutePath());
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.TABLE_TYPE, tableType)
.option(FlinkOptions.READ_START_COMMIT, latestCommit)
.end();
tableEnv.executeSql(hoodieTableDDL);
List<Row> result = CollectionUtil.iterableToList(
() -> tableEnv.sqlQuery("select * from t1").execute().collect());
assertRowsEquals(result, TestData.dataSetInsert(5, 6));
}
// -------------------------------------------------------------------------
// Utilities
// -------------------------------------------------------------------------

View File

@@ -231,6 +231,32 @@ public class TestHoodieTableFactory {
assertThat(conf2.getInteger(FlinkOptions.ARCHIVE_MAX_COMMITS), is(45));
}
@Test
void testSetupReadOptionsForSource() {
// definition with simple primary key and partition path
ResolvedSchema schema1 = SchemaBuilder.instance()
.field("f0", DataTypes.INT().notNull())
.field("f1", DataTypes.VARCHAR(20))
.field("f2", DataTypes.TIMESTAMP(3))
.field("ts", DataTypes.TIMESTAMP(3))
.primaryKey("f0")
.build();
// set up new retains commits that is less than min archive commits
this.conf.setString(FlinkOptions.READ_END_COMMIT, "123");
final MockContext sourceContext1 = MockContext.getInstance(this.conf, schema1, "f2");
final HoodieTableSource tableSource1 = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext1);
final Configuration conf1 = tableSource1.getConf();
assertThat(conf1.getString(FlinkOptions.QUERY_TYPE), is(FlinkOptions.QUERY_TYPE_INCREMENTAL));
this.conf.removeConfig(FlinkOptions.READ_END_COMMIT);
this.conf.setString(FlinkOptions.READ_START_COMMIT, "123");
final MockContext sourceContext2 = MockContext.getInstance(this.conf, schema1, "f2");
final HoodieTableSource tableSource2 = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext2);
final Configuration conf2 = tableSource2.getConf();
assertThat(conf2.getString(FlinkOptions.QUERY_TYPE), is(FlinkOptions.QUERY_TYPE_INCREMENTAL));
}
@Test
void testInferAvroSchemaForSink() {
// infer the schema if not specified

View File

@@ -19,7 +19,6 @@
package org.apache.hudi.table;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.table.format.mor.MergeOnReadInputFormat;
import org.apache.hudi.utils.TestConfigurations;
import org.apache.hudi.utils.TestData;
@@ -31,6 +30,7 @@ import org.apache.flink.configuration.Configuration;
import org.apache.flink.table.data.RowData;
import org.apache.hadoop.fs.Path;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.function.ThrowingSupplier;
import org.junit.jupiter.api.io.TempDir;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -46,9 +46,9 @@ import java.util.stream.Collectors;
import static org.hamcrest.CoreMatchers.instanceOf;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.core.Is.is;
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
/**
* Test cases for HoodieTableSource.
@@ -112,9 +112,9 @@ public class TestHoodieTableSource {
inputFormat = tableSource.getInputFormat();
assertThat(inputFormat, is(instanceOf(MergeOnReadInputFormat.class)));
conf.setString(FlinkOptions.QUERY_TYPE.key(), FlinkOptions.QUERY_TYPE_INCREMENTAL);
assertThrows(HoodieException.class,
() -> tableSource.getInputFormat(),
"Invalid query type : 'incremental'. Only 'snapshot' is supported now");
assertDoesNotThrow(
(ThrowingSupplier<? extends InputFormat<RowData, ?>>) tableSource::getInputFormat,
"Query type: 'incremental' should be supported");
}
@Test

View File

@@ -19,6 +19,8 @@
package org.apache.hudi.table.format;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.table.HoodieTableSource;
import org.apache.hudi.table.format.cow.CopyOnWriteInputFormat;
@@ -44,6 +46,7 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import static org.hamcrest.CoreMatchers.instanceOf;
import static org.hamcrest.CoreMatchers.is;
@@ -71,12 +74,7 @@ public class TestInputFormat {
options.forEach((key, value) -> conf.setString(key, value));
StreamerUtil.initTableIfNotExists(conf);
this.tableSource = new HoodieTableSource(
TestConfigurations.TABLE_SCHEMA,
new Path(tempFile.getAbsolutePath()),
Collections.singletonList("partition"),
"default",
conf);
this.tableSource = getTableSource(conf);
}
@ParameterizedTest
@@ -385,10 +383,81 @@ public class TestInputFormat {
assertThat(actual, is(expected));
}
@ParameterizedTest
@EnumSource(value = HoodieTableType.class)
void testReadIncrementally(HoodieTableType tableType) throws Exception {
Map<String, String> options = new HashMap<>();
options.put(FlinkOptions.QUERY_TYPE.key(), FlinkOptions.QUERY_TYPE_INCREMENTAL);
beforeEach(tableType, options);
// write another commit to read again
for (int i = 0; i < 6; i += 2) {
List<RowData> dataset = TestData.dataSetInsert(i + 1, i + 2);
TestData.writeData(dataset, conf);
}
HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(tempFile.getAbsolutePath());
List<String> commits = metaClient.getCommitsTimeline().filterCompletedInstants().getInstants()
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
assertThat(commits.size(), is(3));
// only the start commit
conf.setString(FlinkOptions.READ_START_COMMIT, commits.get(1));
this.tableSource = getTableSource(conf);
InputFormat<RowData, ?> inputFormat1 = this.tableSource.getInputFormat();
assertThat(inputFormat1, instanceOf(MergeOnReadInputFormat.class));
List<RowData> actual1 = readData(inputFormat1);
final List<RowData> expected1 = TestData.dataSetInsert(3, 4, 5, 6);
TestData.assertRowDataEquals(actual1, expected1);
// only the start commit: earliest
conf.setString(FlinkOptions.READ_START_COMMIT, FlinkOptions.START_COMMIT_EARLIEST);
this.tableSource = getTableSource(conf);
InputFormat<RowData, ?> inputFormat2 = this.tableSource.getInputFormat();
assertThat(inputFormat2, instanceOf(MergeOnReadInputFormat.class));
List<RowData> actual2 = readData(inputFormat2);
final List<RowData> expected2 = TestData.dataSetInsert(1, 2, 3, 4, 5, 6);
TestData.assertRowDataEquals(actual2, expected2);
// start and end commit: [start commit, end commit]
conf.setString(FlinkOptions.READ_START_COMMIT, commits.get(0));
conf.setString(FlinkOptions.READ_END_COMMIT, commits.get(1));
this.tableSource = getTableSource(conf);
InputFormat<RowData, ?> inputFormat3 = this.tableSource.getInputFormat();
assertThat(inputFormat3, instanceOf(MergeOnReadInputFormat.class));
List<RowData> actual3 = readData(inputFormat3);
final List<RowData> expected3 = TestData.dataSetInsert(1, 2, 3, 4);
TestData.assertRowDataEquals(actual3, expected3);
// only the end commit: point in time query
conf.removeConfig(FlinkOptions.READ_START_COMMIT);
conf.setString(FlinkOptions.READ_END_COMMIT, commits.get(1));
this.tableSource = getTableSource(conf);
InputFormat<RowData, ?> inputFormat4 = this.tableSource.getInputFormat();
assertThat(inputFormat4, instanceOf(MergeOnReadInputFormat.class));
List<RowData> actual4 = readData(inputFormat4);
final List<RowData> expected4 = TestData.dataSetInsert(3, 4);
TestData.assertRowDataEquals(actual4, expected4);
}
// -------------------------------------------------------------------------
// Utilities
// -------------------------------------------------------------------------
private HoodieTableSource getTableSource(Configuration conf) {
return new HoodieTableSource(
TestConfigurations.TABLE_SCHEMA,
new Path(tempFile.getAbsolutePath()),
Collections.singletonList("partition"),
"default",
conf);
}
@SuppressWarnings("unchecked, rawtypes")
private static List<RowData> readData(InputFormat inputFormat) throws IOException {
InputSplit[] inputSplits = inputFormat.createInputSplits(1);

View File

@@ -284,6 +284,14 @@ public class TestData {
TimestampData.fromEpochMillis(2), StringData.fromString("par1"))
);
public static List<RowData> dataSetInsert(int... ids) {
List<RowData> inserts = new ArrayList<>();
Arrays.stream(ids).forEach(i -> inserts.add(
insertRow(StringData.fromString("id" + i), StringData.fromString("Danny"), 23,
TimestampData.fromEpochMillis(i), StringData.fromString("par1"))));
return inserts;
}
private static Integer toIdSafely(Object id) {
if (id == null) {
return -1;
@@ -424,7 +432,7 @@ public class TestData {
*/
public static void assertRowDataEquals(List<RowData> rows, List<RowData> expected) {
String rowsString = rowDataToString(rows);
assertThat(rowDataToString(expected), is(rowsString));
assertThat(rowsString, is(rowDataToString(expected)));
}
/**

View File

@@ -28,7 +28,6 @@ import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.fs.Path;
import java.io.File;
import java.util.Collections;
import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -58,6 +57,6 @@ public class TestUtils {
public static StreamReadMonitoringFunction getMonitorFunc(Configuration conf) {
final String basePath = conf.getString(FlinkOptions.PATH);
return new StreamReadMonitoringFunction(conf, new Path(basePath), 1024 * 1024L, Collections.emptySet());
return new StreamReadMonitoringFunction(conf, new Path(basePath), 1024 * 1024L, null);
}
}