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

@@ -140,22 +140,22 @@ public class HoodieJavaApp {
// 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())
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY(), "false")
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY().key(), "false")
// This will remove any existing data at path below, and create a
.mode(SaveMode.Overwrite);
@@ -174,16 +174,16 @@ public class HoodieJavaApp {
Dataset<Row> inputDF2 = spark.read().json(jssc.parallelize(records2, 2));
writer = inputDF2.write().format("org.apache.hudi").option("hoodie.insert.shuffle.parallelism", "2")
.option("hoodie.upsert.shuffle.parallelism", "2")
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY(), tableType) // Hoodie Table Type
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key")
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition")
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "timestamp")
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(),
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY().key(), tableType) // Hoodie Table Type
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY().key(), "_row_key")
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY().key(), "partition")
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY().key(), "timestamp")
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY().key(),
nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName()
: SimpleKeyGenerator.class.getCanonicalName()) // Add Key Extractor
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, "1")
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY(), "false")
.option(HoodieWriteConfig.TABLE_NAME, tableName).mode(SaveMode.Append);
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP.key(), "1")
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY().key(), "false")
.option(HoodieWriteConfig.TABLE_NAME.key(), tableName).mode(SaveMode.Append);
updateHiveSyncConfig(writer);
writer.save(tablePath);
@@ -200,17 +200,17 @@ public class HoodieJavaApp {
writer = inputDF3.write().format("org.apache.hudi").option("hoodie.insert.shuffle.parallelism", "2")
.option("hoodie.upsert.shuffle.parallelism", "2")
.option("hoodie.delete.shuffle.parallelism", "2")
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY(), tableType) // Hoodie Table Type
.option(DataSourceWriteOptions.OPERATION_OPT_KEY(), "delete")
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key")
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition")
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "_row_key")
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(),
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY().key(), tableType) // Hoodie Table Type
.option(DataSourceWriteOptions.OPERATION_OPT_KEY().key(), "delete")
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY().key(), "_row_key")
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY().key(), "partition")
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY().key(), "_row_key")
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY().key(),
nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName()
: SimpleKeyGenerator.class.getCanonicalName()) // Add Key Extractor
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, "1")
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY(), "false")
.option(HoodieWriteConfig.TABLE_NAME, tableName).mode(SaveMode.Append);
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP.key(), "1")
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY().key(), "false")
.option(HoodieWriteConfig.TABLE_NAME.key(), tableName).mode(SaveMode.Append);
updateHiveSyncConfig(writer);
writer.save(tablePath);
@@ -234,9 +234,9 @@ public class HoodieJavaApp {
* Consume incrementally, only changes in commit 2 above. Currently only supported for COPY_ON_WRITE TABLE
*/
Dataset<Row> incQueryDF = spark.read().format("org.apache.hudi")
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY(), DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL())
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY().key(), DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL())
// Only changes in write 2 above
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY(), commitInstantTime1)
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY().key(), commitInstantTime1)
// For incremental view, pass in the root/base path of dataset
.load(tablePath);
@@ -251,23 +251,23 @@ public class HoodieJavaApp {
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;

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);

View File

@@ -332,9 +332,9 @@ public class HoodieJavaStreamingApp {
* Consume incrementally, only changes in commit 2 above. Currently only supported for COPY_ON_WRITE TABLE
*/
Dataset<Row> hoodieIncViewDF = spark.read().format("hudi")
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY(), DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL())
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY().key(), DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL())
// Only changes in write 2 above
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY(), commitInstantTime1)
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY().key(), commitInstantTime1)
// For incremental view, pass in the root/base path of dataset
.load(tablePath);
@@ -355,14 +355,14 @@ public class HoodieJavaStreamingApp {
DataStreamWriter<Row> writer = streamingInput.writeStream().format("org.apache.hudi")
.option("hoodie.insert.shuffle.parallelism", "2").option("hoodie.upsert.shuffle.parallelism", "2")
.option("hoodie.delete.shuffle.parallelism", "2")
.option(DataSourceWriteOptions.OPERATION_OPT_KEY(), operationType)
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY(), tableType)
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key")
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition")
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "timestamp")
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, "1")
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY(), "true")
.option(HoodieWriteConfig.TABLE_NAME, tableName).option("checkpointLocation", checkpointLocation)
.option(DataSourceWriteOptions.OPERATION_OPT_KEY().key(), operationType)
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY().key(), tableType)
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY().key(), "_row_key")
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY().key(), "partition")
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY().key(), "timestamp")
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP.key(), "1")
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY().key(), "true")
.option(HoodieWriteConfig.TABLE_NAME.key(), tableName).option("checkpointLocation", checkpointLocation)
.outputMode(OutputMode.Append());
updateHiveSyncConfig(writer);
@@ -379,18 +379,18 @@ public class HoodieJavaStreamingApp {
private DataStreamWriter<Row> updateHiveSyncConfig(DataStreamWriter<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 (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;

View File

@@ -94,18 +94,18 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
private Map<String, String> getProps(boolean setAll, boolean setKeyGen, boolean setRecordKey, boolean setPartitionPath) {
Map<String, String> props = new HashMap<>();
if (setAll) {
props.put(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(), "org.apache.hudi.keygen.SimpleKeyGenerator");
props.put(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key");
props.put(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition");
props.put(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY().key(), "org.apache.hudi.keygen.SimpleKeyGenerator");
props.put(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY().key(), "_row_key");
props.put(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY().key(), "partition");
} else {
if (setKeyGen) {
props.put(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(), "org.apache.hudi.keygen.SimpleKeyGenerator");
props.put(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY().key(), "org.apache.hudi.keygen.SimpleKeyGenerator");
}
if (setRecordKey) {
props.put(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key");
props.put(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY().key(), "_row_key");
}
if (setPartitionPath) {
props.put(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition");
props.put(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY().key(), "partition");
}
}
return props;

View File

@@ -371,7 +371,7 @@ public class TestBootstrap extends HoodieClientTestBase {
List<GenericRecord> records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
jsc.hadoopConfiguration(),
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream()
HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream()
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>());
assertEquals(totalRecords, records.size());
@@ -390,7 +390,7 @@ public class TestBootstrap extends HoodieClientTestBase {
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
jsc.hadoopConfiguration(),
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream()
HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream()
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>());
assertEquals(totalRecords, records.size());
@@ -407,7 +407,7 @@ public class TestBootstrap extends HoodieClientTestBase {
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
jsc.hadoopConfiguration(),
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream()
HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream()
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES,
true, HoodieRecord.HOODIE_META_COLUMNS);
@@ -425,7 +425,7 @@ public class TestBootstrap extends HoodieClientTestBase {
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
jsc.hadoopConfiguration(),
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream()
HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream()
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true,
HoodieRecord.HOODIE_META_COLUMNS);
@@ -441,7 +441,7 @@ public class TestBootstrap extends HoodieClientTestBase {
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
jsc.hadoopConfiguration(),
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream()
HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream()
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, true,
Arrays.asList("_row_key"));
@@ -459,7 +459,7 @@ public class TestBootstrap extends HoodieClientTestBase {
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
jsc.hadoopConfiguration(),
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream()
HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream()
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true,
Arrays.asList("_row_key"));
@@ -550,8 +550,8 @@ public class TestBootstrap extends HoodieClientTestBase {
HoodieWriteConfig.Builder builder = getConfigBuilder(schemaStr, IndexType.BLOOM)
.withExternalSchemaTrasformation(true);
TypedProperties properties = new TypedProperties();
properties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key");
properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "datestr");
properties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY().key(), "_row_key");
properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY().key(), "datestr");
builder = builder.withProps(properties);
return builder;
}