[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 {
|
String payloadClassName, String enableHiveSync, String propsFilePath, List<String> configs) throws IOException {
|
||||||
|
|
||||||
TypedProperties properties = propsFilePath == null ? UtilHelpers.buildProperties(configs)
|
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);
|
properties.setProperty(HoodieBootstrapConfig.BASE_PATH.key(), sourcePath);
|
||||||
|
|
||||||
|
|||||||
@@ -22,6 +22,7 @@ import org.apache.hudi.common.config.ConfigClassProperty;
|
|||||||
import org.apache.hudi.common.config.ConfigGroups;
|
import org.apache.hudi.common.config.ConfigGroups;
|
||||||
import org.apache.hudi.common.config.ConfigProperty;
|
import org.apache.hudi.common.config.ConfigProperty;
|
||||||
import org.apache.hudi.common.config.HoodieConfig;
|
import org.apache.hudi.common.config.HoodieConfig;
|
||||||
|
import org.apache.hudi.common.config.TypedProperties;
|
||||||
import org.apache.hudi.common.engine.EngineType;
|
import org.apache.hudi.common.engine.EngineType;
|
||||||
import org.apache.hudi.exception.HoodieException;
|
import org.apache.hudi.exception.HoodieException;
|
||||||
import org.apache.hudi.exception.HoodieNotSupportedException;
|
import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||||
@@ -335,10 +336,23 @@ public class HoodieClusteringConfig extends HoodieConfig {
|
|||||||
@Deprecated
|
@Deprecated
|
||||||
public static final String DEFAULT_ASYNC_CLUSTERING_ENABLE_OPT_VAL = ASYNC_CLUSTERING_ENABLE.defaultValue();
|
public static final String DEFAULT_ASYNC_CLUSTERING_ENABLE_OPT_VAL = ASYNC_CLUSTERING_ENABLE.defaultValue();
|
||||||
|
|
||||||
|
// NOTE: This ctor is required for appropriate deserialization
|
||||||
public HoodieClusteringConfig() {
|
public HoodieClusteringConfig() {
|
||||||
super();
|
super();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public boolean isAsyncClusteringEnabled() {
|
||||||
|
return getBooleanOrDefault(HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE);
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean isInlineClusteringEnabled() {
|
||||||
|
return getBooleanOrDefault(HoodieClusteringConfig.INLINE_CLUSTERING);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static HoodieClusteringConfig from(TypedProperties props) {
|
||||||
|
return HoodieClusteringConfig.newBuilder().fromProperties(props).build();
|
||||||
|
}
|
||||||
|
|
||||||
public static Builder newBuilder() {
|
public static Builder newBuilder() {
|
||||||
return new Builder();
|
return new Builder();
|
||||||
}
|
}
|
||||||
@@ -421,6 +435,7 @@ public class HoodieClusteringConfig extends HoodieConfig {
|
|||||||
}
|
}
|
||||||
|
|
||||||
public Builder fromProperties(Properties props) {
|
public Builder fromProperties(Properties props) {
|
||||||
|
// TODO this should cherry-pick only clustering properties
|
||||||
this.clusteringConfig.getProps().putAll(props);
|
this.clusteringConfig.getProps().putAll(props);
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -166,11 +166,6 @@ public class HoodieMetricsPrometheusConfig extends HoodieConfig {
|
|||||||
return new HoodieMetricsPrometheusConfig.Builder();
|
return new HoodieMetricsPrometheusConfig.Builder();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Properties getProps() {
|
|
||||||
return super.getProps();
|
|
||||||
}
|
|
||||||
|
|
||||||
public static class Builder {
|
public static class Builder {
|
||||||
|
|
||||||
private HoodieMetricsPrometheusConfig hoodieMetricsPrometheusConfig = new HoodieMetricsPrometheusConfig();
|
private HoodieMetricsPrometheusConfig hoodieMetricsPrometheusConfig = new HoodieMetricsPrometheusConfig();
|
||||||
|
|||||||
@@ -19,6 +19,7 @@
|
|||||||
|
|
||||||
package org.apache.hudi.metrics;
|
package org.apache.hudi.metrics;
|
||||||
|
|
||||||
|
import org.apache.hudi.common.config.TypedProperties;
|
||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
|
|
||||||
import com.codahale.metrics.MetricRegistry;
|
import com.codahale.metrics.MetricRegistry;
|
||||||
@@ -57,7 +58,7 @@ public class TestMetricsReporterFactory {
|
|||||||
public void metricsReporterFactoryShouldReturnUserDefinedReporter() {
|
public void metricsReporterFactoryShouldReturnUserDefinedReporter() {
|
||||||
when(config.getMetricReporterClassName()).thenReturn(DummyMetricsReporter.class.getName());
|
when(config.getMetricReporterClassName()).thenReturn(DummyMetricsReporter.class.getName());
|
||||||
|
|
||||||
Properties props = new Properties();
|
TypedProperties props = new TypedProperties();
|
||||||
props.setProperty("testKey", "testValue");
|
props.setProperty("testKey", "testValue");
|
||||||
|
|
||||||
when(config.getProps()).thenReturn(props);
|
when(config.getProps()).thenReturn(props);
|
||||||
@@ -70,7 +71,7 @@ public class TestMetricsReporterFactory {
|
|||||||
@Test
|
@Test
|
||||||
public void metricsReporterFactoryShouldThrowExceptionWhenMetricsReporterClassIsIllegal() {
|
public void metricsReporterFactoryShouldThrowExceptionWhenMetricsReporterClassIsIllegal() {
|
||||||
when(config.getMetricReporterClassName()).thenReturn(IllegalTestMetricsReporter.class.getName());
|
when(config.getMetricReporterClassName()).thenReturn(IllegalTestMetricsReporter.class.getName());
|
||||||
when(config.getProps()).thenReturn(new Properties());
|
when(config.getProps()).thenReturn(new TypedProperties());
|
||||||
assertThrows(HoodieException.class, () -> MetricsReporterFactory.createReporter(config, registry));
|
assertThrows(HoodieException.class, () -> MetricsReporterFactory.createReporter(config, registry));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@@ -21,12 +21,15 @@ package org.apache.hudi.common.config;
|
|||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
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.Option;
|
||||||
import org.apache.hudi.common.util.StringUtils;
|
import org.apache.hudi.common.util.StringUtils;
|
||||||
import org.apache.hudi.common.util.ValidationUtils;
|
import org.apache.hudi.common.util.ValidationUtils;
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
|
import javax.annotation.Nonnull;
|
||||||
|
import javax.annotation.Nullable;
|
||||||
import java.io.BufferedReader;
|
import java.io.BufferedReader;
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
@@ -58,7 +61,8 @@ public class DFSPropertiesConfiguration {
|
|||||||
// props read from hudi-defaults.conf
|
// props read from hudi-defaults.conf
|
||||||
private static TypedProperties GLOBAL_PROPS = loadGlobalProps();
|
private static TypedProperties GLOBAL_PROPS = loadGlobalProps();
|
||||||
|
|
||||||
private final FileSystem fs;
|
@Nullable
|
||||||
|
private final Configuration hadoopConfig;
|
||||||
|
|
||||||
private Path currentFilePath;
|
private Path currentFilePath;
|
||||||
|
|
||||||
@@ -68,8 +72,8 @@ public class DFSPropertiesConfiguration {
|
|||||||
// Keep track of files visited, to detect loops
|
// Keep track of files visited, to detect loops
|
||||||
private final Set<String> visitedFilePaths;
|
private final Set<String> visitedFilePaths;
|
||||||
|
|
||||||
public DFSPropertiesConfiguration(FileSystem fs, Path filePath) {
|
public DFSPropertiesConfiguration(@Nonnull Configuration hadoopConf, @Nonnull Path filePath) {
|
||||||
this.fs = fs;
|
this.hadoopConfig = hadoopConf;
|
||||||
this.currentFilePath = filePath;
|
this.currentFilePath = filePath;
|
||||||
this.hoodieConfig = new HoodieConfig();
|
this.hoodieConfig = new HoodieConfig();
|
||||||
this.visitedFilePaths = new HashSet<>();
|
this.visitedFilePaths = new HashSet<>();
|
||||||
@@ -77,7 +81,7 @@ public class DFSPropertiesConfiguration {
|
|||||||
}
|
}
|
||||||
|
|
||||||
public DFSPropertiesConfiguration() {
|
public DFSPropertiesConfiguration() {
|
||||||
this.fs = null;
|
this.hadoopConfig = null;
|
||||||
this.currentFilePath = null;
|
this.currentFilePath = null;
|
||||||
this.hoodieConfig = new HoodieConfig();
|
this.hoodieConfig = new HoodieConfig();
|
||||||
this.visitedFilePaths = new HashSet<>();
|
this.visitedFilePaths = new HashSet<>();
|
||||||
@@ -119,13 +123,13 @@ public class DFSPropertiesConfiguration {
|
|||||||
if (visitedFilePaths.contains(filePath.toString())) {
|
if (visitedFilePaths.contains(filePath.toString())) {
|
||||||
throw new IllegalStateException("Loop detected; file " + filePath + " already referenced");
|
throw new IllegalStateException("Loop detected; file " + filePath + " already referenced");
|
||||||
}
|
}
|
||||||
FileSystem fileSystem;
|
|
||||||
try {
|
FileSystem fs = FSUtils.getFs(
|
||||||
fileSystem = fs != null ? fs : filePath.getFileSystem(new Configuration());
|
filePath.toString(),
|
||||||
} catch (IOException e) {
|
Option.ofNullable(hadoopConfig).orElseGet(Configuration::new)
|
||||||
throw new IllegalArgumentException("Cannot get the file system from file path", e);
|
);
|
||||||
}
|
|
||||||
try (BufferedReader reader = new BufferedReader(new InputStreamReader(fileSystem.open(filePath)))) {
|
try (BufferedReader reader = new BufferedReader(new InputStreamReader(fs.open(filePath)))) {
|
||||||
visitedFilePaths.add(filePath.toString());
|
visitedFilePaths.add(filePath.toString());
|
||||||
currentFilePath = filePath;
|
currentFilePath = filePath;
|
||||||
addPropsFromStream(reader);
|
addPropsFromStream(reader);
|
||||||
|
|||||||
@@ -44,14 +44,14 @@ public class HoodieConfig implements Serializable {
|
|||||||
return config;
|
return config;
|
||||||
}
|
}
|
||||||
|
|
||||||
protected Properties props;
|
protected TypedProperties props;
|
||||||
|
|
||||||
public HoodieConfig() {
|
public HoodieConfig() {
|
||||||
this.props = new Properties();
|
this.props = new TypedProperties();
|
||||||
}
|
}
|
||||||
|
|
||||||
public HoodieConfig(Properties props) {
|
public HoodieConfig(Properties props) {
|
||||||
this.props = props;
|
this.props = new TypedProperties(props);
|
||||||
}
|
}
|
||||||
|
|
||||||
public <T> void setValue(ConfigProperty<T> cfg, String val) {
|
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) {
|
public <T> boolean getBooleanOrDefault(ConfigProperty<T> configProperty) {
|
||||||
Option<Object> rawValue = getRawValue(configProperty);
|
Option<Object> rawValue = getRawValue(configProperty);
|
||||||
return rawValue.map(v -> Boolean.parseBoolean(v.toString()))
|
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) {
|
public <T> Long getLong(ConfigProperty<T> configProperty) {
|
||||||
@@ -174,13 +174,13 @@ public class HoodieConfig implements Serializable {
|
|||||||
return rawValue.map(Object::toString).orElse(defaultVal);
|
return rawValue.map(Object::toString).orElse(defaultVal);
|
||||||
}
|
}
|
||||||
|
|
||||||
public Properties getProps() {
|
public TypedProperties getProps() {
|
||||||
return getProps(false);
|
return getProps(false);
|
||||||
}
|
}
|
||||||
|
|
||||||
public Properties getProps(boolean includeGlobalProps) {
|
public TypedProperties getProps(boolean includeGlobalProps) {
|
||||||
if (includeGlobalProps) {
|
if (includeGlobalProps) {
|
||||||
Properties mergedProps = DFSPropertiesConfiguration.getGlobalProps();
|
TypedProperties mergedProps = DFSPropertiesConfiguration.getGlobalProps();
|
||||||
mergedProps.putAll(props);
|
mergedProps.putAll(props);
|
||||||
return mergedProps;
|
return mergedProps;
|
||||||
} else {
|
} else {
|
||||||
|
|||||||
@@ -103,7 +103,7 @@ public class TestDFSPropertiesConfiguration {
|
|||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testParsing() {
|
public void testParsing() {
|
||||||
DFSPropertiesConfiguration cfg = new DFSPropertiesConfiguration(dfs, new Path(dfsBasePath + "/t1.props"));
|
DFSPropertiesConfiguration cfg = new DFSPropertiesConfiguration(dfs.getConf(), new Path(dfsBasePath + "/t1.props"));
|
||||||
TypedProperties props = cfg.getProps();
|
TypedProperties props = cfg.getProps();
|
||||||
assertEquals(5, props.size());
|
assertEquals(5, props.size());
|
||||||
assertThrows(IllegalArgumentException.class, () -> {
|
assertThrows(IllegalArgumentException.class, () -> {
|
||||||
@@ -131,7 +131,7 @@ public class TestDFSPropertiesConfiguration {
|
|||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testIncludes() {
|
public void testIncludes() {
|
||||||
DFSPropertiesConfiguration cfg = new DFSPropertiesConfiguration(dfs, new Path(dfsBasePath + "/t3.props"));
|
DFSPropertiesConfiguration cfg = new DFSPropertiesConfiguration(dfs.getConf(), new Path(dfsBasePath + "/t3.props"));
|
||||||
TypedProperties props = cfg.getProps();
|
TypedProperties props = cfg.getProps();
|
||||||
|
|
||||||
assertEquals(123, props.getInteger("int.prop"));
|
assertEquals(123, props.getInteger("int.prop"));
|
||||||
@@ -144,6 +144,31 @@ public class TestDFSPropertiesConfiguration {
|
|||||||
}, "Should error out on a self-included file.");
|
}, "Should error out on a self-included file.");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testLocalFileSystemLoading() {
|
||||||
|
DFSPropertiesConfiguration cfg = new DFSPropertiesConfiguration(dfs.getConf(), new Path(dfsBasePath + "/t1.props"));
|
||||||
|
|
||||||
|
cfg.addPropsFromFile(
|
||||||
|
new Path(
|
||||||
|
String.format(
|
||||||
|
"file:%s",
|
||||||
|
getClass().getClassLoader()
|
||||||
|
.getResource("props/test.properties")
|
||||||
|
.getPath()
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
TypedProperties props = cfg.getProps();
|
||||||
|
|
||||||
|
assertEquals(123, props.getInteger("int.prop"));
|
||||||
|
assertEquals(113.4, props.getDouble("double.prop"), 0.001);
|
||||||
|
assertTrue(props.getBoolean("boolean.prop"));
|
||||||
|
assertEquals("str", props.getString("string.prop"));
|
||||||
|
assertEquals(1354354354, props.getLong("long.prop"));
|
||||||
|
assertEquals(123, props.getInteger("some.random.prop"));
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testNoGlobalConfFileConfigured() {
|
public void testNoGlobalConfFileConfigured() {
|
||||||
ENVIRONMENT_VARIABLES.clear(DFSPropertiesConfiguration.CONF_FILE_DIR_ENV_NAME);
|
ENVIRONMENT_VARIABLES.clear(DFSPropertiesConfiguration.CONF_FILE_DIR_ENV_NAME);
|
||||||
|
|||||||
18
hudi-common/src/test/resources/props/test.properties
Normal file
18
hudi-common/src/test/resources/props/test.properties
Normal file
@@ -0,0 +1,18 @@
|
|||||||
|
|
||||||
|
# Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
# or more contributor license agreements. See the NOTICE file
|
||||||
|
# distributed with this work for additional information
|
||||||
|
# regarding copyright ownership. The ASF licenses this file
|
||||||
|
# to you under the Apache License, Version 2.0 (the
|
||||||
|
# "License"); you may not use this file except in compliance
|
||||||
|
# with the License. You may obtain a copy of the License at
|
||||||
|
#
|
||||||
|
# http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
#
|
||||||
|
# Unless required by applicable law or agreed to in writing, software
|
||||||
|
# distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
# See the License for the specific language governing permissions and
|
||||||
|
# limitations under the License.
|
||||||
|
|
||||||
|
some.random.prop=123
|
||||||
@@ -102,7 +102,7 @@ public class StreamerUtil {
|
|||||||
return new TypedProperties();
|
return new TypedProperties();
|
||||||
}
|
}
|
||||||
return readConfig(
|
return readConfig(
|
||||||
FSUtils.getFs(cfg.propsFilePath, getHadoopConf()),
|
getHadoopConf(),
|
||||||
new Path(cfg.propsFilePath), cfg.configs).getProps();
|
new Path(cfg.propsFilePath), cfg.configs).getProps();
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -127,8 +127,8 @@ public class StreamerUtil {
|
|||||||
/**
|
/**
|
||||||
* Read config from properties file (`--props` option) and cmd line (`--hoodie-conf` option).
|
* Read config from properties file (`--props` option) and cmd line (`--hoodie-conf` option).
|
||||||
*/
|
*/
|
||||||
public static DFSPropertiesConfiguration readConfig(FileSystem fs, Path cfgPath, List<String> overriddenProps) {
|
public static DFSPropertiesConfiguration readConfig(org.apache.hadoop.conf.Configuration hadoopConfig, Path cfgPath, List<String> overriddenProps) {
|
||||||
DFSPropertiesConfiguration conf = new DFSPropertiesConfiguration(fs, cfgPath);
|
DFSPropertiesConfiguration conf = new DFSPropertiesConfiguration(hadoopConfig, cfgPath);
|
||||||
try {
|
try {
|
||||||
if (!overriddenProps.isEmpty()) {
|
if (!overriddenProps.isEmpty()) {
|
||||||
LOG.info("Adding overridden properties to file properties.");
|
LOG.info("Adding overridden properties to file properties.");
|
||||||
|
|||||||
@@ -103,7 +103,7 @@ public class HoodieTestSuiteJob {
|
|||||||
cfg.propsFilePath = FSUtils.addSchemeIfLocalPath(cfg.propsFilePath).toString();
|
cfg.propsFilePath = FSUtils.addSchemeIfLocalPath(cfg.propsFilePath).toString();
|
||||||
this.sparkSession = SparkSession.builder().config(jsc.getConf()).enableHiveSupport().getOrCreate();
|
this.sparkSession = SparkSession.builder().config(jsc.getConf()).enableHiveSupport().getOrCreate();
|
||||||
this.fs = FSUtils.getFs(cfg.inputBasePath, jsc.hadoopConfiguration());
|
this.fs = FSUtils.getFs(cfg.inputBasePath, jsc.hadoopConfiguration());
|
||||||
this.props = UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getProps();
|
this.props = UtilHelpers.readConfig(fs.getConf(), new Path(cfg.propsFilePath), cfg.configs).getProps();
|
||||||
log.info("Creating workload generator with configs : {}", props.toString());
|
log.info("Creating workload generator with configs : {}", props.toString());
|
||||||
this.hiveConf = getDefaultHiveConf(jsc.hadoopConfiguration());
|
this.hiveConf = getDefaultHiveConf(jsc.hadoopConfiguration());
|
||||||
this.keyGenerator = (BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
|
this.keyGenerator = (BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
|
||||||
|
|||||||
@@ -112,7 +112,7 @@ public class HDFSParquetImporter implements Serializable {
|
|||||||
public int dataImport(JavaSparkContext jsc, int retry) {
|
public int dataImport(JavaSparkContext jsc, int retry) {
|
||||||
this.fs = FSUtils.getFs(cfg.targetPath, jsc.hadoopConfiguration());
|
this.fs = FSUtils.getFs(cfg.targetPath, jsc.hadoopConfiguration());
|
||||||
this.props = cfg.propsFilePath == null ? UtilHelpers.buildProperties(cfg.configs)
|
this.props = cfg.propsFilePath == null ? UtilHelpers.buildProperties(cfg.configs)
|
||||||
: UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getProps(true);
|
: UtilHelpers.readConfig(fs.getConf(), new Path(cfg.propsFilePath), cfg.configs).getProps(true);
|
||||||
LOG.info("Starting data import with configs : " + props.toString());
|
LOG.info("Starting data import with configs : " + props.toString());
|
||||||
int ret = -1;
|
int ret = -1;
|
||||||
try {
|
try {
|
||||||
|
|||||||
@@ -18,16 +18,13 @@
|
|||||||
|
|
||||||
package org.apache.hudi.utilities;
|
package org.apache.hudi.utilities;
|
||||||
|
|
||||||
|
import com.beust.jcommander.JCommander;
|
||||||
|
import com.beust.jcommander.Parameter;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||||
import org.apache.hudi.common.config.TypedProperties;
|
import org.apache.hudi.common.config.TypedProperties;
|
||||||
import org.apache.hudi.common.fs.FSUtils;
|
|
||||||
import org.apache.hudi.config.HoodieWriteConfig;
|
import org.apache.hudi.config.HoodieWriteConfig;
|
||||||
|
|
||||||
import com.beust.jcommander.JCommander;
|
|
||||||
import com.beust.jcommander.Parameter;
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
|
||||||
import org.apache.hadoop.fs.Path;
|
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
@@ -61,9 +58,8 @@ public class HoodieCleaner {
|
|||||||
/*
|
/*
|
||||||
* Filesystem used.
|
* Filesystem used.
|
||||||
*/
|
*/
|
||||||
FileSystem fs = FSUtils.getFs(cfg.basePath, jssc.hadoopConfiguration());
|
|
||||||
this.props = cfg.propsFilePath == null ? UtilHelpers.buildProperties(cfg.configs)
|
this.props = cfg.propsFilePath == null ? UtilHelpers.buildProperties(cfg.configs)
|
||||||
: UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getProps(true);
|
: UtilHelpers.readConfig(jssc.hadoopConfiguration(), new Path(cfg.propsFilePath), cfg.configs).getProps(true);
|
||||||
LOG.info("Creating Cleaner with configs : " + props.toString());
|
LOG.info("Creating Cleaner with configs : " + props.toString());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@@ -73,10 +73,7 @@ public class HoodieClusteringJob {
|
|||||||
}
|
}
|
||||||
|
|
||||||
private TypedProperties readConfigFromFileSystem(JavaSparkContext jsc, Config cfg) {
|
private TypedProperties readConfigFromFileSystem(JavaSparkContext jsc, Config cfg) {
|
||||||
final FileSystem fs = FSUtils.getFs(cfg.basePath, jsc.hadoopConfiguration());
|
return UtilHelpers.readConfig(jsc.hadoopConfiguration(), new Path(cfg.propsFilePath), cfg.configs)
|
||||||
|
|
||||||
return UtilHelpers
|
|
||||||
.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs)
|
|
||||||
.getProps(true);
|
.getProps(true);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@@ -62,10 +62,7 @@ public class HoodieCompactor {
|
|||||||
}
|
}
|
||||||
|
|
||||||
private TypedProperties readConfigFromFileSystem(JavaSparkContext jsc, Config cfg) {
|
private TypedProperties readConfigFromFileSystem(JavaSparkContext jsc, Config cfg) {
|
||||||
final FileSystem fs = FSUtils.getFs(cfg.basePath, jsc.hadoopConfiguration());
|
return UtilHelpers.readConfig(jsc.hadoopConfiguration(), new Path(cfg.propsFilePath), cfg.configs)
|
||||||
|
|
||||||
return UtilHelpers
|
|
||||||
.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs)
|
|
||||||
.getProps(true);
|
.getProps(true);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@@ -18,6 +18,7 @@
|
|||||||
|
|
||||||
package org.apache.hudi.utilities;
|
package org.apache.hudi.utilities;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hudi.AvroConversionUtils;
|
import org.apache.hudi.AvroConversionUtils;
|
||||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||||
import org.apache.hudi.client.WriteStatus;
|
import org.apache.hudi.client.WriteStatus;
|
||||||
@@ -157,11 +158,8 @@ public class UtilHelpers {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
public static DFSPropertiesConfiguration readConfig(Configuration hadoopConfig, Path cfgPath, List<String> overriddenProps) {
|
||||||
*
|
DFSPropertiesConfiguration conf = new DFSPropertiesConfiguration(hadoopConfig, cfgPath);
|
||||||
*/
|
|
||||||
public static DFSPropertiesConfiguration readConfig(FileSystem fs, Path cfgPath, List<String> overriddenProps) {
|
|
||||||
DFSPropertiesConfiguration conf = new DFSPropertiesConfiguration(fs, cfgPath);
|
|
||||||
try {
|
try {
|
||||||
if (!overriddenProps.isEmpty()) {
|
if (!overriddenProps.isEmpty()) {
|
||||||
LOG.info("Adding overridden properties to file properties.");
|
LOG.info("Adding overridden properties to file properties.");
|
||||||
|
|||||||
@@ -172,6 +172,8 @@ public class DeltaSync implements Serializable {
|
|||||||
|
|
||||||
/**
|
/**
|
||||||
* Bag of properties with source, hoodie client, key generator etc.
|
* Bag of properties with source, hoodie client, key generator etc.
|
||||||
|
*
|
||||||
|
* NOTE: These properties are already consolidated w/ CLI provided config-overrides
|
||||||
*/
|
*/
|
||||||
private final TypedProperties props;
|
private final TypedProperties props;
|
||||||
|
|
||||||
@@ -698,22 +700,33 @@ public class DeltaSync implements Serializable {
|
|||||||
private HoodieWriteConfig getHoodieClientConfig(Schema schema) {
|
private HoodieWriteConfig getHoodieClientConfig(Schema schema) {
|
||||||
final boolean combineBeforeUpsert = true;
|
final boolean combineBeforeUpsert = true;
|
||||||
final boolean autoCommit = false;
|
final boolean autoCommit = false;
|
||||||
HoodieWriteConfig.Builder builder =
|
|
||||||
HoodieWriteConfig.newBuilder().withPath(cfg.targetBasePath).combineInput(cfg.filterDupes, combineBeforeUpsert)
|
|
||||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().withPayloadClass(cfg.payloadClassName)
|
|
||||||
// Inline compaction is disabled for continuous mode. otherwise enabled for MOR
|
|
||||||
.withInlineCompaction(cfg.isInlineCompactionEnabled()).build())
|
|
||||||
.withClusteringConfig(HoodieClusteringConfig.newBuilder()
|
|
||||||
.withInlineClustering(cfg.isInlineClusteringEnabled())
|
|
||||||
.withAsyncClustering(cfg.isAsyncClusteringEnabled()).build())
|
|
||||||
.withPayloadConfig(HoodiePayloadConfig.newBuilder().withPayloadOrderingField(cfg.sourceOrderingField)
|
|
||||||
.build())
|
|
||||||
.forTable(cfg.targetTableName)
|
|
||||||
.withAutoCommit(autoCommit).withProps(props);
|
|
||||||
|
|
||||||
if (null != schema) {
|
// NOTE: Provided that we're injecting combined properties
|
||||||
builder = builder.withSchema(schema.toString());
|
// (from {@code props}, including CLI overrides), there's no
|
||||||
|
// need to explicitly set up some configuration aspects that
|
||||||
|
// are based on these (for ex Clustering configuration)
|
||||||
|
HoodieWriteConfig.Builder builder =
|
||||||
|
HoodieWriteConfig.newBuilder()
|
||||||
|
.withPath(cfg.targetBasePath)
|
||||||
|
.combineInput(cfg.filterDupes, combineBeforeUpsert)
|
||||||
|
.withCompactionConfig(
|
||||||
|
HoodieCompactionConfig.newBuilder()
|
||||||
|
.withPayloadClass(cfg.payloadClassName)
|
||||||
|
.withInlineCompaction(cfg.isInlineCompactionEnabled())
|
||||||
|
.build()
|
||||||
|
)
|
||||||
|
.withPayloadConfig(
|
||||||
|
HoodiePayloadConfig.newBuilder()
|
||||||
|
.withPayloadOrderingField(cfg.sourceOrderingField)
|
||||||
|
.build())
|
||||||
|
.forTable(cfg.targetTableName)
|
||||||
|
.withAutoCommit(autoCommit)
|
||||||
|
.withProps(props);
|
||||||
|
|
||||||
|
if (schema != null) {
|
||||||
|
builder.withSchema(schema.toString());
|
||||||
}
|
}
|
||||||
|
|
||||||
HoodieWriteConfig config = builder.build();
|
HoodieWriteConfig config = builder.build();
|
||||||
|
|
||||||
// set default value for {@link HoodieWriteCommitKafkaCallbackConfig} if needed.
|
// set default value for {@link HoodieWriteCommitKafkaCallbackConfig} if needed.
|
||||||
@@ -721,13 +734,15 @@ public class DeltaSync implements Serializable {
|
|||||||
HoodieWriteCommitKafkaCallbackConfig.setCallbackKafkaConfigIfNeeded(config);
|
HoodieWriteCommitKafkaCallbackConfig.setCallbackKafkaConfigIfNeeded(config);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.from(props);
|
||||||
|
|
||||||
// Validate what deltastreamer assumes of write-config to be really safe
|
// Validate what deltastreamer assumes of write-config to be really safe
|
||||||
ValidationUtils.checkArgument(config.inlineCompactionEnabled() == cfg.isInlineCompactionEnabled(),
|
ValidationUtils.checkArgument(config.inlineCompactionEnabled() == cfg.isInlineCompactionEnabled(),
|
||||||
String.format("%s should be set to %s", INLINE_COMPACT.key(), cfg.isInlineCompactionEnabled()));
|
String.format("%s should be set to %s", INLINE_COMPACT.key(), cfg.isInlineCompactionEnabled()));
|
||||||
ValidationUtils.checkArgument(config.inlineClusteringEnabled() == cfg.isInlineClusteringEnabled(),
|
ValidationUtils.checkArgument(config.inlineClusteringEnabled() == clusteringConfig.isInlineClusteringEnabled(),
|
||||||
String.format("%s should be set to %s", INLINE_CLUSTERING.key(), cfg.isInlineClusteringEnabled()));
|
String.format("%s should be set to %s", INLINE_CLUSTERING.key(), clusteringConfig.isInlineClusteringEnabled()));
|
||||||
ValidationUtils.checkArgument(config.isAsyncClusteringEnabled() == cfg.isAsyncClusteringEnabled(),
|
ValidationUtils.checkArgument(config.isAsyncClusteringEnabled() == clusteringConfig.isAsyncClusteringEnabled(),
|
||||||
String.format("%s should be set to %s", ASYNC_CLUSTERING_ENABLE.key(), cfg.isAsyncClusteringEnabled()));
|
String.format("%s should be set to %s", ASYNC_CLUSTERING_ENABLE.key(), clusteringConfig.isAsyncClusteringEnabled()));
|
||||||
ValidationUtils.checkArgument(!config.shouldAutoCommit(),
|
ValidationUtils.checkArgument(!config.shouldAutoCommit(),
|
||||||
String.format("%s should be set to %s", AUTO_COMMIT_ENABLE.key(), autoCommit));
|
String.format("%s should be set to %s", AUTO_COMMIT_ENABLE.key(), autoCommit));
|
||||||
ValidationUtils.checkArgument(config.shouldCombineBeforeInsert() == cfg.filterDupes,
|
ValidationUtils.checkArgument(config.shouldCombineBeforeInsert() == cfg.filterDupes,
|
||||||
|
|||||||
@@ -99,6 +99,9 @@ public class HoodieDeltaStreamer implements Serializable {
|
|||||||
|
|
||||||
protected final transient Config cfg;
|
protected final transient Config cfg;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* NOTE: These properties are already consolidated w/ CLI provided config-overrides.
|
||||||
|
*/
|
||||||
private final TypedProperties properties;
|
private final TypedProperties properties;
|
||||||
|
|
||||||
protected transient Option<DeltaSyncService> deltaSyncService;
|
protected transient Option<DeltaSyncService> deltaSyncService;
|
||||||
@@ -122,20 +125,8 @@ public class HoodieDeltaStreamer implements Serializable {
|
|||||||
}
|
}
|
||||||
|
|
||||||
public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc, FileSystem fs, Configuration conf,
|
public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc, FileSystem fs, Configuration conf,
|
||||||
Option<TypedProperties> props) throws IOException {
|
Option<TypedProperties> propsOverride) throws IOException {
|
||||||
// Resolving the properties first in a consistent way
|
this.properties = combineProperties(cfg, propsOverride, jssc.hadoopConfiguration());
|
||||||
HoodieConfig hoodieConfig = new HoodieConfig();
|
|
||||||
if (props.isPresent()) {
|
|
||||||
hoodieConfig.setAll(props.get());
|
|
||||||
} else if (cfg.propsFilePath.equals(Config.DEFAULT_DFS_SOURCE_PROPERTIES)) {
|
|
||||||
hoodieConfig.setAll(UtilHelpers.getConfig(cfg.configs).getProps());
|
|
||||||
} else {
|
|
||||||
hoodieConfig.setAll(UtilHelpers.readConfig(
|
|
||||||
FSUtils.getFs(cfg.propsFilePath, jssc.hadoopConfiguration()),
|
|
||||||
new Path(cfg.propsFilePath), cfg.configs).getProps());
|
|
||||||
}
|
|
||||||
hoodieConfig.setDefaultValue(DataSourceWriteOptions.RECONCILE_SCHEMA());
|
|
||||||
this.properties = (TypedProperties) hoodieConfig.getProps(true);
|
|
||||||
|
|
||||||
if (cfg.initialCheckpointProvider != null && cfg.checkpoint == null) {
|
if (cfg.initialCheckpointProvider != null && cfg.checkpoint == null) {
|
||||||
InitialCheckPointProvider checkPointProvider =
|
InitialCheckPointProvider checkPointProvider =
|
||||||
@@ -143,6 +134,7 @@ public class HoodieDeltaStreamer implements Serializable {
|
|||||||
checkPointProvider.init(conf);
|
checkPointProvider.init(conf);
|
||||||
cfg.checkpoint = checkPointProvider.getCheckpoint();
|
cfg.checkpoint = checkPointProvider.getCheckpoint();
|
||||||
}
|
}
|
||||||
|
|
||||||
this.cfg = cfg;
|
this.cfg = cfg;
|
||||||
this.bootstrapExecutor = Option.ofNullable(
|
this.bootstrapExecutor = Option.ofNullable(
|
||||||
cfg.runBootstrap ? new BootstrapExecutor(cfg, jssc, fs, conf, this.properties) : null);
|
cfg.runBootstrap ? new BootstrapExecutor(cfg, jssc, fs, conf, this.properties) : null);
|
||||||
@@ -150,6 +142,25 @@ public class HoodieDeltaStreamer implements Serializable {
|
|||||||
cfg.runBootstrap ? null : new DeltaSyncService(cfg, jssc, fs, conf, Option.ofNullable(this.properties)));
|
cfg.runBootstrap ? null : new DeltaSyncService(cfg, jssc, fs, conf, Option.ofNullable(this.properties)));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static TypedProperties combineProperties(Config cfg, Option<TypedProperties> propsOverride, Configuration hadoopConf) {
|
||||||
|
HoodieConfig hoodieConfig = new HoodieConfig();
|
||||||
|
// Resolving the properties in a consistent way:
|
||||||
|
// 1. Properties override always takes precedence
|
||||||
|
// 2. Otherwise, check if there's no props file specified (merging in CLI overrides)
|
||||||
|
// 3. Otherwise, parse provided specified props file (merging in CLI overrides)
|
||||||
|
if (propsOverride.isPresent()) {
|
||||||
|
hoodieConfig.setAll(propsOverride.get());
|
||||||
|
} else if (cfg.propsFilePath.equals(Config.DEFAULT_DFS_SOURCE_PROPERTIES)) {
|
||||||
|
hoodieConfig.setAll(UtilHelpers.getConfig(cfg.configs).getProps());
|
||||||
|
} else {
|
||||||
|
hoodieConfig.setAll(UtilHelpers.readConfig(hadoopConf, new Path(cfg.propsFilePath), cfg.configs).getProps());
|
||||||
|
}
|
||||||
|
|
||||||
|
hoodieConfig.setDefaultValue(DataSourceWriteOptions.RECONCILE_SCHEMA());
|
||||||
|
|
||||||
|
return hoodieConfig.getProps(true);
|
||||||
|
}
|
||||||
|
|
||||||
public void shutdownGracefully() {
|
public void shutdownGracefully() {
|
||||||
deltaSyncService.ifPresent(ds -> ds.shutdown(false));
|
deltaSyncService.ifPresent(ds -> ds.shutdown(false));
|
||||||
}
|
}
|
||||||
@@ -364,20 +375,11 @@ public class HoodieDeltaStreamer implements Serializable {
|
|||||||
}
|
}
|
||||||
|
|
||||||
public boolean isInlineCompactionEnabled() {
|
public boolean isInlineCompactionEnabled() {
|
||||||
|
// Inline compaction is disabled for continuous mode, otherwise enabled for MOR
|
||||||
return !continuousMode && !forceDisableCompaction
|
return !continuousMode && !forceDisableCompaction
|
||||||
&& HoodieTableType.MERGE_ON_READ.equals(HoodieTableType.valueOf(tableType));
|
&& HoodieTableType.MERGE_ON_READ.equals(HoodieTableType.valueOf(tableType));
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean isAsyncClusteringEnabled() {
|
|
||||||
return Boolean.parseBoolean(String.valueOf(UtilHelpers.getConfig(this.configs).getProps()
|
|
||||||
.getOrDefault(HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE.key(), false)));
|
|
||||||
}
|
|
||||||
|
|
||||||
public boolean isInlineClusteringEnabled() {
|
|
||||||
return Boolean.parseBoolean(String.valueOf(UtilHelpers.getConfig(this.configs).getProps()
|
|
||||||
.getOrDefault(HoodieClusteringConfig.INLINE_CLUSTERING.key(), false)));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean equals(Object o) {
|
public boolean equals(Object o) {
|
||||||
if (this == o) {
|
if (this == o) {
|
||||||
@@ -626,6 +628,8 @@ public class HoodieDeltaStreamer implements Serializable {
|
|||||||
LOG.info("Setting Spark Pool name for delta-sync to " + DELTASYNC_POOL_NAME);
|
LOG.info("Setting Spark Pool name for delta-sync to " + DELTASYNC_POOL_NAME);
|
||||||
jssc.setLocalProperty("spark.scheduler.pool", DELTASYNC_POOL_NAME);
|
jssc.setLocalProperty("spark.scheduler.pool", DELTASYNC_POOL_NAME);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.from(props);
|
||||||
try {
|
try {
|
||||||
while (!isShutdownRequested()) {
|
while (!isShutdownRequested()) {
|
||||||
try {
|
try {
|
||||||
@@ -637,7 +641,7 @@ public class HoodieDeltaStreamer implements Serializable {
|
|||||||
HoodieTimeline.COMPACTION_ACTION, scheduledCompactionInstantAndRDD.get().getLeft().get()));
|
HoodieTimeline.COMPACTION_ACTION, scheduledCompactionInstantAndRDD.get().getLeft().get()));
|
||||||
asyncCompactService.get().waitTillPendingAsyncServiceInstantsReducesTo(cfg.maxPendingCompactions);
|
asyncCompactService.get().waitTillPendingAsyncServiceInstantsReducesTo(cfg.maxPendingCompactions);
|
||||||
}
|
}
|
||||||
if (cfg.isAsyncClusteringEnabled()) {
|
if (clusteringConfig.isAsyncClusteringEnabled()) {
|
||||||
Option<String> clusteringInstant = deltaSync.getClusteringInstantOpt();
|
Option<String> clusteringInstant = deltaSync.getClusteringInstantOpt();
|
||||||
if (clusteringInstant.isPresent()) {
|
if (clusteringInstant.isPresent()) {
|
||||||
LOG.info("Scheduled async clustering for instant: " + clusteringInstant.get());
|
LOG.info("Scheduled async clustering for instant: " + clusteringInstant.get());
|
||||||
@@ -727,7 +731,7 @@ public class HoodieDeltaStreamer implements Serializable {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
// start async clustering if required
|
// start async clustering if required
|
||||||
if (cfg.isAsyncClusteringEnabled()) {
|
if (HoodieClusteringConfig.from(props).isAsyncClusteringEnabled()) {
|
||||||
if (asyncClusteringService.isPresent()) {
|
if (asyncClusteringService.isPresent()) {
|
||||||
asyncClusteringService.get().updateWriteClient(writeClient);
|
asyncClusteringService.get().updateWriteClient(writeClient);
|
||||||
} else {
|
} else {
|
||||||
|
|||||||
@@ -77,7 +77,7 @@ public class HoodieMultiTableDeltaStreamer {
|
|||||||
FileSystem fs = FSUtils.getFs(commonPropsFile, jssc.hadoopConfiguration());
|
FileSystem fs = FSUtils.getFs(commonPropsFile, jssc.hadoopConfiguration());
|
||||||
configFolder = configFolder.charAt(configFolder.length() - 1) == '/' ? configFolder.substring(0, configFolder.length() - 1) : configFolder;
|
configFolder = configFolder.charAt(configFolder.length() - 1) == '/' ? configFolder.substring(0, configFolder.length() - 1) : configFolder;
|
||||||
checkIfPropsFileAndConfigFolderExist(commonPropsFile, configFolder, fs);
|
checkIfPropsFileAndConfigFolderExist(commonPropsFile, configFolder, fs);
|
||||||
TypedProperties commonProperties = UtilHelpers.readConfig(fs, new Path(commonPropsFile), new ArrayList<>()).getProps();
|
TypedProperties commonProperties = UtilHelpers.readConfig(fs.getConf(), new Path(commonPropsFile), new ArrayList<String>()).getProps();
|
||||||
//get the tables to be ingested and their corresponding config files from this properties instance
|
//get the tables to be ingested and their corresponding config files from this properties instance
|
||||||
populateTableExecutionContextList(commonProperties, configFolder, fs, config);
|
populateTableExecutionContextList(commonProperties, configFolder, fs, config);
|
||||||
}
|
}
|
||||||
@@ -116,7 +116,7 @@ public class HoodieMultiTableDeltaStreamer {
|
|||||||
String configProp = Constants.INGESTION_PREFIX + database + Constants.DELIMITER + currentTable + Constants.INGESTION_CONFIG_SUFFIX;
|
String configProp = Constants.INGESTION_PREFIX + database + Constants.DELIMITER + currentTable + Constants.INGESTION_CONFIG_SUFFIX;
|
||||||
String configFilePath = properties.getString(configProp, Helpers.getDefaultConfigFilePath(configFolder, database, currentTable));
|
String configFilePath = properties.getString(configProp, Helpers.getDefaultConfigFilePath(configFolder, database, currentTable));
|
||||||
checkIfTableConfigFileExists(configFolder, fs, configFilePath);
|
checkIfTableConfigFileExists(configFolder, fs, configFilePath);
|
||||||
TypedProperties tableProperties = UtilHelpers.readConfig(fs, new Path(configFilePath), new ArrayList<>()).getProps();
|
TypedProperties tableProperties = UtilHelpers.readConfig(fs.getConf(), new Path(configFilePath), new ArrayList<String>()).getProps();
|
||||||
properties.forEach((k, v) -> {
|
properties.forEach((k, v) -> {
|
||||||
if (tableProperties.get(k) == null) {
|
if (tableProperties.get(k) == null) {
|
||||||
tableProperties.setProperty(k.toString(), v.toString());
|
tableProperties.setProperty(k.toString(), v.toString());
|
||||||
|
|||||||
@@ -378,7 +378,7 @@ public class TestHoodieDeltaStreamer extends HoodieDeltaStreamerTestBase {
|
|||||||
@Test
|
@Test
|
||||||
public void testProps() {
|
public void testProps() {
|
||||||
TypedProperties props =
|
TypedProperties props =
|
||||||
new DFSPropertiesConfiguration(dfs, new Path(dfsBasePath + "/" + PROPS_FILENAME_TEST_SOURCE)).getProps();
|
new DFSPropertiesConfiguration(dfs.getConf(), new Path(dfsBasePath + "/" + PROPS_FILENAME_TEST_SOURCE)).getProps();
|
||||||
assertEquals(2, props.getInteger("hoodie.upsert.shuffle.parallelism"));
|
assertEquals(2, props.getInteger("hoodie.upsert.shuffle.parallelism"));
|
||||||
assertEquals("_row_key", props.getString("hoodie.datasource.write.recordkey.field"));
|
assertEquals("_row_key", props.getString("hoodie.datasource.write.recordkey.field"));
|
||||||
assertEquals("org.apache.hudi.utilities.functional.TestHoodieDeltaStreamer$TestGenerator",
|
assertEquals("org.apache.hudi.utilities.functional.TestHoodieDeltaStreamer$TestGenerator",
|
||||||
@@ -498,7 +498,7 @@ public class TestHoodieDeltaStreamer extends HoodieDeltaStreamerTestBase {
|
|||||||
String checkpointProviderClass = "org.apache.hudi.utilities.checkpointing.KafkaConnectHdfsProvider";
|
String checkpointProviderClass = "org.apache.hudi.utilities.checkpointing.KafkaConnectHdfsProvider";
|
||||||
HoodieDeltaStreamer.Config cfg = TestHelpers.makeDropAllConfig(tableBasePath, WriteOperationType.UPSERT);
|
HoodieDeltaStreamer.Config cfg = TestHelpers.makeDropAllConfig(tableBasePath, WriteOperationType.UPSERT);
|
||||||
TypedProperties props =
|
TypedProperties props =
|
||||||
new DFSPropertiesConfiguration(dfs, new Path(dfsBasePath + "/" + PROPS_FILENAME_TEST_SOURCE)).getProps();
|
new DFSPropertiesConfiguration(dfs.getConf(), new Path(dfsBasePath + "/" + PROPS_FILENAME_TEST_SOURCE)).getProps();
|
||||||
props.put("hoodie.deltastreamer.checkpoint.provider.path", bootstrapPath);
|
props.put("hoodie.deltastreamer.checkpoint.provider.path", bootstrapPath);
|
||||||
cfg.initialCheckpointProvider = checkpointProviderClass;
|
cfg.initialCheckpointProvider = checkpointProviderClass;
|
||||||
// create regular kafka connect hdfs dirs
|
// create regular kafka connect hdfs dirs
|
||||||
|
|||||||
Reference in New Issue
Block a user