1
0

[HUDI-4391] Incremental read from archived commits for flink (#6096)

This commit is contained in:
Danny Chan
2022-07-14 15:19:26 +08:00
committed by GitHub
parent ee956b8951
commit e70a427956
8 changed files with 461 additions and 40 deletions

View File

@@ -40,6 +40,7 @@ import org.apache.flink.table.api.config.ExecutionConfigOptions;
import org.apache.flink.table.api.internal.TableEnvironmentImpl;
import org.apache.flink.table.catalog.ObjectPath;
import org.apache.flink.table.catalog.exceptions.TableNotExistException;
import org.apache.flink.table.data.RowData;
import org.apache.flink.test.util.AbstractTestBase;
import org.apache.flink.types.Row;
import org.apache.flink.util.CollectionUtil;
@@ -1120,6 +1121,39 @@ public class ITTestHoodieDataSource extends AbstractTestBase {
assertRowsEquals(result, TestData.dataSetInsert(5, 6));
}
@ParameterizedTest
@EnumSource(value = HoodieTableType.class)
void testIncrementalReadArchivedCommits(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());
conf.setInteger(FlinkOptions.ARCHIVE_MIN_COMMITS, 3);
conf.setInteger(FlinkOptions.ARCHIVE_MAX_COMMITS, 4);
conf.setInteger(FlinkOptions.CLEAN_RETAIN_COMMITS, 2);
conf.setString("hoodie.commits.archival.batch", "1");
// write 10 batches of data set
for (int i = 0; i < 20; i += 2) {
List<RowData> dataset = TestData.dataSetInsert(i + 1, i + 2);
TestData.writeData(dataset, conf);
}
String secondArchived = TestUtils.getNthArchivedInstant(tempFile.getAbsolutePath(), 1);
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.TABLE_TYPE, tableType)
.option(FlinkOptions.READ_START_COMMIT, secondArchived)
.end();
tableEnv.executeSql(hoodieTableDDL);
List<Row> result = CollectionUtil.iterableToList(
() -> tableEnv.sqlQuery("select * from t1").execute().collect());
assertRowsEquals(result, TestData.dataSetInsert(3, 4, 5, 6, 7, 8, 9, 10,
11, 12, 13, 14, 15, 16, 17, 18, 19, 20));
}
@ParameterizedTest
@EnumSource(value = HoodieTableType.class)
void testReadWithWiderSchema(HoodieTableType tableType) throws Exception {

View File

@@ -18,6 +18,8 @@
package org.apache.hudi.table.format;
import org.apache.hudi.client.HoodieFlinkWriteClient;
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
@@ -448,6 +450,104 @@ public class TestInputFormat {
List<RowData> actual4 = readData(inputFormat4);
final List<RowData> expected4 = TestData.dataSetInsert(3, 4);
TestData.assertRowDataEquals(actual4, expected4);
// start and end commit: start commit out of range
conf.setString(FlinkOptions.READ_START_COMMIT, "000");
conf.setString(FlinkOptions.READ_END_COMMIT, commits.get(1));
this.tableSource = getTableSource(conf);
InputFormat<RowData, ?> inputFormat5 = this.tableSource.getInputFormat();
assertThat(inputFormat4, instanceOf(MergeOnReadInputFormat.class));
List<RowData> actual5 = readData(inputFormat5);
final List<RowData> expected5 = TestData.dataSetInsert(1, 2, 3, 4);
TestData.assertRowDataEquals(actual5, expected5);
// start and end commit: both are out of range
conf.setString(FlinkOptions.READ_START_COMMIT, "001");
conf.setString(FlinkOptions.READ_END_COMMIT, "002");
this.tableSource = getTableSource(conf);
InputFormat<RowData, ?> inputFormat6 = this.tableSource.getInputFormat();
assertThat(inputFormat6, instanceOf(MergeOnReadInputFormat.class));
List<RowData> actual6 = readData(inputFormat6);
TestData.assertRowDataEquals(actual6, Collections.emptyList());
}
@Test
void testReadArchivedCommitsIncrementally() throws Exception {
Map<String, String> options = new HashMap<>();
options.put(FlinkOptions.QUERY_TYPE.key(), FlinkOptions.QUERY_TYPE_INCREMENTAL);
options.put(FlinkOptions.ARCHIVE_MIN_COMMITS.key(), "3");
options.put(FlinkOptions.ARCHIVE_MAX_COMMITS.key(), "4");
options.put(FlinkOptions.CLEAN_RETAIN_COMMITS.key(), "2");
options.put("hoodie.commits.archival.batch", "1");
beforeEach(HoodieTableType.COPY_ON_WRITE, options);
// write 10 batches of data set
for (int i = 0; i < 20; i += 2) {
List<RowData> dataset = TestData.dataSetInsert(i + 1, i + 2);
TestData.writeData(dataset, conf);
}
// cleaning
HoodieFlinkWriteClient<?> writeClient = new HoodieFlinkWriteClient<>(
HoodieFlinkEngineContext.DEFAULT, StreamerUtil.getHoodieClientConfig(conf));
writeClient.clean();
HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(tempFile.getAbsolutePath(), HadoopConfigurations.getHadoopConf(conf));
List<String> commits = metaClient.getCommitsTimeline().filterCompletedInstants().getInstants()
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
assertThat(commits.size(), is(4));
List<String> archivedCommits = metaClient.getArchivedTimeline().getCommitsTimeline().filterCompletedInstants()
.getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList());
assertThat(archivedCommits.size(), is(6));
// start and end commit: both are archived and cleaned
conf.setString(FlinkOptions.READ_START_COMMIT, archivedCommits.get(0));
conf.setString(FlinkOptions.READ_END_COMMIT, archivedCommits.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(1, 2, 3, 4);
TestData.assertRowDataEquals(actual1, expected1);
// only the start commit: is archived and cleaned
conf.setString(FlinkOptions.READ_START_COMMIT, archivedCommits.get(1));
conf.removeConfig(FlinkOptions.READ_END_COMMIT);
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(3, 4, 5, 6, 7, 8, 9, 10,
11, 12, 13, 14, 15, 16, 17, 18, 19, 20);
TestData.assertRowDataEquals(actual2, expected2);
// only the end commit: is archived and cleaned
conf.removeConfig(FlinkOptions.READ_START_COMMIT);
conf.setString(FlinkOptions.READ_END_COMMIT, archivedCommits.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(3, 4);
TestData.assertRowDataEquals(actual3, expected3);
// start and end commit: start is archived and cleaned, end is active
conf.setString(FlinkOptions.READ_START_COMMIT, archivedCommits.get(1));
conf.setString(FlinkOptions.READ_END_COMMIT, commits.get(0));
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, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14);
TestData.assertRowDataEquals(actual4, expected4);
}
@ParameterizedTest

View File

@@ -88,6 +88,14 @@ public class TestUtils {
.orElse(null);
}
@Nullable
public static String getNthArchivedInstant(String basePath, int n) {
final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
.setConf(HadoopConfigurations.getHadoopConf(new Configuration())).setBasePath(basePath).build();
return metaClient.getArchivedTimeline().getCommitsTimeline().filterCompletedInstants()
.nthInstant(n).map(HoodieInstant::getTimestamp).orElse(null);
}
public static String getSplitPartitionPath(MergeOnReadInputSplit split) {
assertTrue(split.getLogPaths().isPresent());
final String logPath = split.getLogPaths().get().get(0);