[HUDI-2362] Add external config file support (#3416)
Co-authored-by: Wenning Ding <wenningd@amazon.com>
This commit is contained in:
@@ -18,6 +18,8 @@
|
||||
|
||||
package org.apache.hudi.streamer;
|
||||
|
||||
import org.apache.hudi.common.config.DFSPropertiesConfiguration;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
@@ -38,8 +40,6 @@ import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* A utility which can incrementally consume data from Kafka and apply it to the target table.
|
||||
* It has the similar functionality with SQL data source except that the source is bind to Kafka
|
||||
@@ -65,7 +65,8 @@ public class HoodieFlinkStreamer {
|
||||
env.setStateBackend(new FsStateBackend(cfg.flinkCheckPointPath));
|
||||
}
|
||||
|
||||
Properties kafkaProps = StreamerUtil.appendKafkaProps(cfg);
|
||||
TypedProperties kafkaProps = DFSPropertiesConfiguration.getGlobalProps();
|
||||
kafkaProps.putAll(StreamerUtil.appendKafkaProps(cfg));
|
||||
|
||||
// Read from kafka source
|
||||
RowType rowType =
|
||||
|
||||
@@ -103,7 +103,7 @@ public class StreamerUtil {
|
||||
}
|
||||
return readConfig(
|
||||
FSUtils.getFs(cfg.propsFilePath, getHadoopConf()),
|
||||
new Path(cfg.propsFilePath), cfg.configs).getConfig();
|
||||
new Path(cfg.propsFilePath), cfg.configs).getProps();
|
||||
}
|
||||
|
||||
public static Schema getSourceSchema(FlinkStreamerConfig cfg) {
|
||||
@@ -128,18 +128,11 @@ public class StreamerUtil {
|
||||
* Read config from properties file (`--props` option) and cmd line (`--hoodie-conf` option).
|
||||
*/
|
||||
public static DFSPropertiesConfiguration readConfig(FileSystem fs, Path cfgPath, List<String> overriddenProps) {
|
||||
DFSPropertiesConfiguration conf;
|
||||
try {
|
||||
conf = new DFSPropertiesConfiguration(cfgPath.getFileSystem(fs.getConf()), cfgPath);
|
||||
} catch (Exception e) {
|
||||
conf = new DFSPropertiesConfiguration();
|
||||
LOG.warn("Unexpected error read props file at :" + cfgPath, e);
|
||||
}
|
||||
|
||||
DFSPropertiesConfiguration conf = new DFSPropertiesConfiguration(fs, cfgPath);
|
||||
try {
|
||||
if (!overriddenProps.isEmpty()) {
|
||||
LOG.info("Adding overridden properties to file properties.");
|
||||
conf.addProperties(new BufferedReader(new StringReader(String.join("\n", overriddenProps))));
|
||||
conf.addPropsFromStream(new BufferedReader(new StringReader(String.join("\n", overriddenProps))));
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException("Unexpected error adding config overrides", ioe);
|
||||
|
||||
Reference in New Issue
Block a user