1
0

[HUDI-2255] Refactor Datasource options (#3373)

Co-authored-by: Wenning Ding <wenningd@amazon.com>
This commit is contained in:
wenningd
2021-08-03 17:50:30 -07:00
committed by GitHub
parent 1ff2d3459a
commit 91bb0d1318
83 changed files with 887 additions and 887 deletions

View File

@@ -119,7 +119,7 @@ public class HoodieClusteringConfig extends HoodieConfig {
.withDocumentation("Determines how to handle updates, deletes to file groups that are under clustering."
+ " Default strategy just rejects the update");
public static final ConfigProperty<String> ASYNC_CLUSTERING_ENABLE_OPT_KEY = ConfigProperty
public static final ConfigProperty<String> ASYNC_CLUSTERING_ENABLE = ConfigProperty
.key("hoodie.clustering.async.enabled")
.defaultValue("false")
.sinceVersion("0.7.0")
@@ -210,7 +210,7 @@ public class HoodieClusteringConfig extends HoodieConfig {
}
public Builder withAsyncClustering(Boolean asyncClustering) {
clusteringConfig.setValue(ASYNC_CLUSTERING_ENABLE_OPT_KEY, String.valueOf(asyncClustering));
clusteringConfig.setValue(ASYNC_CLUSTERING_ENABLE, String.valueOf(asyncClustering));
return this;
}

View File

@@ -697,7 +697,7 @@ public class HoodieWriteConfig extends HoodieConfig {
}
public boolean isAsyncClusteringEnabled() {
return getBoolean(HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE_OPT_KEY);
return getBoolean(HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE);
}
public boolean isClusteringEnabled() {

View File

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

View File

@@ -55,8 +55,8 @@ public class CustomAvroKeyGenerator extends BaseKeyGenerator {
public CustomAvroKeyGenerator(TypedProperties props) {
super(props);
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());
this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD.key()).split(",")).map(String::trim).collect(Collectors.toList());
this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD.key()).split(",")).map(String::trim).collect(Collectors.toList());
}
@Override

View File

@@ -35,7 +35,7 @@ public class GlobalAvroDeleteKeyGenerator extends BaseKeyGenerator {
public GlobalAvroDeleteKeyGenerator(TypedProperties config) {
super(config);
this.recordKeyFields = Arrays.asList(config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key()).split(","));
this.recordKeyFields = Arrays.asList(config.getString(KeyGeneratorOptions.RECORDKEY_FIELD.key()).split(","));
}
@Override

View File

@@ -36,7 +36,7 @@ public class NonpartitionedAvroKeyGenerator extends BaseKeyGenerator {
public NonpartitionedAvroKeyGenerator(TypedProperties props) {
super(props);
this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key())
this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD.key())
.split(",")).map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList());
this.partitionPathFields = EMPTY_PARTITION_FIELD_LIST;
}

View File

@@ -29,8 +29,8 @@ import java.util.Collections;
public class SimpleAvroKeyGenerator extends BaseKeyGenerator {
public SimpleAvroKeyGenerator(TypedProperties props) {
this(props, props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key()),
props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key()));
this(props, props.getString(KeyGeneratorOptions.RECORDKEY_FIELD.key()),
props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD.key()));
}
SimpleAvroKeyGenerator(TypedProperties props, String partitionPathField) {

View File

@@ -88,8 +88,8 @@ public class TimestampBasedAvroKeyGenerator extends SimpleAvroKeyGenerator {
}
public TimestampBasedAvroKeyGenerator(TypedProperties config) throws IOException {
this(config, config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key()),
config.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key()));
this(config, config.getString(KeyGeneratorOptions.RECORDKEY_FIELD.key()),
config.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD.key()));
}
TimestampBasedAvroKeyGenerator(TypedProperties config, String partitionPathField) throws IOException {
@@ -119,8 +119,8 @@ public class TimestampBasedAvroKeyGenerator extends SimpleAvroKeyGenerator {
default:
timeUnit = null;
}
this.encodePartitionPath = config.getBoolean(KeyGeneratorOptions.URL_ENCODE_PARTITIONING_OPT_KEY.key(),
Boolean.parseBoolean(KeyGeneratorOptions.URL_ENCODE_PARTITIONING_OPT_KEY.defaultValue()));
this.encodePartitionPath = config.getBoolean(KeyGeneratorOptions.URL_ENCODE_PARTITIONING.key(),
Boolean.parseBoolean(KeyGeneratorOptions.URL_ENCODE_PARTITIONING.defaultValue()));
}
@Override