[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:
@@ -360,7 +360,7 @@ public class SparkMain {
|
||||
String payloadClassName, String enableHiveSync, String propsFilePath, List<String> configs) throws IOException {
|
||||
|
||||
TypedProperties properties = propsFilePath == null ? UtilHelpers.buildProperties(configs)
|
||||
: UtilHelpers.readConfig(FSUtils.getFs(propsFilePath, jsc.hadoopConfiguration()), new Path(propsFilePath), configs).getProps(true);
|
||||
: UtilHelpers.readConfig(jsc.hadoopConfiguration(), new Path(propsFilePath), configs).getProps(true);
|
||||
|
||||
properties.setProperty(HoodieBootstrapConfig.BASE_PATH.key(), sourcePath);
|
||||
|
||||
|
||||
Reference in New Issue
Block a user