1
0

[HUDI-2538] persist some configs to hoodie.properties when the first write (#3823)

This commit is contained in:
Yann Byron
2021-11-03 10:04:23 +08:00
committed by GitHub
parent 1f17467f73
commit 6351e5f4d0
25 changed files with 544 additions and 207 deletions

View File

@@ -74,6 +74,10 @@ public class HoodieConfig implements Serializable {
}
}
public Boolean contains(String key) {
return props.containsKey(key);
}
public <T> boolean contains(ConfigProperty<T> configProperty) {
if (props.containsKey(configProperty.key())) {
return true;
@@ -135,7 +139,7 @@ public class HoodieConfig implements Serializable {
public <T> boolean getBooleanOrDefault(ConfigProperty<T> configProperty) {
Option<Object> rawValue = getRawValue(configProperty);
return rawValue.map(v -> Boolean.parseBoolean(v.toString()))
.orElse((Boolean) configProperty.defaultValue());
.orElse(Boolean.parseBoolean(configProperty.defaultValue().toString()));
}
public <T> Long getLong(ConfigProperty<T> configProperty) {

View File

@@ -31,6 +31,7 @@ import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FSDataInputStream;
@@ -161,6 +162,9 @@ public class HoodieTableConfig extends HoodieConfig {
.noDefaultValue()
.withDocumentation("Key Generator class property for the hoodie table");
public static final ConfigProperty<String> URL_ENCODE_PARTITIONING = KeyGeneratorOptions.URL_ENCODE_PARTITIONING;
public static final ConfigProperty<String> HIVE_STYLE_PARTITIONING_ENABLE = KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE;
public static final String NO_OP_BOOTSTRAP_INDEX_CLASS = NoOpBootstrapIndex.class.getName();
public HoodieTableConfig(FileSystem fs, String metaPath, String payloadClassName) {
@@ -363,6 +367,18 @@ public class HoodieTableConfig extends HoodieConfig {
return getString(RECORDKEY_FIELDS);
}
public String getKeyGeneratorClassName() {
return getString(KEY_GENERATOR_CLASS_NAME);
}
public String getHiveStylePartitioningEnable() {
return getString(HIVE_STYLE_PARTITIONING_ENABLE);
}
public String getUrlEncodePartitoning() {
return getString(URL_ENCODE_PARTITIONING);
}
public Map<String, String> propsMap() {
return props.entrySet().stream()
.collect(Collectors.toMap(e -> String.valueOf(e.getKey()), e -> String.valueOf(e.getValue())));

View File

@@ -637,6 +637,8 @@ public class HoodieTableMetaClient implements Serializable {
private Boolean bootstrapIndexEnable;
private Boolean populateMetaFields;
private String keyGeneratorClassProp;
private Boolean hiveStylePartitioningEnable;
private Boolean urlEncodePartitioning;
private PropertyBuilder() {
@@ -725,6 +727,16 @@ public class HoodieTableMetaClient implements Serializable {
return this;
}
public PropertyBuilder setHiveStylePartitioningEnable(Boolean hiveStylePartitioningEnable) {
this.hiveStylePartitioningEnable = hiveStylePartitioningEnable;
return this;
}
public PropertyBuilder setUrlEncodePartitioning(Boolean urlEncodePartitioning) {
this.urlEncodePartitioning = urlEncodePartitioning;
return this;
}
public PropertyBuilder fromMetaClient(HoodieTableMetaClient metaClient) {
return setTableType(metaClient.getTableType())
.setTableName(metaClient.getTableConfig().getTableName())
@@ -786,6 +798,12 @@ public class HoodieTableMetaClient implements Serializable {
if (hoodieConfig.contains(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME)) {
setKeyGeneratorClassProp(hoodieConfig.getString(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME));
}
if (hoodieConfig.contains(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE)) {
setHiveStylePartitioningEnable(hoodieConfig.getBoolean(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE));
}
if (hoodieConfig.contains(HoodieTableConfig.URL_ENCODE_PARTITIONING)) {
setUrlEncodePartitioning(hoodieConfig.getBoolean(HoodieTableConfig.URL_ENCODE_PARTITIONING));
}
return this;
}
@@ -849,6 +867,12 @@ public class HoodieTableMetaClient implements Serializable {
if (null != keyGeneratorClassProp) {
tableConfig.setValue(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME, keyGeneratorClassProp);
}
if (null != hiveStylePartitioningEnable) {
tableConfig.setValue(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE, Boolean.toString(hiveStylePartitioningEnable));
}
if (null != urlEncodePartitioning) {
tableConfig.setValue(HoodieTableConfig.URL_ENCODE_PARTITIONING, Boolean.toString(urlEncodePartitioning));
}
return tableConfig.getProps();
}

View File

@@ -34,12 +34,17 @@ import java.io.IOException;
*/
public class HoodieCommonTestHarness {
protected String tableName = null;
protected String basePath = null;
protected transient HoodieTestDataGenerator dataGen = null;
protected transient HoodieTableMetaClient metaClient;
@TempDir
public java.nio.file.Path tempDir;
protected void setTableName(String tableName) {
this.tableName = tableName;
}
/**
* Initializes basePath.
*/