This is an automated email from the ASF dual-hosted git repository.
wanghailin pushed a commit to branch dev
in repository https://gitbox.apache.org/repos/asf/seatunnel.git
The following commit(s) were added to refs/heads/dev by this push:
new e2578a5b4d Fix hive load hive_site_path and hdfs_site_path too late
(#7017)
e2578a5b4d is described below
commit e2578a5b4d54ba8d713d051d8b0378c43a860f74
Author: Eric <[email protected]>
AuthorDate: Mon Jun 24 21:39:32 2024 +0800
Fix hive load hive_site_path and hdfs_site_path too late (#7017)
---
.../seatunnel/hive/storage/AbstractStorage.java | 28 ++++++++++++++++++++++
1 file changed, 28 insertions(+)
diff --git
a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/storage/AbstractStorage.java
b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/storage/AbstractStorage.java
index 0e61226b1e..22f9e61880 100644
---
a/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/storage/AbstractStorage.java
+++
b/seatunnel-connectors-v2/connector-hive/src/main/java/org/apache/seatunnel/connectors/seatunnel/hive/storage/AbstractStorage.java
@@ -23,6 +23,7 @@ import
org.apache.seatunnel.shade.com.typesafe.config.ConfigValueFactory;
import org.apache.seatunnel.api.configuration.Option;
import org.apache.seatunnel.api.configuration.Options;
import org.apache.seatunnel.api.configuration.ReadonlyConfig;
+import org.apache.seatunnel.connectors.seatunnel.file.config.BaseSinkConfig;
import org.apache.seatunnel.connectors.seatunnel.hive.config.HiveConfig;
import org.apache.commons.lang3.StringUtils;
@@ -31,6 +32,7 @@ import org.apache.hadoop.conf.Configuration;
import com.google.common.collect.ImmutableList;
import lombok.extern.slf4j.Slf4j;
+import java.io.File;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Paths;
@@ -80,6 +82,32 @@ public abstract class AbstractStorage implements Storage {
}
});
}
+ readonlyConfig
+ .getOptional(BaseSinkConfig.HDFS_SITE_PATH)
+ .ifPresent(
+ hdfsSitePath -> {
+ try {
+ configuration.addResource(new
File(hdfsSitePath).toURI().toURL());
+ } catch (IOException e) {
+ log.warn(
+ "Error adding Hadoop resource {},
resource was not added",
+ hdfsSitePath,
+ e);
+ }
+ });
+ readonlyConfig
+ .getOptional(HiveConfig.HIVE_SITE_PATH)
+ .ifPresent(
+ hiveSitePath -> {
+ try {
+ configuration.addResource(new
File(hiveSitePath).toURI().toURL());
+ } catch (IOException e) {
+ log.warn(
+ "Error adding Hadoop resource {},
resource was not added",
+ hiveSitePath,
+ e);
+ }
+ });
// Try to load from hadoopConf
Optional<Map<String, String>> hadoopConf =
readonlyConfig.getOptional(HiveConfig.HADOOP_CONF);