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

@@ -104,7 +104,7 @@ public class SparkRecentDaysClusteringPlanStrategy<T extends HoodieRecordPayload
protected Map<String, String> getStrategyParams() {
Map<String, String> params = new HashMap<>();
if (!StringUtils.isNullOrEmpty(getWriteConfig().getClusteringSortColumns())) {
params.put(CLUSTERING_SORT_COLUMNS_PROPERTY, getWriteConfig().getClusteringSortColumns());
params.put(CLUSTERING_SORT_COLUMNS_PROPERTY.key(), getWriteConfig().getClusteringSortColumns());
}
return params;
}

View File

@@ -69,10 +69,10 @@ public class SparkSortAndSizeExecutionStrategy<T extends HoodieRecordPayload<T>>
final String instantTime, final Map<String, String> strategyParams, final Schema schema) {
LOG.info("Starting clustering for a group, parallelism:" + numOutputGroups + " commit:" + instantTime);
Properties props = getWriteConfig().getProps();
props.put(HoodieWriteConfig.BULKINSERT_PARALLELISM, String.valueOf(numOutputGroups));
props.put(HoodieWriteConfig.BULKINSERT_PARALLELISM.key(), String.valueOf(numOutputGroups));
// We are calling another action executor - disable auto commit. Strategy is only expected to write data in new files.
props.put(HoodieWriteConfig.HOODIE_AUTO_COMMIT_PROP, Boolean.FALSE.toString());
props.put(HoodieStorageConfig.PARQUET_FILE_MAX_BYTES, String.valueOf(getWriteConfig().getClusteringTargetFileMaxBytes()));
props.put(HoodieWriteConfig.HOODIE_AUTO_COMMIT_PROP.key(), Boolean.FALSE.toString());
props.put(HoodieStorageConfig.PARQUET_FILE_MAX_BYTES.key(), String.valueOf(getWriteConfig().getClusteringTargetFileMaxBytes()));
HoodieWriteConfig newConfig = HoodieWriteConfig.newBuilder().withProps(props).build();
return (JavaRDD<WriteStatus>) SparkBulkInsertHelper.newInstance().bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig,
false, getPartitioner(strategyParams, schema), true, numOutputGroups);
@@ -82,8 +82,8 @@ public class SparkSortAndSizeExecutionStrategy<T extends HoodieRecordPayload<T>>
* Create BulkInsertPartitioner based on strategy params.
*/
protected Option<BulkInsertPartitioner<T>> getPartitioner(Map<String, String> strategyParams, Schema schema) {
if (strategyParams.containsKey(CLUSTERING_SORT_COLUMNS_PROPERTY)) {
return Option.of(new RDDCustomColumnsSortPartitioner(strategyParams.get(CLUSTERING_SORT_COLUMNS_PROPERTY).split(","),
if (strategyParams.containsKey(CLUSTERING_SORT_COLUMNS_PROPERTY.key())) {
return Option.of(new RDDCustomColumnsSortPartitioner(strategyParams.get(CLUSTERING_SORT_COLUMNS_PROPERTY.key()).split(","),
HoodieAvroUtils.addMetadataFields(schema)));
} else {
return Option.empty();

View File

@@ -18,6 +18,7 @@
package org.apache.hudi.client.utils;
import org.apache.hudi.common.config.HoodieConfig;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.spark.storage.StorageLevel;
@@ -31,14 +32,14 @@ import static org.apache.hudi.config.HoodieWriteConfig.WRITE_STATUS_STORAGE_LEVE
*/
public class SparkMemoryUtils {
public static StorageLevel getWriteStatusStorageLevel(Properties properties) {
return StorageLevel.fromString(properties.getProperty(WRITE_STATUS_STORAGE_LEVEL));
return StorageLevel.fromString(new HoodieConfig(properties).getString(WRITE_STATUS_STORAGE_LEVEL));
}
public static StorageLevel getBloomIndexInputStorageLevel(Properties properties) {
return StorageLevel.fromString(properties.getProperty(HoodieIndexConfig.BLOOM_INDEX_INPUT_STORAGE_LEVEL));
return StorageLevel.fromString(new HoodieConfig(properties).getString(HoodieIndexConfig.BLOOM_INDEX_INPUT_STORAGE_LEVEL));
}
public static StorageLevel getSimpleIndexInputStorageLevel(Properties properties) {
return StorageLevel.fromString(properties.getProperty(HoodieIndexConfig.SIMPLE_INDEX_INPUT_STORAGE_LEVEL));
return StorageLevel.fromString(new HoodieConfig(properties).getString(HoodieIndexConfig.SIMPLE_INDEX_INPUT_STORAGE_LEVEL));
}
}

View File

@@ -34,9 +34,9 @@ public class ComplexKeyGenerator extends BuiltinKeyGenerator {
public ComplexKeyGenerator(TypedProperties props) {
super(props);
this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY)
this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key())
.split(",")).map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList());
this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY)
this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key())
.split(",")).map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList());
complexAvroKeyGenerator = new ComplexAvroKeyGenerator(props);
}

View File

@@ -49,8 +49,8 @@ public class CustomKeyGenerator extends BuiltinKeyGenerator {
public CustomKeyGenerator(TypedProperties props) {
super(props);
this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY).split(",")).map(String::trim).collect(Collectors.toList());
this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY).split(",")).map(String::trim).collect(Collectors.toList());
this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key()).split(",")).map(String::trim).collect(Collectors.toList());
this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key()).split(",")).map(String::trim).collect(Collectors.toList());
customAvroKeyGenerator = new CustomAvroKeyGenerator(props);
}

View File

@@ -36,7 +36,7 @@ public class GlobalDeleteKeyGenerator extends BuiltinKeyGenerator {
private final GlobalAvroDeleteKeyGenerator globalAvroDeleteKeyGenerator;
public GlobalDeleteKeyGenerator(TypedProperties config) {
super(config);
this.recordKeyFields = Arrays.asList(config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY).split(","));
this.recordKeyFields = Arrays.asList(config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key()).split(","));
globalAvroDeleteKeyGenerator = new GlobalAvroDeleteKeyGenerator(config);
}

View File

@@ -37,7 +37,7 @@ public class NonpartitionedKeyGenerator extends BuiltinKeyGenerator {
public NonpartitionedKeyGenerator(TypedProperties props) {
super(props);
this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY)
this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key())
.split(",")).map(String::trim).collect(Collectors.toList());
this.partitionPathFields = Collections.emptyList();
nonpartitionedAvroKeyGenerator = new NonpartitionedAvroKeyGenerator(props);

View File

@@ -33,8 +33,8 @@ public class SimpleKeyGenerator extends BuiltinKeyGenerator {
private final SimpleAvroKeyGenerator simpleAvroKeyGenerator;
public SimpleKeyGenerator(TypedProperties props) {
this(props, props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY),
props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY));
this(props, props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key()),
props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key()));
}
SimpleKeyGenerator(TypedProperties props, String partitionPathField) {

View File

@@ -38,8 +38,8 @@ public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
private final TimestampBasedAvroKeyGenerator timestampBasedAvroKeyGenerator;
public TimestampBasedKeyGenerator(TypedProperties config) throws IOException {
this(config, config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY),
config.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY));
this(config, config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key()),
config.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key()));
}
TimestampBasedKeyGenerator(TypedProperties config, String partitionPathField) throws IOException {

View File

@@ -54,7 +54,7 @@ public class HoodieSparkKeyGeneratorFactory {
private static BuiltinKeyGenerator createKeyGeneratorByType(TypedProperties props) throws IOException {
// Use KeyGeneratorType.SIMPLE as default keyGeneratorType
String keyGeneratorType =
props.getString(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP, KeyGeneratorType.SIMPLE.name());
props.getString(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP.key(), KeyGeneratorType.SIMPLE.name());
KeyGeneratorType keyGeneratorTypeEnum;
try {

View File

@@ -190,7 +190,7 @@ public class SparkExecuteClusteringCommitActionExecutor<T extends HoodieRecordPa
return jsc.parallelize(clusteringOps, clusteringOps.size()).mapPartitions(clusteringOpsPartition -> {
List<Iterator<HoodieRecord<? extends HoodieRecordPayload>>> recordIterators = new ArrayList<>();
clusteringOpsPartition.forEachRemaining(clusteringOp -> {
long maxMemoryPerCompaction = IOUtils.getMaxMemoryPerCompaction(new SparkTaskContextSupplier(), config.getProps());
long maxMemoryPerCompaction = IOUtils.getMaxMemoryPerCompaction(new SparkTaskContextSupplier(), config);
LOG.info("MaxMemoryPerCompaction run as part of clustering => " + maxMemoryPerCompaction);
try {
Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));

View File

@@ -122,7 +122,7 @@ public class HoodieSparkMergeOnReadTableCompactor<T extends HoodieRecordPayload>
.getActiveTimeline().getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION,
HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION))
.filterCompletedInstants().lastInstant().get().getTimestamp();
long maxMemoryPerCompaction = IOUtils.getMaxMemoryPerCompaction(new SparkTaskContextSupplier(), config.getProps());
long maxMemoryPerCompaction = IOUtils.getMaxMemoryPerCompaction(new SparkTaskContextSupplier(), config);
LOG.info("MaxMemoryPerCompaction => " + maxMemoryPerCompaction);
List<String> logFiles = operation.getDeltaFileNames().stream().map(