[HUDI-3104] Kafka-connect support of hadoop config environments and properties (#4451)
This commit is contained in:
@@ -49,9 +49,14 @@ import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.FileVisitOption;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.Paths;
|
||||
import java.security.MessageDigest;
|
||||
import java.security.NoSuchAlgorithmException;
|
||||
import java.util.Arrays;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
@@ -65,6 +70,52 @@ public class KafkaConnectUtils {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(KafkaConnectUtils.class);
|
||||
private static final String HOODIE_CONF_PREFIX = "hoodie.";
|
||||
public static final String HADOOP_CONF_DIR = "HADOOP_CONF_DIR";
|
||||
public static final String HADOOP_HOME = "HADOOP_HOME";
|
||||
private static final List<Path> DEFAULT_HADOOP_CONF_FILES;
|
||||
|
||||
static {
|
||||
DEFAULT_HADOOP_CONF_FILES = new ArrayList<>();
|
||||
try {
|
||||
String hadoopConfigPath = System.getenv(HADOOP_CONF_DIR);
|
||||
String hadoopHomePath = System.getenv(HADOOP_HOME);
|
||||
DEFAULT_HADOOP_CONF_FILES.addAll(getHadoopConfigFiles(hadoopConfigPath, hadoopHomePath));
|
||||
if (!DEFAULT_HADOOP_CONF_FILES.isEmpty()) {
|
||||
LOG.info(String.format("Found Hadoop default config files %s", DEFAULT_HADOOP_CONF_FILES));
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.error("An error occurred while getting the default Hadoop configuration. "
|
||||
+ "Please use hadoop.conf.dir or hadoop.home to configure Hadoop environment variables", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get hadoop config files by HADOOP_CONF_DIR or HADOOP_HOME
|
||||
*/
|
||||
public static List<Path> getHadoopConfigFiles(String hadoopConfigPath, String hadoopHomePath)
|
||||
throws IOException {
|
||||
List<Path> hadoopConfigFiles = new ArrayList<>();
|
||||
if (!StringUtils.isNullOrEmpty(hadoopConfigPath)) {
|
||||
hadoopConfigFiles.addAll(walkTreeForXml(Paths.get(hadoopConfigPath)));
|
||||
}
|
||||
if (hadoopConfigFiles.isEmpty() && !StringUtils.isNullOrEmpty(hadoopHomePath)) {
|
||||
hadoopConfigFiles.addAll(walkTreeForXml(Paths.get(hadoopHomePath, "etc", "hadoop")));
|
||||
}
|
||||
return hadoopConfigFiles;
|
||||
}
|
||||
|
||||
/**
|
||||
* Files walk to find xml
|
||||
*/
|
||||
private static List<Path> walkTreeForXml(Path basePath) throws IOException {
|
||||
if (Files.notExists(basePath)) {
|
||||
return new ArrayList<>();
|
||||
}
|
||||
return Files.walk(basePath, FileVisitOption.FOLLOW_LINKS)
|
||||
.filter(path -> path.toFile().isFile())
|
||||
.filter(path -> path.toString().endsWith(".xml"))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public static int getLatestNumPartitions(String bootstrapServers, String topicName) {
|
||||
Properties props = new Properties();
|
||||
@@ -89,6 +140,23 @@ public class KafkaConnectUtils {
|
||||
*/
|
||||
public static Configuration getDefaultHadoopConf(KafkaConnectConfigs connectConfigs) {
|
||||
Configuration hadoopConf = new Configuration();
|
||||
|
||||
// add hadoop config files
|
||||
if (!StringUtils.isNullOrEmpty(connectConfigs.getHadoopConfDir())
|
||||
|| !StringUtils.isNullOrEmpty(connectConfigs.getHadoopConfHome())) {
|
||||
try {
|
||||
List<Path> configFiles = getHadoopConfigFiles(connectConfigs.getHadoopConfDir(),
|
||||
connectConfigs.getHadoopConfHome());
|
||||
configFiles.forEach(f ->
|
||||
hadoopConf.addResource(new org.apache.hadoop.fs.Path(f.toAbsolutePath().toUri())));
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException("Failed to read hadoop configuration!", e);
|
||||
}
|
||||
} else {
|
||||
DEFAULT_HADOOP_CONF_FILES.forEach(f ->
|
||||
hadoopConf.addResource(new org.apache.hadoop.fs.Path(f.toAbsolutePath().toUri())));
|
||||
}
|
||||
|
||||
connectConfigs.getProps().keySet().stream().filter(prop -> {
|
||||
// In order to prevent printing unnecessary warn logs, here filter out the hoodie
|
||||
// configuration items before passing to hadoop/hive configs
|
||||
|
||||
@@ -93,6 +93,17 @@ public class KafkaConnectConfigs extends HoodieConfig {
|
||||
.defaultValue(true)
|
||||
.withDocumentation("Commit even when some records failed to be written");
|
||||
|
||||
// Reference https://docs.confluent.io/kafka-connect-hdfs/current/configuration_options.html#hdfs
|
||||
public static final ConfigProperty<String> HADOOP_CONF_DIR = ConfigProperty
|
||||
.key("hadoop.conf.dir")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("The Hadoop configuration directory.");
|
||||
|
||||
public static final ConfigProperty<String> HADOOP_HOME = ConfigProperty
|
||||
.key("hadoop.home")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("The Hadoop home directory.");
|
||||
|
||||
protected KafkaConnectConfigs() {
|
||||
super();
|
||||
}
|
||||
@@ -145,6 +156,14 @@ public class KafkaConnectConfigs extends HoodieConfig {
|
||||
return getBoolean(ALLOW_COMMIT_ON_ERRORS);
|
||||
}
|
||||
|
||||
public String getHadoopConfDir() {
|
||||
return getString(HADOOP_CONF_DIR);
|
||||
}
|
||||
|
||||
public String getHadoopConfHome() {
|
||||
return getString(HADOOP_HOME);
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
|
||||
protected final KafkaConnectConfigs connectConfigs = new KafkaConnectConfigs();
|
||||
@@ -185,6 +204,16 @@ public class KafkaConnectConfigs extends HoodieConfig {
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withHadoopConfDir(String hadoopConfDir) {
|
||||
connectConfigs.setValue(HADOOP_CONF_DIR, String.valueOf(hadoopConfDir));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withHadoopHome(String hadoopHome) {
|
||||
connectConfigs.setValue(HADOOP_HOME, String.valueOf(hadoopHome));
|
||||
return this;
|
||||
}
|
||||
|
||||
protected void setDefaults() {
|
||||
// Check for mandatory properties
|
||||
connectConfigs.setDefaults(KafkaConnectConfigs.class.getName());
|
||||
|
||||
Reference in New Issue
Block a user