1
0

[HUDI-2840] Fixed DeltaStreaemer to properly respect configuration passed t/h properties file (#4090)

* Rebased `DFSPropertiesConfiguration` to access Hadoop config in liue of FS to avoid confusion

* Fixed `readConfig` to take Hadoop's `Configuration` instead of FS;
Fixing usages

* Added test for local FS access

* Rebase to use `FSUtils.getFs`

* Combine properties provided as a file along w/ overrides provided from the CLI

* Added helper utilities to `HoodieClusteringConfig`;
Make sure corresponding config methods fallback to defaults;

* Fixed DeltaStreamer usage to respect properly combined configuration;
Abstracted `HoodieClusteringConfig.from` convenience utility to init Clustering config from `Properties`

* Tidying up

* `lint`

* Reverting changes to `HoodieWriteConfig`

* Tdiying up

* Fixed incorrect merge of the props

* Converted `HoodieConfig` to wrap around `Properties` into `TypedProperties`

* Fixed compilation

* Fixed compilation
This commit is contained in:
Alexey Kudinkin
2021-11-25 14:48:22 -08:00
committed by GitHub
parent e0125a7911
commit 6f5d8d04cd
19 changed files with 168 additions and 103 deletions

View File

@@ -21,12 +21,15 @@ package org.apache.hudi.common.config;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import java.io.BufferedReader;
import java.io.File;
import java.io.IOException;
@@ -58,7 +61,8 @@ public class DFSPropertiesConfiguration {
// props read from hudi-defaults.conf
private static TypedProperties GLOBAL_PROPS = loadGlobalProps();
private final FileSystem fs;
@Nullable
private final Configuration hadoopConfig;
private Path currentFilePath;
@@ -68,8 +72,8 @@ public class DFSPropertiesConfiguration {
// Keep track of files visited, to detect loops
private final Set<String> visitedFilePaths;
public DFSPropertiesConfiguration(FileSystem fs, Path filePath) {
this.fs = fs;
public DFSPropertiesConfiguration(@Nonnull Configuration hadoopConf, @Nonnull Path filePath) {
this.hadoopConfig = hadoopConf;
this.currentFilePath = filePath;
this.hoodieConfig = new HoodieConfig();
this.visitedFilePaths = new HashSet<>();
@@ -77,7 +81,7 @@ public class DFSPropertiesConfiguration {
}
public DFSPropertiesConfiguration() {
this.fs = null;
this.hadoopConfig = null;
this.currentFilePath = null;
this.hoodieConfig = new HoodieConfig();
this.visitedFilePaths = new HashSet<>();
@@ -119,13 +123,13 @@ public class DFSPropertiesConfiguration {
if (visitedFilePaths.contains(filePath.toString())) {
throw new IllegalStateException("Loop detected; file " + filePath + " already referenced");
}
FileSystem fileSystem;
try {
fileSystem = fs != null ? fs : filePath.getFileSystem(new Configuration());
} catch (IOException e) {
throw new IllegalArgumentException("Cannot get the file system from file path", e);
}
try (BufferedReader reader = new BufferedReader(new InputStreamReader(fileSystem.open(filePath)))) {
FileSystem fs = FSUtils.getFs(
filePath.toString(),
Option.ofNullable(hadoopConfig).orElseGet(Configuration::new)
);
try (BufferedReader reader = new BufferedReader(new InputStreamReader(fs.open(filePath)))) {
visitedFilePaths.add(filePath.toString());
currentFilePath = filePath;
addPropsFromStream(reader);

View File

@@ -44,14 +44,14 @@ public class HoodieConfig implements Serializable {
return config;
}
protected Properties props;
protected TypedProperties props;
public HoodieConfig() {
this.props = new Properties();
this.props = new TypedProperties();
}
public HoodieConfig(Properties props) {
this.props = props;
this.props = new TypedProperties(props);
}
public <T> void setValue(ConfigProperty<T> cfg, String val) {
@@ -147,7 +147,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.parseBoolean(configProperty.defaultValue().toString()));
.orElseGet(() -> Boolean.parseBoolean(configProperty.defaultValue().toString()));
}
public <T> Long getLong(ConfigProperty<T> configProperty) {
@@ -174,13 +174,13 @@ public class HoodieConfig implements Serializable {
return rawValue.map(Object::toString).orElse(defaultVal);
}
public Properties getProps() {
public TypedProperties getProps() {
return getProps(false);
}
public Properties getProps(boolean includeGlobalProps) {
public TypedProperties getProps(boolean includeGlobalProps) {
if (includeGlobalProps) {
Properties mergedProps = DFSPropertiesConfiguration.getGlobalProps();
TypedProperties mergedProps = DFSPropertiesConfiguration.getGlobalProps();
mergedProps.putAll(props);
return mergedProps;
} else {