1
0

[HUDI-2412] Add timestamp based partitioning for flink writer (#3638)

This commit is contained in:
Danny Chan
2021-09-11 13:17:16 +08:00
committed by GitHub
parent 06240417e9
commit b30c5bdaef
12 changed files with 355 additions and 65 deletions

View File

@@ -20,9 +20,11 @@ package org.apache.hudi.sink.bulk;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.exception.HoodieKeyException;
import org.apache.hudi.table.HoodieTableFactory;
import org.apache.hudi.utils.TestConfigurations;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.data.StringData;
import org.apache.flink.table.data.TimestampData;
@@ -93,4 +95,35 @@ public class TestRowDataKeyGen {
assertThat(keyGen2.getPartitionPath(rowData2), is("partition=default/ts=default"));
assertThat(keyGen2.getPartitionPath(rowData3), is("partition=default/ts=1970-01-01T00:00:00.001"));
}
@Test
void testTimestampBasedKeyGenerator() {
Configuration conf = TestConfigurations.getDefaultConf("path1");
conf.setString(FlinkOptions.PARTITION_PATH_FIELD, "ts");
HoodieTableFactory.setupTimestampKeygenOptions(conf, DataTypes.TIMESTAMP(3));
final RowData rowData1 = insertRow(StringData.fromString("id1"), StringData.fromString("Danny"), 23,
TimestampData.fromEpochMillis(7200000), StringData.fromString("par1"));
final RowDataKeyGen keyGen1 = RowDataKeyGen.instance(conf, TestConfigurations.ROW_TYPE);
assertThat(keyGen1.getRecordKey(rowData1), is("id1"));
assertThat(keyGen1.getPartitionPath(rowData1), is("1970010102"));
// null record key and partition path
final RowData rowData2 = insertRow(TestConfigurations.ROW_TYPE, null, StringData.fromString("Danny"), 23,
null, StringData.fromString("par1"));
assertThrows(HoodieKeyException.class, () -> keyGen1.getRecordKey(rowData2));
assertThat(keyGen1.getPartitionPath(rowData2), is("1970010100"));
// empty record key and partition path
final RowData rowData3 = insertRow(StringData.fromString(""), StringData.fromString("Danny"), 23,
TimestampData.fromEpochMillis(1), StringData.fromString("par1"));
assertThrows(HoodieKeyException.class, () -> keyGen1.getRecordKey(rowData3));
assertThat(keyGen1.getPartitionPath(rowData3), is("1970010100"));
// hive style partitioning
conf.set(FlinkOptions.HIVE_STYLE_PARTITIONING, true);
final RowDataKeyGen keyGen2 = RowDataKeyGen.instance(conf, TestConfigurations.ROW_TYPE);
assertThat(keyGen2.getPartitionPath(rowData1), is("ts=1970010102"));
assertThat(keyGen2.getPartitionPath(rowData2), is("ts=1970010100"));
assertThat(keyGen2.getPartitionPath(rowData3), is("ts=1970010100"));
}
}

View File

@@ -20,12 +20,14 @@ package org.apache.hudi.source;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.keygen.NonpartitionedAvroKeyGenerator;
import org.apache.hudi.utils.TestConfigurations;
import org.apache.hudi.utils.TestData;
import org.apache.flink.configuration.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
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;
@@ -69,4 +71,21 @@ public class TestFileIndex {
assertTrue(Arrays.stream(fileStatuses)
.allMatch(fileStatus -> fileStatus.getPath().toString().endsWith(HoodieFileFormat.PARQUET.getFileExtension())));
}
@Test
void testFileListingUsingMetadataNonPartitionedTable() throws Exception {
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
conf.setString(FlinkOptions.PARTITION_PATH_FIELD, "");
conf.setString(FlinkOptions.KEYGEN_CLASS_NAME, NonpartitionedAvroKeyGenerator.class.getName());
conf.setBoolean(FlinkOptions.METADATA_ENABLED, true);
TestData.writeData(TestData.DATA_SET_INSERT, conf);
FileIndex fileIndex = FileIndex.instance(new Path(tempFile.getAbsolutePath()), conf);
List<String> partitionKeys = Collections.singletonList("");
List<Map<String, String>> partitions = fileIndex.getPartitions(partitionKeys, "default", false);
assertThat(partitions.size(), is(0));
FileStatus[] fileStatuses = fileIndex.getFilesInPartitions();
assertThat(fileStatuses.length, is(1));
assertTrue(fileStatuses[0].getPath().toString().endsWith(HoodieFileFormat.PARQUET.getFileExtension()));
}
}

View File

@@ -100,8 +100,8 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.READ_AS_STREAMING, "true")
.option(FlinkOptions.TABLE_TYPE, tableType.name())
.option(FlinkOptions.READ_AS_STREAMING, true)
.option(FlinkOptions.TABLE_TYPE, tableType)
.end();
streamTableEnv.executeSql(hoodieTableDDL);
String insertInto = "insert into t1 select * from source";
@@ -111,8 +111,8 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
streamTableEnv.executeSql("drop table t1");
hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.READ_AS_STREAMING, "true")
.option(FlinkOptions.TABLE_TYPE, tableType.name())
.option(FlinkOptions.READ_AS_STREAMING, true)
.option(FlinkOptions.TABLE_TYPE, tableType)
.option(FlinkOptions.READ_STREAMING_START_COMMIT, firstCommit)
.end();
streamTableEnv.executeSql(hoodieTableDDL);
@@ -141,14 +141,14 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.READ_AS_STREAMING, "true")
.option(FlinkOptions.TABLE_TYPE, tableType.name())
.option(FlinkOptions.READ_AS_STREAMING, true)
.option(FlinkOptions.TABLE_TYPE, tableType)
.end();
streamTableEnv.executeSql(hoodieTableDDL);
String insertInto = "insert into t1 select * from source";
execInsertSql(streamTableEnv, insertInto);
// reading from latest commit instance.
// reading from the latest commit instance.
List<Row> rows = execSelectSql(streamTableEnv, "select * from t1", 10);
assertRowsEquals(rows, TestData.DATA_SET_SOURCE_INSERT_LATEST_COMMIT);
@@ -169,8 +169,8 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
String createHoodieTable = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.READ_AS_STREAMING, "true")
.option(FlinkOptions.TABLE_TYPE, tableType.name())
.option(FlinkOptions.READ_AS_STREAMING, true)
.option(FlinkOptions.TABLE_TYPE, tableType)
.end();
streamTableEnv.executeSql(createHoodieTable);
String insertInto = "insert into t1 select * from source";
@@ -184,8 +184,8 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
// now we consume starting from the oldest commit
String createHoodieTable2 = sql("t2")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.READ_AS_STREAMING, "true")
.option(FlinkOptions.TABLE_TYPE, tableType.name())
.option(FlinkOptions.READ_AS_STREAMING, true)
.option(FlinkOptions.TABLE_TYPE, tableType)
.option(FlinkOptions.READ_STREAMING_START_COMMIT, specifiedCommit)
.end();
streamTableEnv.executeSql(createHoodieTable2);
@@ -226,8 +226,8 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
// test MOR streaming write with compaction then reads as
// query type 'read_optimized'.
.option(FlinkOptions.QUERY_TYPE, FlinkOptions.QUERY_TYPE_READ_OPTIMIZED)
.option(FlinkOptions.COMPACTION_DELTA_COMMITS, "1")
.option(FlinkOptions.COMPACTION_TASKS, "1")
.option(FlinkOptions.COMPACTION_DELTA_COMMITS, 1)
.option(FlinkOptions.COMPACTION_TASKS, 1)
.end();
streamTableEnv.executeSql(hoodieTableDDL);
String insertInto = "insert into t1 select * from source";
@@ -250,7 +250,7 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.CLEAN_RETAIN_COMMITS, "1")
.option(FlinkOptions.CLEAN_RETAIN_COMMITS, 1)
.end();
streamTableEnv.executeSql(hoodieTableDDL);
String insertInto = "insert into t1 select * from source";
@@ -287,10 +287,10 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
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.READ_AS_STREAMING, true)
.option(FlinkOptions.READ_STREAMING_CHECK_INTERVAL, 2)
.option(FlinkOptions.READ_STREAMING_START_COMMIT, latestCommit)
.option(FlinkOptions.CHANGELOG_ENABLED, "true")
.option(FlinkOptions.CHANGELOG_ENABLED, true)
.end();
streamTableEnv.executeSql(hoodieTableDDL);
@@ -319,9 +319,9 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.TABLE_TYPE, tableType.name())
.option(FlinkOptions.READ_AS_STREAMING, "true")
.option(FlinkOptions.READ_STREAMING_CHECK_INTERVAL, "2")
.option(FlinkOptions.TABLE_TYPE, tableType)
.option(FlinkOptions.READ_AS_STREAMING, true)
.option(FlinkOptions.READ_STREAMING_CHECK_INTERVAL, 2)
.option(FlinkOptions.HIVE_STYLE_PARTITIONING, hiveStylePartitioning)
.end();
streamTableEnv.executeSql(hoodieTableDDL);
@@ -342,13 +342,13 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
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_AS_STREAMING, true)
.option(FlinkOptions.READ_STREAMING_START_COMMIT, FlinkOptions.START_COMMIT_EARLIEST)
.option(FlinkOptions.READ_STREAMING_CHECK_INTERVAL, "2")
.option(FlinkOptions.READ_STREAMING_CHECK_INTERVAL, 2)
// close the async compaction
.option(FlinkOptions.COMPACTION_ASYNC_ENABLED, false)
// generate compaction plan for each commit
.option(FlinkOptions.COMPACTION_DELTA_COMMITS, "1")
.option(FlinkOptions.COMPACTION_DELTA_COMMITS, 1)
.withPartition(false)
.end();
streamTableEnv.executeSql(hoodieTableDDL);
@@ -422,7 +422,7 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
TableEnvironment tableEnv = batchTableEnv;
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.TABLE_NAME, tableType.name())
.option(FlinkOptions.TABLE_NAME, tableType)
.option(FlinkOptions.HIVE_STYLE_PARTITIONING, hiveStylePartitioning)
.end();
tableEnv.executeSql(hoodieTableDDL);
@@ -557,11 +557,12 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
}
@ParameterizedTest
@EnumSource(value = ExecMode.class)
void testWriteNonPartitionedTable(ExecMode execMode) {
@MethodSource("executionModeAndTableTypeParams")
void testWriteNonPartitionedTable(ExecMode execMode, HoodieTableType tableType) {
TableEnvironment tableEnv = execMode == ExecMode.BATCH ? batchTableEnv : streamTableEnv;
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.TABLE_TYPE, tableType)
.withPartition(false)
.end();
tableEnv.executeSql(hoodieTableDDL);
@@ -593,8 +594,8 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.INDEX_GLOBAL_ENABLED, "true")
.option(FlinkOptions.INSERT_DROP_DUPS, "true")
.option(FlinkOptions.INDEX_GLOBAL_ENABLED, true)
.option(FlinkOptions.INSERT_DROP_DUPS, true)
.end();
streamTableEnv.executeSql(hoodieTableDDL);
@@ -615,8 +616,8 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
streamTableEnv.executeSql(createSource);
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.INDEX_GLOBAL_ENABLED, "false")
.option(FlinkOptions.INSERT_DROP_DUPS, "true")
.option(FlinkOptions.INDEX_GLOBAL_ENABLED, false)
.option(FlinkOptions.INSERT_DROP_DUPS, true)
.end();
streamTableEnv.executeSql(hoodieTableDDL);
@@ -743,7 +744,7 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
String hoodieTableDDL = sql("hoodie_sink")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.option(FlinkOptions.OPERATION, "bulk_insert")
.option(FlinkOptions.WRITE_BULK_INSERT_SHUFFLE_BY_PARTITION, "true")
.option(FlinkOptions.WRITE_BULK_INSERT_SHUFFLE_BY_PARTITION, true)
.option(FlinkOptions.HIVE_STYLE_PARTITIONING, hiveStylePartitioning)
.end();
tableEnv.executeSql(hoodieTableDDL);
@@ -825,6 +826,34 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
+ "+I[id8, Han, 56, 1970-01-01T00:00:08, par4]]");
}
@ParameterizedTest
@EnumSource(value = ExecMode.class)
void testWriteAndReadWithTimestampPartitioning(ExecMode execMode) {
// can not read the hive style and timestamp based partitioning table
// in batch mode, the code path in CopyOnWriteInputFormat relies on
// the value on the partition path to recover the partition value,
// but the date format has changed(milliseconds switch to hours).
TableEnvironment tableEnv = execMode == ExecMode.BATCH ? batchTableEnv : streamTableEnv;
String hoodieTableDDL = sql("t1")
.option(FlinkOptions.PATH, tempFile.getAbsolutePath())
.partitionField("ts") // use timestamp as partition path field
.end();
tableEnv.executeSql(hoodieTableDDL);
execInsertSql(tableEnv, TestSQL.INSERT_T1);
List<Row> result1 = CollectionUtil.iterableToList(
() -> tableEnv.sqlQuery("select * from t1").execute().collect());
assertRowsEquals(result1, TestData.DATA_SET_SOURCE_INSERT);
// apply filters
List<Row> result2 = CollectionUtil.iterableToList(
() -> tableEnv.sqlQuery("select * from t1 where uuid > 'id5'").execute().collect());
assertRowsEquals(result2, "["
+ "+I[id6, Emma, 20, 1970-01-01T00:00:06, par3], "
+ "+I[id7, Bob, 44, 1970-01-01T00:00:07, par4], "
+ "+I[id8, Han, 56, 1970-01-01T00:00:08, par4]]");
}
// -------------------------------------------------------------------------
// Utilities
// -------------------------------------------------------------------------
@@ -832,6 +861,19 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
BATCH, STREAM
}
/**
* Return test params => (execution mode, table type).
*/
private static Stream<Arguments> executionModeAndTableTypeParams() {
Object[][] data =
new Object[][] {
{ExecMode.BATCH, HoodieTableType.MERGE_ON_READ},
{ExecMode.BATCH, HoodieTableType.COPY_ON_WRITE},
{ExecMode.STREAM, HoodieTableType.MERGE_ON_READ},
{ExecMode.STREAM, HoodieTableType.COPY_ON_WRITE}};
return Stream.of(data).map(Arguments::of);
}
/**
* Return test params => (execution mode, hive style partitioning).
*/

View File

@@ -23,6 +23,7 @@ import org.apache.hudi.hive.MultiPartKeysValueExtractor;
import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor;
import org.apache.hudi.keygen.ComplexAvroKeyGenerator;
import org.apache.hudi.keygen.NonpartitionedAvroKeyGenerator;
import org.apache.hudi.keygen.TimestampBasedAvroKeyGenerator;
import org.apache.hudi.util.StreamerUtil;
import org.apache.hudi.utils.SchemaBuilder;
import org.apache.hudi.utils.TestConfigurations;
@@ -142,7 +143,7 @@ public class TestHoodieTableFactory {
ResolvedSchema schema1 = SchemaBuilder.instance()
.field("f0", DataTypes.INT().notNull())
.field("f1", DataTypes.VARCHAR(20))
.field("f2", DataTypes.TIMESTAMP(3))
.field("f2", DataTypes.BIGINT())
.field("ts", DataTypes.TIMESTAMP(3))
.primaryKey("f0")
.build();
@@ -254,7 +255,7 @@ public class TestHoodieTableFactory {
ResolvedSchema schema1 = SchemaBuilder.instance()
.field("f0", DataTypes.INT().notNull())
.field("f1", DataTypes.VARCHAR(20))
.field("f2", DataTypes.TIMESTAMP(3))
.field("f2", DataTypes.BIGINT())
.field("ts", DataTypes.TIMESTAMP(3))
.primaryKey("f0")
.build();
@@ -342,6 +343,31 @@ public class TestHoodieTableFactory {
assertThat(conf2.getInteger(FlinkOptions.ARCHIVE_MAX_COMMITS), is(45));
}
@Test
void testSetupTimestampBasedKeyGenForSink() {
this.conf.setString(FlinkOptions.RECORD_KEY_FIELD, "dummyField");
this.conf.setString(FlinkOptions.KEYGEN_CLASS_NAME, "dummyKeyGenClass");
// 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();
final MockContext sourceContext1 = MockContext.getInstance(this.conf, schema1, "ts");
final HoodieTableSource tableSource1 = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext1);
final Configuration conf1 = tableSource1.getConf();
assertThat(conf1.get(FlinkOptions.RECORD_KEY_FIELD), is("f0"));
assertThat(conf1.get(FlinkOptions.KEYGEN_CLASS_NAME), is(TimestampBasedAvroKeyGenerator.class.getName()));
assertThat(conf1.getString(TimestampBasedAvroKeyGenerator.Config.TIMESTAMP_TYPE_FIELD_PROP, "dummy"),
is("EPOCHMILLISECONDS"));
assertThat(conf1.getString(TimestampBasedAvroKeyGenerator.Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP, "dummy"),
is(FlinkOptions.PARTITION_FORMAT_HOUR));
assertThat(conf1.getString(TimestampBasedAvroKeyGenerator.Config.TIMESTAMP_OUTPUT_TIMEZONE_FORMAT_PROP, "dummy"),
is("UTC"));
}
// -------------------------------------------------------------------------
// Inner Class
// -------------------------------------------------------------------------

View File

@@ -58,10 +58,14 @@ public class TestConfigurations {
.build();
public static String getCreateHoodieTableDDL(String tableName, Map<String, String> options) {
return getCreateHoodieTableDDL(tableName, options, true);
return getCreateHoodieTableDDL(tableName, options, true, "partition");
}
public static String getCreateHoodieTableDDL(String tableName, Map<String, String> options, boolean havePartition) {
public static String getCreateHoodieTableDDL(
String tableName,
Map<String, String> options,
boolean havePartition,
String partitionField) {
StringBuilder builder = new StringBuilder();
builder.append("create table " + tableName + "(\n"
+ " uuid varchar(20),\n"
@@ -72,7 +76,7 @@ public class TestConfigurations {
+ " PRIMARY KEY(uuid) NOT ENFORCED\n"
+ ")\n");
if (havePartition) {
builder.append("PARTITIONED BY (`partition`)\n");
builder.append("PARTITIONED BY (`").append(partitionField).append("`)\n");
}
builder.append("with (\n"
+ " 'connector' = 'hudi'");
@@ -203,19 +207,15 @@ public class TestConfigurations {
private final Map<String, String> options;
private String tableName;
private boolean withPartition = true;
private String partitionField = "partition";
public Sql(String tableName) {
options = new HashMap<>();
this.tableName = tableName;
}
public Sql option(ConfigOption<?> option, String val) {
this.options.put(option.key(), val);
return this;
}
public Sql option(ConfigOption<?> option, boolean val) {
this.options.put(option.key(), val + "");
public Sql option(ConfigOption<?> option, Object val) {
this.options.put(option.key(), val.toString());
return this;
}
@@ -224,8 +224,13 @@ public class TestConfigurations {
return this;
}
public Sql partitionField(String partitionField) {
this.partitionField = partitionField;
return this;
}
public String end() {
return TestConfigurations.getCreateHoodieTableDDL(this.tableName, options, this.withPartition);
return TestConfigurations.getCreateHoodieTableDDL(this.tableName, options, this.withPartition, this.partitionField);
}
}
}