1
0

[HUDI-89] Add configOption & refactor all configs based on that (#2833)

Co-authored-by: Wenning Ding <wenningd@amazon.com>
This commit is contained in:
wenningd
2021-06-30 14:26:30 -07:00
committed by GitHub
parent 07e93de8b4
commit d412fb2fe6
173 changed files with 4277 additions and 3309 deletions

View File

@@ -124,23 +124,23 @@ public class HoodieJavaGenerateApp {
private DataFrameWriter<Row> updateHiveSyncConfig(DataFrameWriter<Row> writer) {
if (enableHiveSync) {
LOG.info("Enabling Hive sync to " + hiveJdbcUrl);
writer = writer.option(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY(), hiveTable)
.option(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY(), hiveDB)
.option(DataSourceWriteOptions.HIVE_URL_OPT_KEY(), hiveJdbcUrl)
.option(DataSourceWriteOptions.HIVE_USER_OPT_KEY(), hiveUser)
.option(DataSourceWriteOptions.HIVE_PASS_OPT_KEY(), hivePass)
.option(DataSourceWriteOptions.HIVE_SYNC_ENABLED_OPT_KEY(), "true");
writer = writer.option(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY().key(), hiveTable)
.option(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY().key(), hiveDB)
.option(DataSourceWriteOptions.HIVE_URL_OPT_KEY().key(), hiveJdbcUrl)
.option(DataSourceWriteOptions.HIVE_USER_OPT_KEY().key(), hiveUser)
.option(DataSourceWriteOptions.HIVE_PASS_OPT_KEY().key(), hivePass)
.option(DataSourceWriteOptions.HIVE_SYNC_ENABLED_OPT_KEY().key(), "true");
if (nonPartitionedTable) {
writer = writer
.option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY(),
.option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY().key(),
NonPartitionedExtractor.class.getCanonicalName())
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "");
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY().key(), "");
} else if (useMultiPartitionKeys) {
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), "year,month,day").option(
DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY(),
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY().key(), "year,month,day").option(
DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY().key(),
MultiPartKeysValueExtractor.class.getCanonicalName());
} else {
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), "dateStr");
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY().key(), "dateStr");
}
}
return writer;
@@ -165,19 +165,19 @@ public class HoodieJavaGenerateApp {
// full list in HoodieWriteConfig & its package
.option("hoodie.upsert.shuffle.parallelism", "2")
// Hoodie Table Type
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY(), tableType)
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY().key(), tableType)
// insert
.option(DataSourceWriteOptions.OPERATION_OPT_KEY(), DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL())
.option(DataSourceWriteOptions.OPERATION_OPT_KEY().key(), DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL())
// This is the record key
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key")
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY().key(), "_row_key")
// this is the partition to place it into
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition")
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY().key(), "partition")
// use to combine duplicate records in input/with disk val
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "timestamp")
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY().key(), "timestamp")
// Used by hive sync and queries
.option(HoodieWriteConfig.TABLE_NAME, tableName)
.option(HoodieWriteConfig.TABLE_NAME.key(), tableName)
// Add Key Extractor
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(),
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY().key(),
nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName()
: SimpleKeyGenerator.class.getCanonicalName())
.mode(commitType);