[HUDI-1663] Streaming read for Flink MOR table (#2640)
Supports two read modes: * Read the full data set starting from the latest commit instant and subsequent incremental data set * Read data set that starts from a specified commit instant
This commit is contained in:
@@ -20,15 +20,19 @@ package org.apache.hudi.operator.utils;
|
||||
|
||||
import org.apache.hudi.operator.FlinkOptions;
|
||||
import org.apache.hudi.streamer.FlinkStreamerConfig;
|
||||
import org.apache.hudi.utils.factory.CollectSinkTableFactory;
|
||||
import org.apache.hudi.utils.factory.ContinuousFileSourceFactory;
|
||||
|
||||
import org.apache.flink.api.common.ExecutionConfig;
|
||||
import org.apache.flink.api.common.typeinfo.TypeInformation;
|
||||
import org.apache.flink.api.common.typeinfo.Types;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.table.api.DataTypes;
|
||||
import org.apache.flink.table.api.TableSchema;
|
||||
import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
|
||||
import org.apache.flink.table.types.DataType;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
import org.apache.flink.types.Row;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
@@ -56,6 +60,13 @@ public class TestConfigurations {
|
||||
ROW_DATA_TYPE.getChildren().toArray(new DataType[0]))
|
||||
.build();
|
||||
|
||||
public static final TypeInformation<Row> ROW_TYPE_INFO = Types.ROW(
|
||||
Types.STRING,
|
||||
Types.STRING,
|
||||
Types.INT,
|
||||
Types.LOCAL_DATE_TIME,
|
||||
Types.STRING);
|
||||
|
||||
public static String getCreateHoodieTableDDL(String tableName, Map<String, String> options) {
|
||||
String createTable = "create table " + tableName + "(\n"
|
||||
+ " uuid varchar(20),\n"
|
||||
@@ -77,8 +88,12 @@ public class TestConfigurations {
|
||||
}
|
||||
|
||||
public static String getFileSourceDDL(String tableName) {
|
||||
return getFileSourceDDL(tableName, "test_source.data");
|
||||
}
|
||||
|
||||
public static String getFileSourceDDL(String tableName, String fileName) {
|
||||
String sourcePath = Objects.requireNonNull(Thread.currentThread()
|
||||
.getContextClassLoader().getResource("test_source.data")).toString();
|
||||
.getContextClassLoader().getResource(fileName)).toString();
|
||||
return "create table " + tableName + "(\n"
|
||||
+ " uuid varchar(20),\n"
|
||||
+ " name varchar(10),\n"
|
||||
@@ -91,6 +106,18 @@ public class TestConfigurations {
|
||||
+ ")";
|
||||
}
|
||||
|
||||
public static String getCollectSinkDDL(String tableName) {
|
||||
return "create table " + tableName + "(\n"
|
||||
+ " uuid varchar(20),\n"
|
||||
+ " name varchar(10),\n"
|
||||
+ " age int,\n"
|
||||
+ " ts timestamp(3),\n"
|
||||
+ " `partition` varchar(20)\n"
|
||||
+ ") with (\n"
|
||||
+ " 'connector' = '" + CollectSinkTableFactory.FACTORY_ID + "'"
|
||||
+ ")";
|
||||
}
|
||||
|
||||
public static final RowDataSerializer SERIALIZER = new RowDataSerializer(new ExecutionConfig(), ROW_TYPE);
|
||||
|
||||
public static Configuration getDefaultConf(String tablePath) {
|
||||
|
||||
@@ -43,6 +43,7 @@ import org.apache.flink.table.data.writer.BinaryWriter;
|
||||
import org.apache.flink.table.runtime.types.InternalSerializers;
|
||||
import org.apache.flink.table.types.logical.LogicalType;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
import org.apache.flink.types.Row;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.parquet.Strings;
|
||||
@@ -117,6 +118,52 @@ public class TestData {
|
||||
TimestampData.fromEpochMillis(1), StringData.fromString("par1"))));
|
||||
}
|
||||
|
||||
// data set of test_source.data
|
||||
public static List<RowData> DATA_SET_FOUR = Arrays.asList(
|
||||
binaryRow(StringData.fromString("id1"), StringData.fromString("Danny"), 23,
|
||||
TimestampData.fromEpochMillis(1000), StringData.fromString("par1")),
|
||||
binaryRow(StringData.fromString("id2"), StringData.fromString("Stephen"), 33,
|
||||
TimestampData.fromEpochMillis(2000), StringData.fromString("par1")),
|
||||
binaryRow(StringData.fromString("id3"), StringData.fromString("Julian"), 53,
|
||||
TimestampData.fromEpochMillis(3000), StringData.fromString("par2")),
|
||||
binaryRow(StringData.fromString("id4"), StringData.fromString("Fabian"), 31,
|
||||
TimestampData.fromEpochMillis(4000), StringData.fromString("par2")),
|
||||
binaryRow(StringData.fromString("id5"), StringData.fromString("Sophia"), 18,
|
||||
TimestampData.fromEpochMillis(5000), StringData.fromString("par3")),
|
||||
binaryRow(StringData.fromString("id6"), StringData.fromString("Emma"), 20,
|
||||
TimestampData.fromEpochMillis(6000), StringData.fromString("par3")),
|
||||
binaryRow(StringData.fromString("id7"), StringData.fromString("Bob"), 44,
|
||||
TimestampData.fromEpochMillis(7000), StringData.fromString("par4")),
|
||||
binaryRow(StringData.fromString("id8"), StringData.fromString("Han"), 56,
|
||||
TimestampData.fromEpochMillis(8000), StringData.fromString("par4"))
|
||||
);
|
||||
|
||||
// merged data set of test_source.data and test_source2.data
|
||||
public static List<RowData> DATA_SET_FIVE = Arrays.asList(
|
||||
binaryRow(StringData.fromString("id1"), StringData.fromString("Danny"), 24,
|
||||
TimestampData.fromEpochMillis(1000), StringData.fromString("par1")),
|
||||
binaryRow(StringData.fromString("id2"), StringData.fromString("Stephen"), 34,
|
||||
TimestampData.fromEpochMillis(2000), StringData.fromString("par1")),
|
||||
binaryRow(StringData.fromString("id3"), StringData.fromString("Julian"), 54,
|
||||
TimestampData.fromEpochMillis(3000), StringData.fromString("par2")),
|
||||
binaryRow(StringData.fromString("id4"), StringData.fromString("Fabian"), 32,
|
||||
TimestampData.fromEpochMillis(4000), StringData.fromString("par2")),
|
||||
binaryRow(StringData.fromString("id5"), StringData.fromString("Sophia"), 18,
|
||||
TimestampData.fromEpochMillis(5000), StringData.fromString("par3")),
|
||||
binaryRow(StringData.fromString("id6"), StringData.fromString("Emma"), 20,
|
||||
TimestampData.fromEpochMillis(6000), StringData.fromString("par3")),
|
||||
binaryRow(StringData.fromString("id7"), StringData.fromString("Bob"), 44,
|
||||
TimestampData.fromEpochMillis(7000), StringData.fromString("par4")),
|
||||
binaryRow(StringData.fromString("id8"), StringData.fromString("Han"), 56,
|
||||
TimestampData.fromEpochMillis(8000), StringData.fromString("par4")),
|
||||
binaryRow(StringData.fromString("id9"), StringData.fromString("Jane"), 19,
|
||||
TimestampData.fromEpochMillis(6000), StringData.fromString("par3")),
|
||||
binaryRow(StringData.fromString("id10"), StringData.fromString("Ella"), 38,
|
||||
TimestampData.fromEpochMillis(7000), StringData.fromString("par4")),
|
||||
binaryRow(StringData.fromString("id11"), StringData.fromString("Phoebe"), 52,
|
||||
TimestampData.fromEpochMillis(8000), StringData.fromString("par4"))
|
||||
);
|
||||
|
||||
/**
|
||||
* Returns string format of a list of RowData.
|
||||
*/
|
||||
@@ -159,24 +206,78 @@ public class TestData {
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks the source data TestConfigurations.DATA_SET_ONE are written as expected.
|
||||
* Sort the {@code rows} using field at index 0 and asserts
|
||||
* it equals with the expected string {@code expected}.
|
||||
*
|
||||
* @param rows Actual result rows
|
||||
* @param expected Expected string of the sorted rows
|
||||
*/
|
||||
public static void assertRowsEquals(List<Row> rows, String expected) {
|
||||
String rowsString = rows.stream()
|
||||
.sorted(Comparator.comparing(o -> o.getField(0).toString()))
|
||||
.collect(Collectors.toList()).toString();
|
||||
assertThat(rowsString, is(expected));
|
||||
}
|
||||
|
||||
/**
|
||||
* Sort the {@code rows} using field at index 0 and asserts
|
||||
* it equals with the expected row data list {@code expected}.
|
||||
*
|
||||
* @param rows Actual result rows
|
||||
* @param expected Expected row data list
|
||||
*/
|
||||
public static void assertRowsEquals(List<Row> rows, List<RowData> expected) {
|
||||
String rowsString = rows.stream()
|
||||
.sorted(Comparator.comparing(o -> o.getField(0).toString()))
|
||||
.collect(Collectors.toList()).toString();
|
||||
assertThat(rowsString, is(rowDataToString(expected)));
|
||||
}
|
||||
|
||||
/**
|
||||
* Sort the {@code rows} using field at index 0 and asserts
|
||||
* it equals with the expected string {@code expected}.
|
||||
*
|
||||
* @param rows Actual result rows
|
||||
* @param expected Expected string of the sorted rows
|
||||
*/
|
||||
public static void assertRowDataEquals(List<RowData> rows, String expected) {
|
||||
String rowsString = rowDataToString(rows);
|
||||
assertThat(rowsString, is(expected));
|
||||
}
|
||||
|
||||
/**
|
||||
* Sort the {@code rows} using field at index 0 and asserts
|
||||
* it equals with the expected row data list {@code expected}.
|
||||
*
|
||||
* @param rows Actual result rows
|
||||
* @param expected Expected row data list
|
||||
*/
|
||||
public static void assertRowDataEquals(List<RowData> rows, List<RowData> expected) {
|
||||
String rowsString = rowDataToString(rows);
|
||||
assertThat(rowsString, is(rowDataToString(expected)));
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks the source data set are written as expected.
|
||||
*
|
||||
* <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
|
||||
*/
|
||||
public static void checkWrittenData(File baseFile, Map<String, String> expected) throws IOException {
|
||||
checkWrittenData(baseFile, expected, 4);
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks the source data TestConfigurations.DATA_SET_ONE are written as expected.
|
||||
* Checks the source data set are written as expected.
|
||||
*
|
||||
* <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 checkWrittenData(
|
||||
|
||||
@@ -20,7 +20,11 @@ package org.apache.hudi.source;
|
||||
|
||||
import org.apache.hudi.operator.FlinkOptions;
|
||||
import org.apache.hudi.operator.utils.TestConfigurations;
|
||||
import org.apache.hudi.operator.utils.TestData;
|
||||
import org.apache.hudi.utils.TestUtils;
|
||||
import org.apache.hudi.utils.factory.CollectSinkTableFactory;
|
||||
|
||||
import org.apache.flink.core.execution.JobClient;
|
||||
import org.apache.flink.table.api.EnvironmentSettings;
|
||||
import org.apache.flink.table.api.TableEnvironment;
|
||||
import org.apache.flink.table.api.TableResult;
|
||||
@@ -34,16 +38,16 @@ import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.Comparator;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.apache.hudi.operator.utils.TestData.assertRowsEquals;
|
||||
|
||||
/**
|
||||
* IT cases for Hoodie table source and sink.
|
||||
@@ -72,6 +76,68 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
|
||||
@TempDir
|
||||
File tempFile;
|
||||
|
||||
@Test
|
||||
void testStreamWriteAndRead() throws Exception {
|
||||
// create filesystem table named source
|
||||
String createSource = TestConfigurations.getFileSourceDDL("source");
|
||||
streamTableEnv.executeSql(createSource);
|
||||
|
||||
Map<String, String> options = new HashMap<>();
|
||||
options.put(FlinkOptions.PATH.key(), tempFile.getAbsolutePath());
|
||||
options.put(FlinkOptions.READ_SCHEMA_FILE_PATH.key(),
|
||||
Objects.requireNonNull(Thread.currentThread()
|
||||
.getContextClassLoader().getResource("test_read_schema.avsc")).toString());
|
||||
options.put(FlinkOptions.READ_AS_STREAMING.key(), "true");
|
||||
options.put(FlinkOptions.TABLE_TYPE.key(), FlinkOptions.TABLE_TYPE_MERGE_ON_READ);
|
||||
String hoodieTableDDL = TestConfigurations.getCreateHoodieTableDDL("t1", options);
|
||||
streamTableEnv.executeSql(hoodieTableDDL);
|
||||
String insertInto = "insert into t1 select * from source";
|
||||
execInsertSql(streamTableEnv, insertInto);
|
||||
|
||||
List<Row> rows = execSelectSql(streamTableEnv, "select * from t1", 10);
|
||||
assertRowsEquals(rows, TestData.DATA_SET_FOUR);
|
||||
|
||||
// insert another batch of data
|
||||
execInsertSql(streamTableEnv, insertInto);
|
||||
List<Row> rows2 = execSelectSql(streamTableEnv, "select * from t1", 10);
|
||||
assertRowsEquals(rows2, TestData.DATA_SET_FOUR);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testStreamReadAppendData() throws Exception {
|
||||
// create filesystem table named source
|
||||
String createSource = TestConfigurations.getFileSourceDDL("source");
|
||||
String createSource2 = TestConfigurations.getFileSourceDDL("source2", "test_source2.data");
|
||||
streamTableEnv.executeSql(createSource);
|
||||
streamTableEnv.executeSql(createSource2);
|
||||
|
||||
Map<String, String> options = new HashMap<>();
|
||||
options.put(FlinkOptions.PATH.key(), tempFile.getAbsolutePath());
|
||||
options.put(FlinkOptions.READ_SCHEMA_FILE_PATH.key(),
|
||||
Objects.requireNonNull(Thread.currentThread()
|
||||
.getContextClassLoader().getResource("test_read_schema.avsc")).toString());
|
||||
options.put(FlinkOptions.READ_AS_STREAMING.key(), "true");
|
||||
options.put(FlinkOptions.TABLE_TYPE.key(), FlinkOptions.TABLE_TYPE_MERGE_ON_READ);
|
||||
String createHoodieTable = TestConfigurations.getCreateHoodieTableDDL("t1", options);
|
||||
streamTableEnv.executeSql(createHoodieTable);
|
||||
String insertInto = "insert into t1 select * from source";
|
||||
// execute 2 times
|
||||
execInsertSql(streamTableEnv, insertInto);
|
||||
// remember the commit
|
||||
String specifiedCommit = TestUtils.getFirstCommit(tempFile.getAbsolutePath());
|
||||
// another update batch
|
||||
String insertInto2 = "insert into t1 select * from source2";
|
||||
execInsertSql(streamTableEnv, insertInto2);
|
||||
// now we consume starting from the oldest commit
|
||||
options.put(FlinkOptions.READ_STREAMING_START_COMMIT.key(), specifiedCommit);
|
||||
String createHoodieTable2 = TestConfigurations.getCreateHoodieTableDDL("t2", options);
|
||||
streamTableEnv.executeSql(createHoodieTable2);
|
||||
List<Row> rows = execSelectSql(streamTableEnv, "select * from t2", 10);
|
||||
// all the data with same keys are appended within one data bucket and one log file,
|
||||
// so when consume, the same keys are merged
|
||||
assertRowsEquals(rows, TestData.DATA_SET_FIVE);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testStreamWriteBatchRead() {
|
||||
// create filesystem table named source
|
||||
@@ -90,15 +156,7 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
|
||||
|
||||
List<Row> rows = CollectionUtil.iterableToList(
|
||||
() -> streamTableEnv.sqlQuery("select * from t1").execute().collect());
|
||||
final String expected = "[id1,Danny,23,1970-01-01T00:00:01,par1, "
|
||||
+ "id2,Stephen,33,1970-01-01T00:00:02,par1, "
|
||||
+ "id3,Julian,53,1970-01-01T00:00:03,par2, "
|
||||
+ "id4,Fabian,31,1970-01-01T00:00:04,par2, "
|
||||
+ "id5,Sophia,18,1970-01-01T00:00:05,par3, "
|
||||
+ "id6,Emma,20,1970-01-01T00:00:06,par3, "
|
||||
+ "id7,Bob,44,1970-01-01T00:00:07,par4, "
|
||||
+ "id8,Han,56,1970-01-01T00:00:08,par4]";
|
||||
assertRowsEquals(rows, expected);
|
||||
assertRowsEquals(rows, TestData.DATA_SET_FOUR);
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -124,29 +182,7 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
|
||||
|
||||
List<Row> rows = CollectionUtil.iterableToList(
|
||||
() -> batchTableEnv.sqlQuery("select * from t1").execute().collect());
|
||||
final String expected = "[id1,Danny,23,1970-01-01T00:00:01,par1, "
|
||||
+ "id2,Stephen,33,1970-01-01T00:00:02,par1, "
|
||||
+ "id3,Julian,53,1970-01-01T00:00:03,par2, "
|
||||
+ "id4,Fabian,31,1970-01-01T00:00:04,par2, "
|
||||
+ "id5,Sophia,18,1970-01-01T00:00:05,par3, "
|
||||
+ "id6,Emma,20,1970-01-01T00:00:06,par3, "
|
||||
+ "id7,Bob,44,1970-01-01T00:00:07,par4, "
|
||||
+ "id8,Han,56,1970-01-01T00:00:08,par4]";
|
||||
assertRowsEquals(rows, expected);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sort the {@code rows} using field at index 0 and asserts
|
||||
* it equals with the expected string {@code expected}.
|
||||
*
|
||||
* @param rows Actual result rows
|
||||
* @param expected Expected string of the sorted rows
|
||||
*/
|
||||
private static void assertRowsEquals(List<Row> rows, String expected) {
|
||||
String rowsString = rows.stream()
|
||||
.sorted(Comparator.comparing(o -> o.getField(0).toString()))
|
||||
.collect(Collectors.toList()).toString();
|
||||
assertThat(rowsString, is(expected));
|
||||
assertRowsEquals(rows, TestData.DATA_SET_FOUR);
|
||||
}
|
||||
|
||||
private void execInsertSql(TableEnvironment tEnv, String insert) {
|
||||
@@ -159,4 +195,16 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
|
||||
throw new RuntimeException(ex);
|
||||
}
|
||||
}
|
||||
|
||||
private List<Row> execSelectSql(TableEnvironment tEnv, String select, long timeout) throws InterruptedException {
|
||||
tEnv.executeSql(TestConfigurations.getCollectSinkDDL("sink"));
|
||||
TableResult tableResult = tEnv.executeSql("insert into sink " + select);
|
||||
// wait for the timeout then cancels the job
|
||||
TimeUnit.SECONDS.sleep(timeout);
|
||||
tableResult.getJobClient().ifPresent(JobClient::cancel);
|
||||
tEnv.executeSql("DROP TABLE IF EXISTS sink");
|
||||
return CollectSinkTableFactory.RESULT.values().stream()
|
||||
.flatMap(Collection::stream)
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -54,8 +54,8 @@ import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
/**
|
||||
* Test cases for HoodieTableSource.
|
||||
*/
|
||||
public class HoodieTableSourceTest {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(HoodieTableSourceTest.class);
|
||||
public class TestHoodieTableSource {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(TestHoodieTableSource.class);
|
||||
|
||||
private Configuration conf;
|
||||
|
||||
@@ -0,0 +1,269 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.source;
|
||||
|
||||
import org.apache.hudi.operator.FlinkOptions;
|
||||
import org.apache.hudi.operator.StreamReadMonitoringFunction;
|
||||
import org.apache.hudi.operator.utils.TestConfigurations;
|
||||
import org.apache.hudi.operator.utils.TestData;
|
||||
import org.apache.hudi.source.format.mor.MergeOnReadInputSplit;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
import org.apache.hudi.utils.TestUtils;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
|
||||
import org.apache.flink.streaming.api.functions.source.SourceFunction;
|
||||
import org.apache.flink.streaming.api.operators.StreamSource;
|
||||
import org.apache.flink.streaming.api.watermark.Watermark;
|
||||
import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
* Test cases for {@link StreamReadMonitoringFunction}.
|
||||
*/
|
||||
public class TestStreamReadMonitoringFunction {
|
||||
private static final long WAIT_TIME_MILLIS = 5 * 1000L;
|
||||
|
||||
private Configuration conf;
|
||||
|
||||
@TempDir
|
||||
File tempFile;
|
||||
|
||||
@BeforeEach
|
||||
public void before() throws Exception {
|
||||
final String basePath = tempFile.getAbsolutePath();
|
||||
conf = TestConfigurations.getDefaultConf(basePath);
|
||||
conf.setString(FlinkOptions.TABLE_TYPE, FlinkOptions.TABLE_TYPE_MERGE_ON_READ);
|
||||
conf.setInteger(FlinkOptions.READ_STREAMING_CHECK_INTERVAL, 2); // check every 2 seconds
|
||||
|
||||
StreamerUtil.initTableIfNotExists(conf);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConsumeFromLatestCommit() throws Exception {
|
||||
TestData.writeData(TestData.DATA_SET_ONE, conf);
|
||||
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");
|
||||
|
||||
Thread.sleep(1000L);
|
||||
|
||||
// reset the source context
|
||||
latch = new CountDownLatch(4);
|
||||
sourceContext.reset(latch);
|
||||
|
||||
// write another instant and validate
|
||||
TestData.writeData(TestData.DATA_SET_TWO, conf);
|
||||
|
||||
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().allMatch(split -> split.getInstantRange().isPresent()),
|
||||
"All the instants should have range limit");
|
||||
|
||||
// Stop the stream task.
|
||||
function.close();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConsumeFromSpecifiedCommit() throws Exception {
|
||||
// write 2 commits first, use the second commit time as the specified start instant,
|
||||
// all the splits should come from the second commit.
|
||||
TestData.writeData(TestData.DATA_SET_ONE, conf);
|
||||
TestData.writeData(TestData.DATA_SET_TWO, conf);
|
||||
String specifiedCommit = TestUtils.getLatestCommit(tempFile.getAbsolutePath());
|
||||
conf.setString(FlinkOptions.READ_STREAMING_START_COMMIT, specifiedCommit);
|
||||
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().allMatch(split -> split.getInstantRange().isPresent()),
|
||||
"All the 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_ONE, conf);
|
||||
|
||||
StreamReadMonitoringFunction function = TestUtils.getMonitorFunc(conf);
|
||||
OperatorSubtaskState state;
|
||||
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");
|
||||
Thread.sleep(1000L);
|
||||
|
||||
state = harness.snapshot(1, 1);
|
||||
|
||||
// Stop the stream task.
|
||||
function.close();
|
||||
|
||||
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");
|
||||
|
||||
}
|
||||
|
||||
TestData.writeData(TestData.DATA_SET_TWO, conf);
|
||||
StreamReadMonitoringFunction function2 = TestUtils.getMonitorFunc(conf);
|
||||
try (AbstractStreamOperatorTestHarness<MergeOnReadInputSplit> harness = createHarness(function2)) {
|
||||
harness.setup();
|
||||
// Recover to process the remaining snapshots.
|
||||
harness.initializeState(state);
|
||||
harness.open();
|
||||
|
||||
CountDownLatch latch = new CountDownLatch(4);
|
||||
CollectingSourceContext sourceContext = new CollectingSourceContext(latch);
|
||||
runAsync(sourceContext, function2);
|
||||
|
||||
// Stop the stream task.
|
||||
function.close();
|
||||
|
||||
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().allMatch(split -> split.getInstantRange().isPresent()),
|
||||
"All the instants should have range limit");
|
||||
}
|
||||
}
|
||||
|
||||
private AbstractStreamOperatorTestHarness<MergeOnReadInputSplit> createHarness(
|
||||
StreamReadMonitoringFunction function) throws Exception {
|
||||
StreamSource<MergeOnReadInputSplit, StreamReadMonitoringFunction> streamSource = new StreamSource<>(function);
|
||||
return new AbstractStreamOperatorTestHarness<>(streamSource, 1, 1, 0);
|
||||
}
|
||||
|
||||
private void runAsync(
|
||||
CollectingSourceContext sourceContext,
|
||||
StreamReadMonitoringFunction function) {
|
||||
Thread task = new Thread(() -> {
|
||||
try {
|
||||
function.run(sourceContext);
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
});
|
||||
task.start();
|
||||
}
|
||||
|
||||
/**
|
||||
* Source context that collects the outputs in to a list.
|
||||
*/
|
||||
private static class CollectingSourceContext implements SourceFunction.SourceContext<MergeOnReadInputSplit> {
|
||||
private final List<MergeOnReadInputSplit> splits = new ArrayList<>();
|
||||
private final Object checkpointLock = new Object();
|
||||
private volatile CountDownLatch latch;
|
||||
|
||||
CollectingSourceContext(CountDownLatch latch) {
|
||||
this.latch = latch;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collect(MergeOnReadInputSplit element) {
|
||||
splits.add(element);
|
||||
latch.countDown();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collectWithTimestamp(MergeOnReadInputSplit element, long timestamp) {
|
||||
collect(element);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void emitWatermark(Watermark mark) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void markAsTemporarilyIdle() {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getCheckpointLock() {
|
||||
return checkpointLock;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
|
||||
}
|
||||
|
||||
public void reset(CountDownLatch latch) {
|
||||
this.latch = latch;
|
||||
this.splits.clear();
|
||||
}
|
||||
|
||||
public String getPartitionPaths() {
|
||||
return this.splits.stream()
|
||||
.map(TestUtils::getSplitPartitionPath)
|
||||
.sorted(Comparator.naturalOrder())
|
||||
.collect(Collectors.joining(","));
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,290 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.source;
|
||||
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.TableSchemaResolver;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.operator.FlinkOptions;
|
||||
import org.apache.hudi.operator.StreamReadMonitoringFunction;
|
||||
import org.apache.hudi.operator.StreamReadOperator;
|
||||
import org.apache.hudi.operator.utils.TestConfigurations;
|
||||
import org.apache.hudi.operator.utils.TestData;
|
||||
import org.apache.hudi.source.format.FilePathUtils;
|
||||
import org.apache.hudi.source.format.mor.MergeOnReadInputFormat;
|
||||
import org.apache.hudi.source.format.mor.MergeOnReadInputSplit;
|
||||
import org.apache.hudi.source.format.mor.MergeOnReadTableState;
|
||||
import org.apache.hudi.util.AvroSchemaConverter;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
import org.apache.hudi.utils.TestUtils;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.core.fs.Path;
|
||||
import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
|
||||
import org.apache.flink.streaming.api.TimeCharacteristic;
|
||||
import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
|
||||
import org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor;
|
||||
import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction;
|
||||
import org.apache.flink.streaming.runtime.tasks.mailbox.SteppingMailboxProcessor;
|
||||
import org.apache.flink.streaming.util.CollectingSourceContext;
|
||||
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.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
* Test cases for {@link StreamReadOperator}.
|
||||
*/
|
||||
public class TestStreamReadOperator {
|
||||
private static final Map<String, String> EXPECTED = new HashMap<>();
|
||||
static {
|
||||
EXPECTED.put("par1", "id1,Danny,23,1970-01-01T00:00:00.001,par1, id2,Stephen,33,1970-01-01T00:00:00.002,par1");
|
||||
EXPECTED.put("par2", "id3,Julian,53,1970-01-01T00:00:00.003,par2, id4,Fabian,31,1970-01-01T00:00:00.004,par2");
|
||||
EXPECTED.put("par3", "id5,Sophia,18,1970-01-01T00:00:00.005,par3, id6,Emma,20,1970-01-01T00:00:00.006,par3");
|
||||
EXPECTED.put("par4", "id7,Bob,44,1970-01-01T00:00:00.007,par4, id8,Han,56,1970-01-01T00:00:00.008,par4");
|
||||
}
|
||||
|
||||
private Configuration conf;
|
||||
|
||||
@TempDir
|
||||
File tempFile;
|
||||
|
||||
@BeforeEach
|
||||
public void before() throws Exception {
|
||||
final String basePath = tempFile.getAbsolutePath();
|
||||
conf = TestConfigurations.getDefaultConf(basePath);
|
||||
conf.setString(FlinkOptions.TABLE_TYPE, FlinkOptions.TABLE_TYPE_MERGE_ON_READ);
|
||||
|
||||
StreamerUtil.initTableIfNotExists(conf);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testWriteRecords() throws Exception {
|
||||
TestData.writeData(TestData.DATA_SET_ONE, conf);
|
||||
try (OneInputStreamOperatorTestHarness<MergeOnReadInputSplit, RowData> harness = createReader()) {
|
||||
harness.setup();
|
||||
harness.open();
|
||||
|
||||
SteppingMailboxProcessor processor = createLocalMailbox(harness);
|
||||
StreamReadMonitoringFunction func = TestUtils.getMonitorFunc(conf);
|
||||
|
||||
List<MergeOnReadInputSplit> splits = generateSplits(func);
|
||||
assertThat("Should have 4 splits", splits.size(), is(4));
|
||||
for (MergeOnReadInputSplit split : splits) {
|
||||
// Process this element to enqueue to mail-box.
|
||||
harness.processElement(split, -1);
|
||||
|
||||
// Run the mail-box once to read all records from the given split.
|
||||
assertThat("Should process 1 split", processor.runMailboxStep());
|
||||
}
|
||||
// Assert the output has expected elements.
|
||||
TestData.assertRowDataEquals(harness.extractOutputValues(), TestData.DATA_SET_ONE);
|
||||
|
||||
TestData.writeData(TestData.DATA_SET_TWO, conf);
|
||||
final List<MergeOnReadInputSplit> splits2 = generateSplits(func);
|
||||
assertThat("Should have 4 splits", splits2.size(), is(4));
|
||||
for (MergeOnReadInputSplit split : splits2) {
|
||||
// Process this element to enqueue to mail-box.
|
||||
harness.processElement(split, -1);
|
||||
|
||||
// Run the mail-box once to read all records from the given split.
|
||||
assertThat("Should processed 1 split", processor.runMailboxStep());
|
||||
}
|
||||
// The result sets behaves like append only: DATA_SET_ONE + DATA_SET_TWO
|
||||
List<RowData> expected = new ArrayList<>(TestData.DATA_SET_ONE);
|
||||
expected.addAll(TestData.DATA_SET_TWO);
|
||||
TestData.assertRowDataEquals(harness.extractOutputValues(), expected);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckpoint() throws Exception {
|
||||
// Received emitted splits: split1, split2, split3, split4, checkpoint request is triggered
|
||||
// when reading records from split1.
|
||||
TestData.writeData(TestData.DATA_SET_ONE, conf);
|
||||
long timestamp = 0;
|
||||
try (OneInputStreamOperatorTestHarness<MergeOnReadInputSplit, RowData> harness = createReader()) {
|
||||
harness.setup();
|
||||
harness.open();
|
||||
|
||||
SteppingMailboxProcessor processor = createLocalMailbox(harness);
|
||||
StreamReadMonitoringFunction func = TestUtils.getMonitorFunc(conf);
|
||||
|
||||
List<MergeOnReadInputSplit> splits = generateSplits(func);
|
||||
assertThat("Should have 4 splits", splits.size(), is(4));
|
||||
|
||||
for (MergeOnReadInputSplit split : splits) {
|
||||
harness.processElement(split, ++timestamp);
|
||||
}
|
||||
|
||||
// Trigger snapshot state, it will start to work once all records from split0 are read.
|
||||
processor.getMainMailboxExecutor()
|
||||
.execute(() -> harness.snapshot(1, 3), "Trigger snapshot");
|
||||
|
||||
assertTrue(processor.runMailboxStep(), "Should have processed the split0");
|
||||
assertTrue(processor.runMailboxStep(), "Should have processed the snapshot state action");
|
||||
|
||||
assertThat(TestData.rowDataToString(harness.extractOutputValues()),
|
||||
is(getSplitExpected(Collections.singletonList(splits.get(0)), EXPECTED)));
|
||||
|
||||
// Read records from split1.
|
||||
assertTrue(processor.runMailboxStep(), "Should have processed the split1");
|
||||
|
||||
// Read records from split2.
|
||||
assertTrue(processor.runMailboxStep(), "Should have processed the split2");
|
||||
|
||||
// Read records from split3.
|
||||
assertTrue(processor.runMailboxStep(), "Should have processed the split3");
|
||||
|
||||
// Assert the output has expected elements.
|
||||
TestData.assertRowDataEquals(harness.extractOutputValues(), TestData.DATA_SET_ONE);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckpointRestore() throws Exception {
|
||||
TestData.writeData(TestData.DATA_SET_ONE, conf);
|
||||
|
||||
OperatorSubtaskState state;
|
||||
final List<MergeOnReadInputSplit> splits;
|
||||
try (OneInputStreamOperatorTestHarness<MergeOnReadInputSplit, RowData> harness = createReader()) {
|
||||
harness.setup();
|
||||
harness.open();
|
||||
|
||||
StreamReadMonitoringFunction func = TestUtils.getMonitorFunc(conf);
|
||||
|
||||
splits = generateSplits(func);
|
||||
assertThat("Should have 4 splits", splits.size(), is(4));
|
||||
|
||||
// Enqueue all the splits.
|
||||
for (MergeOnReadInputSplit split : splits) {
|
||||
harness.processElement(split, -1);
|
||||
}
|
||||
|
||||
// Read all records from the first 2 splits.
|
||||
SteppingMailboxProcessor localMailbox = createLocalMailbox(harness);
|
||||
for (int i = 0; i < 2; i++) {
|
||||
assertTrue(localMailbox.runMailboxStep(), "Should have processed the split#" + i);
|
||||
}
|
||||
|
||||
assertThat(TestData.rowDataToString(harness.extractOutputValues()),
|
||||
is(getSplitExpected(splits.subList(0, 2), EXPECTED)));
|
||||
|
||||
// Snapshot state now, there are 2 splits left in the state.
|
||||
state = harness.snapshot(1, 1);
|
||||
}
|
||||
|
||||
try (OneInputStreamOperatorTestHarness<MergeOnReadInputSplit, RowData> harness = createReader()) {
|
||||
harness.setup();
|
||||
// Recover to process the remaining splits.
|
||||
harness.initializeState(state);
|
||||
harness.open();
|
||||
|
||||
SteppingMailboxProcessor localMailbox = createLocalMailbox(harness);
|
||||
|
||||
for (int i = 2; i < 4; i++) {
|
||||
assertTrue(localMailbox.runMailboxStep(), "Should have processed one split#" + i);
|
||||
}
|
||||
|
||||
// expect to output the left data
|
||||
assertThat(TestData.rowDataToString(harness.extractOutputValues()),
|
||||
is(getSplitExpected(splits.subList(2, 4), EXPECTED)));
|
||||
}
|
||||
}
|
||||
|
||||
private static String getSplitExpected(List<MergeOnReadInputSplit> splits, Map<String, String> expected) {
|
||||
return splits.stream()
|
||||
.map(TestUtils::getSplitPartitionPath)
|
||||
.map(expected::get)
|
||||
.sorted(Comparator.naturalOrder())
|
||||
.collect(Collectors.toList()).toString();
|
||||
}
|
||||
|
||||
private List<MergeOnReadInputSplit> generateSplits(StreamReadMonitoringFunction func) throws Exception {
|
||||
final List<MergeOnReadInputSplit> splits = new ArrayList<>();
|
||||
func.open(conf);
|
||||
func.monitorDirAndForwardSplits(new CollectingSourceContext<>(new Object(), splits));
|
||||
return splits;
|
||||
}
|
||||
|
||||
private OneInputStreamOperatorTestHarness<MergeOnReadInputSplit, RowData> createReader() throws Exception {
|
||||
final String basePath = tempFile.getAbsolutePath();
|
||||
final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
|
||||
.setConf(StreamerUtil.getHadoopConf()).setBasePath(basePath).build();
|
||||
final List<String> partitionKeys = Collections.singletonList("partition");
|
||||
|
||||
// This input format is used to opening the emitted split.
|
||||
TableSchemaResolver schemaUtil = new TableSchemaResolver(metaClient);
|
||||
final Schema tableAvroSchema;
|
||||
try {
|
||||
tableAvroSchema = schemaUtil.getTableAvroSchema();
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException("Get table avro schema error", e);
|
||||
}
|
||||
final DataType rowDataType = AvroSchemaConverter.convertToDataType(tableAvroSchema);
|
||||
final RowType rowType = (RowType) rowDataType.getLogicalType();
|
||||
final MergeOnReadTableState hoodieTableState = new MergeOnReadTableState(
|
||||
rowType,
|
||||
TestConfigurations.ROW_TYPE,
|
||||
tableAvroSchema.toString(),
|
||||
AvroSchemaConverter.convertToSchema(TestConfigurations.ROW_TYPE).toString(),
|
||||
Collections.emptyList());
|
||||
Path[] paths = FilePathUtils.getReadPaths(
|
||||
new Path(basePath), conf, partitionKeys, conf.getString(FlinkOptions.PARTITION_DEFAULT_NAME));
|
||||
MergeOnReadInputFormat inputFormat = new MergeOnReadInputFormat(
|
||||
conf,
|
||||
paths,
|
||||
hoodieTableState,
|
||||
rowDataType.getChildren(),
|
||||
"default",
|
||||
1000L);
|
||||
|
||||
OneInputStreamOperatorFactory<MergeOnReadInputSplit, RowData> factory = StreamReadOperator.factory(inputFormat);
|
||||
OneInputStreamOperatorTestHarness<MergeOnReadInputSplit, RowData> harness = new OneInputStreamOperatorTestHarness<>(
|
||||
factory, 1, 1, 0);
|
||||
harness.getStreamConfig().setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
|
||||
|
||||
return harness;
|
||||
}
|
||||
|
||||
private SteppingMailboxProcessor createLocalMailbox(
|
||||
OneInputStreamOperatorTestHarness<MergeOnReadInputSplit, RowData> harness) {
|
||||
return new SteppingMailboxProcessor(
|
||||
MailboxDefaultAction.Controller::suspendDefaultAction,
|
||||
harness.getTaskMailbox(),
|
||||
StreamTaskActionExecutor.IMMEDIATE);
|
||||
}
|
||||
}
|
||||
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.source.format;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.operator.FlinkOptions;
|
||||
import org.apache.hudi.operator.utils.TestConfigurations;
|
||||
import org.apache.hudi.operator.utils.TestData;
|
||||
@@ -32,7 +33,7 @@ import org.apache.flink.table.data.RowData;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
import org.junit.jupiter.params.provider.EnumSource;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
@@ -48,7 +49,7 @@ import static org.hamcrest.MatcherAssert.assertThat;
|
||||
/**
|
||||
* Test cases for MergeOnReadInputFormat and ParquetInputFormat.
|
||||
*/
|
||||
public class InputFormatTest {
|
||||
public class TestInputFormat {
|
||||
|
||||
private HoodieTableSource tableSource;
|
||||
private Configuration conf;
|
||||
@@ -56,9 +57,9 @@ public class InputFormatTest {
|
||||
@TempDir
|
||||
File tempFile;
|
||||
|
||||
void beforeEach(String tableType) throws IOException {
|
||||
void beforeEach(HoodieTableType tableType) throws IOException {
|
||||
conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||
conf.setString(FlinkOptions.TABLE_TYPE, tableType);
|
||||
conf.setString(FlinkOptions.TABLE_TYPE, tableType.name());
|
||||
conf.setBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED, false); // close the async compaction
|
||||
|
||||
StreamerUtil.initTableIfNotExists(conf);
|
||||
@@ -71,10 +72,8 @@ public class InputFormatTest {
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {
|
||||
FlinkOptions.TABLE_TYPE_COPY_ON_WRITE,
|
||||
FlinkOptions.TABLE_TYPE_MERGE_ON_READ})
|
||||
void testRead(String tableType) throws Exception {
|
||||
@EnumSource(value = HoodieTableType.class)
|
||||
void testRead(HoodieTableType tableType) throws Exception {
|
||||
beforeEach(tableType);
|
||||
|
||||
TestData.writeData(TestData.DATA_SET_ONE, conf);
|
||||
@@ -113,7 +112,7 @@ public class InputFormatTest {
|
||||
|
||||
@Test
|
||||
void testReadBaseAndLogFiles() throws Exception {
|
||||
beforeEach(FlinkOptions.TABLE_TYPE_MERGE_ON_READ);
|
||||
beforeEach(HoodieTableType.MERGE_ON_READ);
|
||||
|
||||
// write parquet first with compaction
|
||||
conf.setBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED, true);
|
||||
@@ -153,10 +152,8 @@ public class InputFormatTest {
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(strings = {
|
||||
FlinkOptions.TABLE_TYPE_COPY_ON_WRITE,
|
||||
FlinkOptions.TABLE_TYPE_MERGE_ON_READ})
|
||||
void testReadWithPartitionPrune(String tableType) throws Exception {
|
||||
@EnumSource(value = HoodieTableType.class)
|
||||
void testReadWithPartitionPrune(HoodieTableType tableType) throws Exception {
|
||||
beforeEach(tableType);
|
||||
|
||||
TestData.writeData(TestData.DATA_SET_ONE, conf);
|
||||
@@ -0,0 +1,64 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.utils;
|
||||
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.operator.FlinkOptions;
|
||||
import org.apache.hudi.operator.StreamReadMonitoringFunction;
|
||||
import org.apache.hudi.source.format.mor.MergeOnReadInputSplit;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.core.fs.Path;
|
||||
|
||||
import java.io.File;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
* Common test utils.
|
||||
*/
|
||||
public class TestUtils {
|
||||
|
||||
public static String getLatestCommit(String basePath) {
|
||||
final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
|
||||
.setConf(StreamerUtil.getHadoopConf()).setBasePath(basePath).build();
|
||||
return metaClient.getCommitsAndCompactionTimeline().filterCompletedInstants().lastInstant().get().getTimestamp();
|
||||
}
|
||||
|
||||
public static String getFirstCommit(String basePath) {
|
||||
final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
|
||||
.setConf(StreamerUtil.getHadoopConf()).setBasePath(basePath).build();
|
||||
return metaClient.getCommitsAndCompactionTimeline().filterCompletedInstants().firstInstant().get().getTimestamp();
|
||||
}
|
||||
|
||||
public static String getSplitPartitionPath(MergeOnReadInputSplit split) {
|
||||
assertTrue(split.getLogPaths().isPresent());
|
||||
final String logPath = split.getLogPaths().get().get(0);
|
||||
String[] paths = logPath.split(File.separator);
|
||||
return paths[paths.length - 2];
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,174 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.utils.factory;
|
||||
|
||||
import org.apache.hudi.operator.utils.TestConfigurations;
|
||||
|
||||
import org.apache.flink.api.common.state.ListState;
|
||||
import org.apache.flink.api.common.state.ListStateDescriptor;
|
||||
import org.apache.flink.configuration.ConfigOption;
|
||||
import org.apache.flink.runtime.state.FunctionInitializationContext;
|
||||
import org.apache.flink.runtime.state.FunctionSnapshotContext;
|
||||
import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
|
||||
import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
|
||||
import org.apache.flink.streaming.api.functions.sink.SinkFunction;
|
||||
import org.apache.flink.table.api.TableSchema;
|
||||
import org.apache.flink.table.connector.ChangelogMode;
|
||||
import org.apache.flink.table.connector.sink.DynamicTableSink;
|
||||
import org.apache.flink.table.connector.sink.SinkFunctionProvider;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.table.factories.DynamicTableSinkFactory;
|
||||
import org.apache.flink.table.factories.FactoryUtil;
|
||||
import org.apache.flink.types.Row;
|
||||
import org.apache.flink.types.RowKind;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Factory for CollectTableSink.
|
||||
*
|
||||
* <p>Note: The CollectTableSink collects all the data of a table into a global collection {@code RESULT},
|
||||
* so the tests should executed in single thread and the table name should be the same.
|
||||
*/
|
||||
public class CollectSinkTableFactory implements DynamicTableSinkFactory {
|
||||
public static final String FACTORY_ID = "collect";
|
||||
|
||||
// global results to collect and query
|
||||
public static final Map<Integer, List<Row>> RESULT = new HashMap<>();
|
||||
|
||||
@Override
|
||||
public DynamicTableSink createDynamicTableSink(Context context) {
|
||||
FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context);
|
||||
helper.validate();
|
||||
|
||||
TableSchema schema = context.getCatalogTable().getSchema();
|
||||
RESULT.clear();
|
||||
return new CollectTableSink(schema, context.getObjectIdentifier().getObjectName());
|
||||
}
|
||||
|
||||
@Override
|
||||
public String factoryIdentifier() {
|
||||
return FACTORY_ID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<ConfigOption<?>> requiredOptions() {
|
||||
return Collections.emptySet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<ConfigOption<?>> optionalOptions() {
|
||||
return Collections.emptySet();
|
||||
}
|
||||
|
||||
// --------------------------------------------------------------------------------------------
|
||||
// Table sinks
|
||||
// --------------------------------------------------------------------------------------------
|
||||
|
||||
/**
|
||||
* Values {@link DynamicTableSink} for testing.
|
||||
*/
|
||||
private static class CollectTableSink implements DynamicTableSink {
|
||||
|
||||
private final TableSchema schema;
|
||||
private final String tableName;
|
||||
|
||||
private CollectTableSink(
|
||||
TableSchema schema,
|
||||
String tableName) {
|
||||
this.schema = schema;
|
||||
this.tableName = tableName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
|
||||
return ChangelogMode.insertOnly();
|
||||
}
|
||||
|
||||
@Override
|
||||
public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
|
||||
DataStructureConverter converter = context.createDataStructureConverter(schema.toPhysicalRowDataType());
|
||||
return SinkFunctionProvider.of(new CollectSinkFunction(converter));
|
||||
}
|
||||
|
||||
@Override
|
||||
public DynamicTableSink copy() {
|
||||
return new CollectTableSink(schema, tableName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String asSummaryString() {
|
||||
return "CollectSink";
|
||||
}
|
||||
}
|
||||
|
||||
static class CollectSinkFunction extends RichSinkFunction<RowData> implements CheckpointedFunction {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
private final DynamicTableSink.DataStructureConverter converter;
|
||||
|
||||
protected transient ListState<Row> resultState;
|
||||
protected transient List<Row> localResult;
|
||||
|
||||
private int taskID;
|
||||
|
||||
protected CollectSinkFunction(DynamicTableSink.DataStructureConverter converter) {
|
||||
this.converter = converter;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void invoke(RowData value, SinkFunction.Context context) {
|
||||
if (value.getRowKind() == RowKind.INSERT) {
|
||||
Row row = (Row) converter.toExternal(value);
|
||||
assert row != null;
|
||||
RESULT.get(taskID).add(row);
|
||||
} else {
|
||||
throw new RuntimeException(
|
||||
"CollectSinkFunction received " + value.getRowKind() + " messages.");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initializeState(FunctionInitializationContext context) throws Exception {
|
||||
this.resultState = context.getOperatorStateStore().getListState(
|
||||
new ListStateDescriptor<>("sink-results", TestConfigurations.ROW_TYPE_INFO));
|
||||
this.localResult = new ArrayList<>();
|
||||
if (context.isRestored()) {
|
||||
for (Row value : resultState.get()) {
|
||||
localResult.add(value);
|
||||
}
|
||||
}
|
||||
this.taskID = getRuntimeContext().getIndexOfThisSubtask();
|
||||
synchronized (CollectSinkTableFactory.class) {
|
||||
RESULT.put(taskID, localResult);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void snapshotState(FunctionSnapshotContext context) throws Exception {
|
||||
resultState.clear();
|
||||
resultState.addAll(RESULT.get(taskID));
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,17 @@
|
||||
# Licensed to the Apache Software Foundation (ASF) under one
|
||||
# or more contributor license agreements. See the NOTICE file
|
||||
# distributed with this work for additional information
|
||||
# regarding copyright ownership. The ASF licenses this file
|
||||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS,
|
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
org.apache.hudi.utils.factory.CollectSinkTableFactory
|
||||
8
hudi-flink/src/test/resources/test_source2.data
Normal file
8
hudi-flink/src/test/resources/test_source2.data
Normal file
@@ -0,0 +1,8 @@
|
||||
{"uuid": "id1", "name": "Danny", "age": 24, "ts": "1970-01-01T00:00:01", "partition": "par1"}
|
||||
{"uuid": "id2", "name": "Stephen", "age": 34, "ts": "1970-01-01T00:00:02", "partition": "par1"}
|
||||
{"uuid": "id3", "name": "Julian", "age": 54, "ts": "1970-01-01T00:00:03", "partition": "par2"}
|
||||
{"uuid": "id4", "name": "Fabian", "age": 32, "ts": "1970-01-01T00:00:04", "partition": "par2"}
|
||||
{"uuid": "id5", "name": "Sophia", "age": 18, "ts": "1970-01-01T00:00:05", "partition": "par3"}
|
||||
{"uuid": "id9", "name": "Jane", "age": 19, "ts": "1970-01-01T00:00:06", "partition": "par3"}
|
||||
{"uuid": "id10", "name": "Ella", "age": 38, "ts": "1970-01-01T00:00:07", "partition": "par4"}
|
||||
{"uuid": "id11", "name": "Phoebe", "age": 52, "ts": "1970-01-01T00:00:08", "partition": "par4"}
|
||||
Reference in New Issue
Block a user