[HUDI-2371] Improvement flink streaming reader (#3552)
- Support reading empty table - Fix filtering by partition path - Support reading from earliest commit
This commit is contained in:
@@ -168,6 +168,37 @@ public class TestStreamReadMonitoringFunction {
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConsumeFromEarliestCommit() throws Exception {
|
||||
// write 2 commits first, then specify the start commit as 'earliest',
|
||||
// all the splits should come from the earliest commit.
|
||||
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);
|
||||
StreamReadMonitoringFunction function = TestUtils.getMonitorFunc(conf);
|
||||
try (AbstractStreamOperatorTestHarness<MergeOnReadInputSplit> harness = createHarness(function)) {
|
||||
harness.setup();
|
||||
harness.open();
|
||||
|
||||
CountDownLatch latch = new CountDownLatch(4);
|
||||
CollectingSourceContext sourceContext = new CollectingSourceContext(latch);
|
||||
|
||||
runAsync(sourceContext, function);
|
||||
|
||||
assertTrue(latch.await(WAIT_TIME_MILLIS, TimeUnit.MILLISECONDS), "Should finish splits generation");
|
||||
assertThat("Should produce the expected splits",
|
||||
sourceContext.getPartitionPaths(), is("par1,par2,par3,par4"));
|
||||
assertTrue(sourceContext.splits.stream().noneMatch(split -> split.getInstantRange().isPresent()),
|
||||
"No instants should have range limit");
|
||||
assertTrue(sourceContext.splits.stream().allMatch(split -> split.getLatestCommit().equals(specifiedCommit)),
|
||||
"All the splits should be with specified instant time");
|
||||
|
||||
// Stop the stream task.
|
||||
function.close();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckpointRestore() throws Exception {
|
||||
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
||||
|
||||
@@ -21,7 +21,6 @@ package org.apache.hudi.table;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
import org.apache.hudi.utils.TestConfigurations;
|
||||
import org.apache.hudi.utils.TestData;
|
||||
@@ -124,6 +123,13 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
|
||||
execInsertSql(streamTableEnv, insertInto);
|
||||
List<Row> rows2 = execSelectSql(streamTableEnv, "select * from t1", 10);
|
||||
assertRowsEquals(rows2, TestData.DATA_SET_SOURCE_INSERT);
|
||||
|
||||
streamTableEnv.getConfig().getConfiguration()
|
||||
.setBoolean("table.dynamic-table-options.enabled", true);
|
||||
// specify the start commit as earliest
|
||||
List<Row> rows3 = execSelectSql(streamTableEnv,
|
||||
"select * from t1/*+options('read.streaming.start-commit'='earliest')*/", 10);
|
||||
assertRowsEquals(rows3, TestData.DATA_SET_SOURCE_INSERT);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@@ -300,6 +306,34 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
|
||||
assertRowsEquals(result, expected, true);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("tableTypeAndPartitioningParams")
|
||||
void testStreamReadFilterByPartition(HoodieTableType tableType, boolean hiveStylePartitioning) throws Exception {
|
||||
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||
conf.setString(FlinkOptions.TABLE_NAME, "t1");
|
||||
conf.setString(FlinkOptions.TABLE_TYPE, tableType.name());
|
||||
conf.setBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING, hiveStylePartitioning);
|
||||
|
||||
// write one commit
|
||||
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
||||
|
||||
String hoodieTableDDL = sql("t1")
|
||||
.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_CHECK_INTERVAL, "2")
|
||||
.option(FlinkOptions.HIVE_STYLE_PARTITIONING, hiveStylePartitioning)
|
||||
.end();
|
||||
streamTableEnv.executeSql(hoodieTableDDL);
|
||||
|
||||
List<Row> result = execSelectSql(streamTableEnv,
|
||||
"select * from t1 where `partition`='par1'", 10);
|
||||
final String expected = "["
|
||||
+ "+I(+I[id1, Danny, 23, 1970-01-01T00:00:00.001, par1]), "
|
||||
+ "+I(+I[id2, Stephen, 33, 1970-01-01T00:00:00.002, par1])]";
|
||||
assertRowsEquals(result, expected, true);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("executionModeAndPartitioningParams")
|
||||
void testWriteAndRead(ExecMode execMode, boolean hiveStylePartitioning) {
|
||||
@@ -568,10 +602,7 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
|
||||
|
||||
@Test
|
||||
void testStreamReadEmptyTablePath() throws Exception {
|
||||
// create an empty table
|
||||
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||
StreamerUtil.initTableIfNotExists(conf);
|
||||
|
||||
// case1: table metadata path does not exists
|
||||
// create a flink source table
|
||||
String createHoodieTable = sql("t1")
|
||||
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
|
||||
@@ -580,10 +611,39 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
|
||||
.end();
|
||||
streamTableEnv.executeSql(createHoodieTable);
|
||||
|
||||
// execute query and assert throws exception
|
||||
assertThrows(HoodieException.class, () -> execSelectSql(streamTableEnv, "select * from t1", 10),
|
||||
"No successful commits under path " + tempFile.getAbsolutePath());
|
||||
// no exception expects to be thrown
|
||||
List<Row> rows1 = execSelectSql(streamTableEnv, "select * from t1", 10);
|
||||
assertRowsEquals(rows1, "[]");
|
||||
|
||||
// case2: empty table without data files
|
||||
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||
StreamerUtil.initTableIfNotExists(conf);
|
||||
|
||||
List<Row> rows2 = execSelectSql(streamTableEnv, "select * from t1", 10);
|
||||
assertRowsEquals(rows2, "[]");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testBatchReadEmptyTablePath() throws Exception {
|
||||
// case1: table metadata path does not exists
|
||||
// create a flink source table
|
||||
String createHoodieTable = sql("t1")
|
||||
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
|
||||
.option(FlinkOptions.TABLE_TYPE, FlinkOptions.TABLE_TYPE_MERGE_ON_READ)
|
||||
.end();
|
||||
batchTableEnv.executeSql(createHoodieTable);
|
||||
|
||||
// no exception expects to be thrown
|
||||
assertThrows(Exception.class,
|
||||
() -> execSelectSql(batchTableEnv, "select * from t1", 10),
|
||||
"Exception should throw when querying non-exists table in batch mode");
|
||||
|
||||
// case2: empty table without data files
|
||||
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||
StreamerUtil.initTableIfNotExists(conf);
|
||||
|
||||
List<Row> rows2 = CollectionUtil.iteratorToList(batchTableEnv.executeSql("select * from t1").collect());
|
||||
assertRowsEquals(rows2, "[]");
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@@ -781,6 +841,7 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
|
||||
|
||||
private List<Row> execSelectSql(TableEnvironment tEnv, String select, String sinkDDL, long timeout)
|
||||
throws InterruptedException {
|
||||
tEnv.executeSql("DROP TABLE IF EXISTS sink");
|
||||
tEnv.executeSql(sinkDDL);
|
||||
TableResult tableResult = tEnv.executeSql("insert into sink " + select);
|
||||
// wait for the timeout then cancels the job
|
||||
|
||||
@@ -25,12 +25,12 @@ import org.apache.hudi.util.StreamerUtil;
|
||||
import org.apache.hudi.utils.TestConfigurations;
|
||||
import org.apache.hudi.utils.TestData;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.flink.api.common.io.FileInputFormat;
|
||||
import org.apache.flink.api.common.io.InputFormat;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
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;
|
||||
import org.slf4j.Logger;
|
||||
@@ -43,12 +43,14 @@ import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
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.assertNotNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
|
||||
/**
|
||||
@@ -62,17 +64,17 @@ public class TestHoodieTableSource {
|
||||
@TempDir
|
||||
File tempFile;
|
||||
|
||||
@BeforeEach
|
||||
void beforeEach() throws IOException {
|
||||
final String path = tempFile.getAbsolutePath();
|
||||
conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||
conf = TestConfigurations.getDefaultConf(path);
|
||||
StreamerUtil.initTableIfNotExists(conf);
|
||||
IntStream.range(1, 5)
|
||||
.forEach(i -> new File(path + File.separator + "par" + i).mkdirs());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testGetReadPaths() {
|
||||
void testGetReadPaths() throws Exception {
|
||||
beforeEach();
|
||||
HoodieTableSource tableSource = new HoodieTableSource(
|
||||
TestConfigurations.TABLE_SCHEMA,
|
||||
new Path(tempFile.getPath()),
|
||||
@@ -99,6 +101,7 @@ public class TestHoodieTableSource {
|
||||
|
||||
@Test
|
||||
void testGetInputFormat() throws Exception {
|
||||
beforeEach();
|
||||
// write some data to let the TableSchemaResolver get the right instant
|
||||
TestData.writeData(TestData.DATA_SET_INSERT, conf);
|
||||
|
||||
@@ -118,4 +121,29 @@ public class TestHoodieTableSource {
|
||||
() -> tableSource.getInputFormat(),
|
||||
"Invalid query type : 'incremental'. Only 'snapshot' is supported now");
|
||||
}
|
||||
|
||||
@Test
|
||||
void testGetTableAvroSchema() {
|
||||
final String path = tempFile.getAbsolutePath();
|
||||
conf = TestConfigurations.getDefaultConf(path);
|
||||
conf.setBoolean(FlinkOptions.READ_AS_STREAMING, true);
|
||||
|
||||
HoodieTableSource tableSource = new HoodieTableSource(
|
||||
TestConfigurations.TABLE_SCHEMA,
|
||||
new Path(tempFile.getPath()),
|
||||
Arrays.asList(conf.getString(FlinkOptions.PARTITION_PATH_FIELD).split(",")),
|
||||
"default-par",
|
||||
conf);
|
||||
assertNull(tableSource.getMetaClient(), "Streaming source with empty table path is allowed");
|
||||
final String schemaFields = tableSource.getTableAvroSchema().getFields().stream()
|
||||
.map(Schema.Field::name)
|
||||
.collect(Collectors.joining(","));
|
||||
final String expected = "_hoodie_commit_time,"
|
||||
+ "_hoodie_commit_seqno,"
|
||||
+ "_hoodie_record_key,"
|
||||
+ "_hoodie_partition_path,"
|
||||
+ "_hoodie_file_name,"
|
||||
+ "uuid,name,age,ts,partition";
|
||||
assertThat(schemaFields, is(expected));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -28,6 +28,7 @@ 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;
|
||||
|
||||
@@ -57,8 +58,6 @@ public class TestUtils {
|
||||
|
||||
public static StreamReadMonitoringFunction getMonitorFunc(Configuration conf) {
|
||||
final String basePath = conf.getString(FlinkOptions.PATH);
|
||||
final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
|
||||
.setConf(StreamerUtil.getHadoopConf()).setBasePath(basePath).build();
|
||||
return new StreamReadMonitoringFunction(conf, new Path(basePath), metaClient, 1024 * 1024L);
|
||||
return new StreamReadMonitoringFunction(conf, new Path(basePath), 1024 * 1024L, Collections.emptySet());
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user