diff --git a/docs/en/connector-v2/sink/Paimon.md b/docs/en/connector-v2/sink/Paimon.md
index 707a0dc0dbd..e0ce1350bb9 100644
--- a/docs/en/connector-v2/sink/Paimon.md
+++ b/docs/en/connector-v2/sink/Paimon.md
@@ -17,12 +17,14 @@ Sink connector for Apache Paimon. It can support cdc mode 、auto create table.
| warehouse | String | Yes | - | Paimon warehouse path |
| database | String | Yes | - | The database you want to access |
| table | String | Yes | - | The table you want to access |
-| hdfs_site_path | String | No | - | |
+| hdfs_site_path | String | No | - | The path of hdfs-site.xml |
| schema_save_mode | Enum | No | CREATE_SCHEMA_WHEN_NOT_EXIST | The schema save mode |
| data_save_mode | Enum | No | APPEND_DATA | The data save mode |
| paimon.table.primary-keys | String | No | - | Default comma-separated list of columns (primary key) that identify a row in tables.(Notice: The partition field needs to be included in the primary key fields) |
| paimon.table.partition-keys | String | No | - | Default comma-separated list of partition fields to use when creating tables. |
| paimon.table.write-props | Map | No | - | Properties passed through to paimon table initialization, [reference](https://paimon.apache.org/docs/0.6/maintenance/configurations/#coreoptions). |
+| paimon.hadoop.conf | Map | No | - | Properties in hadoop conf |
+| paimon.hadoop.conf-path | String | No | - | The specified loading path for the 'core-site.xml', 'hdfs-site.xml', 'hive-site.xml' files |
## Examples
@@ -57,6 +59,48 @@ sink {
}
```
+### Single table(Specify hadoop HA config and kerberos config)
+
+```hocon
+env {
+ parallelism = 1
+ job.mode = "STREAMING"
+ checkpoint.interval = 5000
+}
+
+source {
+ Mysql-CDC {
+ base-url = "jdbc:mysql://127.0.0.1:3306/seatunnel"
+ username = "root"
+ password = "******"
+ table-names = ["seatunnel.role"]
+ }
+}
+
+transform {
+}
+
+sink {
+ Paimon {
+ catalog_name="seatunnel_test"
+ warehouse="hdfs:///tmp/seatunnel/paimon/hadoop-sink/"
+ database="seatunnel"
+ table="role"
+ paimon.hadoop.conf = {
+ fs.defaultFS = "hdfs://nameservice1"
+ dfs.nameservices = "nameservice1"
+ dfs.ha.namenodes.nameservice1 = "nn1,nn2"
+ dfs.namenode.rpc-address.nameservice1.nn1 = "hadoop03:8020"
+ dfs.namenode.rpc-address.nameservice1.nn2 = "hadoop04:8020"
+ dfs.client.failover.proxy.provider.nameservice1 = "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider"
+ dfs.client.use.datanode.hostname = "true"
+ security.kerberos.login.principal = "your-kerberos-principal"
+ security.kerberos.login.keytab = "your-kerberos-keytab-path"
+ }
+ }
+}
+```
+
### Single table with write props of paimon
```hocon
diff --git a/docs/zh/connector-v2/sink/Paimon.md b/docs/zh/connector-v2/sink/Paimon.md
index 306bc12b562..ef22add620d 100644
--- a/docs/zh/connector-v2/sink/Paimon.md
+++ b/docs/zh/connector-v2/sink/Paimon.md
@@ -17,12 +17,14 @@ Apache Paimon数据连接器。支持cdc写以及自动建表。
| warehouse | 字符串 | 是 | - | Paimon warehouse路径 |
| database | 字符串 | 是 | - | 数据库名称 |
| table | 字符串 | 是 | - | 表名 |
-| hdfs_site_path | 字符串 | 否 | - | |
+| hdfs_site_path | 字符串 | 否 | - | hdfs-site.xml文件路径 |
| schema_save_mode | 枚举 | 否 | CREATE_SCHEMA_WHEN_NOT_EXIST | Schema保存模式 |
| data_save_mode | 枚举 | 否 | APPEND_DATA | 数据保存模式 |
| paimon.table.primary-keys | 字符串 | 否 | - | 主键字段列表,联合主键使用逗号分隔(注意:分区字段需要包含在主键字段中) |
| paimon.table.partition-keys | 字符串 | 否 | - | 分区字段列表,多字段使用逗号分隔 |
| paimon.table.write-props | Map | 否 | - | Paimon表初始化指定的属性, [参考](https://paimon.apache.org/docs/0.6/maintenance/configurations/#coreoptions) |
+| paimon.hadoop.conf | Map | 否 | - | Hadoop配置文件属性信息 |
+| paimon.hadoop.conf-path | 字符串 | 否 | - | Hadoop配置文件目录,用于加载'core-site.xml', 'hdfs-site.xml', 'hive-site.xml'文件配置 |
## 示例
@@ -57,6 +59,48 @@ sink {
}
```
+### 单表(指定hadoop HA配置和kerberos配置)
+
+```hocon
+env {
+ parallelism = 1
+ job.mode = "STREAMING"
+ checkpoint.interval = 5000
+}
+
+source {
+ Mysql-CDC {
+ base-url = "jdbc:mysql://127.0.0.1:3306/seatunnel"
+ username = "root"
+ password = "******"
+ table-names = ["seatunnel.role"]
+ }
+}
+
+transform {
+}
+
+sink {
+ Paimon {
+ catalog_name="seatunnel_test"
+ warehouse="hdfs:///tmp/seatunnel/paimon/hadoop-sink/"
+ database="seatunnel"
+ table="role"
+ paimon.hadoop.conf = {
+ fs.defaultFS = "hdfs://nameservice1"
+ dfs.nameservices = "nameservice1"
+ dfs.ha.namenodes.nameservice1 = "nn1,nn2"
+ dfs.namenode.rpc-address.nameservice1.nn1 = "hadoop03:8020"
+ dfs.namenode.rpc-address.nameservice1.nn2 = "hadoop04:8020"
+ dfs.client.failover.proxy.provider.nameservice1 = "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider"
+ dfs.client.use.datanode.hostname = "true"
+ security.kerberos.login.principal = "your-kerberos-principal"
+ security.kerberos.login.keytab = "your-kerberos-keytab-path"
+ }
+ }
+}
+```
+
### 指定paimon的写属性的单表
```hocon
diff --git a/seatunnel-connectors-v2/connector-paimon/pom.xml b/seatunnel-connectors-v2/connector-paimon/pom.xml
index 499165ea6fb..bc0756ecdc4 100644
--- a/seatunnel-connectors-v2/connector-paimon/pom.xml
+++ b/seatunnel-connectors-v2/connector-paimon/pom.xml
@@ -30,7 +30,7 @@
SeaTunnel : Connectors V2 : Paimon
- 0.6.0-incubating
+ 0.7.0-incubating
@@ -46,6 +46,13 @@
${paimon.version}
+
+ org.apache.seatunnel
+ seatunnel-guava
+ ${project.version}
+ optional
+
+
org.apache.seatunnel
seatunnel-hadoop3-3.1.4-uber
diff --git a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalogFactory.java b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalogFactory.java
index b8c8eb10880..9994df1d86f 100644
--- a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalogFactory.java
+++ b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalogFactory.java
@@ -51,7 +51,9 @@ public OptionRule optionRule() {
PaimonSinkConfig.DATA_SAVE_MODE,
PaimonSinkConfig.PRIMARY_KEYS,
PaimonSinkConfig.PARTITION_KEYS,
- PaimonSinkConfig.WRITE_PROPS)
+ PaimonSinkConfig.WRITE_PROPS,
+ PaimonSinkConfig.HADOOP_CONF,
+ PaimonSinkConfig.HADOOP_CONF_PATH)
.build();
}
}
diff --git a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalogLoader.java b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalogLoader.java
index bec66dbe3f2..1c00e6dc3b8 100644
--- a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalogLoader.java
+++ b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/catalog/PaimonCatalogLoader.java
@@ -17,11 +17,14 @@
package org.apache.seatunnel.connectors.seatunnel.paimon.catalog;
+import org.apache.seatunnel.connectors.seatunnel.paimon.config.PaimonHadoopConfiguration;
import org.apache.seatunnel.connectors.seatunnel.paimon.config.PaimonSinkConfig;
+import org.apache.seatunnel.connectors.seatunnel.paimon.exception.PaimonConnectorErrorCode;
+import org.apache.seatunnel.connectors.seatunnel.paimon.exception.PaimonConnectorException;
+import org.apache.seatunnel.connectors.seatunnel.paimon.security.PaimonSecurityContext;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
import org.apache.paimon.catalog.Catalog;
import org.apache.paimon.catalog.CatalogContext;
import org.apache.paimon.catalog.CatalogFactory;
@@ -31,31 +34,66 @@
import java.io.Serializable;
import java.util.HashMap;
+import java.util.Iterator;
import java.util.Map;
import static org.apache.seatunnel.connectors.seatunnel.paimon.config.PaimonConfig.WAREHOUSE;
@Slf4j
public class PaimonCatalogLoader implements Serializable {
- private PaimonSinkConfig config;
+ /** hdfs uri is required */
+ private static final String HDFS_DEF_FS_NAME = "fs.defaultFS";
- public PaimonCatalogLoader(PaimonSinkConfig config) {
- this.config = config;
+ private static final String HDFS_PREFIX = "hdfs://";
+ /** ********* Hdfs constants ************* */
+ private static final String HDFS_IMPL = "org.apache.hadoop.hdfs.DistributedFileSystem";
+
+ private static final String HDFS_IMPL_KEY = "fs.hdfs.impl";
+
+ private String warehouse;
+
+ private PaimonHadoopConfiguration paimonHadoopConfiguration;
+
+ public PaimonCatalogLoader(PaimonSinkConfig paimonSinkConfig) {
+ this.warehouse = paimonSinkConfig.getWarehouse();
+ this.paimonHadoopConfiguration = PaimonSecurityContext.loadHadoopConfig(paimonSinkConfig);
}
public Catalog loadCatalog() {
// When using the seatunel engine, set the current class loader to prevent loading failures
Thread.currentThread().setContextClassLoader(PaimonCatalogLoader.class.getClassLoader());
- final String warehouse = config.getWarehouse();
- final Map optionsMap = new HashMap<>();
+ final Map optionsMap = new HashMap<>(1);
optionsMap.put(WAREHOUSE.key(), warehouse);
final Options options = Options.fromMap(optionsMap);
- final Configuration hadoopConf = new Configuration();
- String hdfsSitePathOptional = config.getHdfsSitePath();
- if (StringUtils.isNotBlank(hdfsSitePathOptional)) {
- hadoopConf.addResource(new Path(hdfsSitePathOptional));
+ if (warehouse.startsWith(HDFS_PREFIX)) {
+ checkConfiguration(paimonHadoopConfiguration, HDFS_DEF_FS_NAME);
+ paimonHadoopConfiguration.set(HDFS_IMPL_KEY, HDFS_IMPL);
+ }
+ PaimonSecurityContext.shouldEnableKerberos(paimonHadoopConfiguration);
+ final CatalogContext catalogContext =
+ CatalogContext.create(options, paimonHadoopConfiguration);
+ try {
+ return PaimonSecurityContext.runSecured(
+ () -> CatalogFactory.createCatalog(catalogContext));
+ } catch (Exception e) {
+ throw new PaimonConnectorException(
+ PaimonConnectorErrorCode.LOAD_CATALOG,
+ "Failed to perform SecurityContext.runSecured",
+ e);
+ }
+ }
+
+ void checkConfiguration(Configuration configuration, String key) {
+ Iterator> entryIterator = configuration.iterator();
+ while (entryIterator.hasNext()) {
+ Map.Entry entry = entryIterator.next();
+ if (entry.getKey().equals(key)) {
+ if (StringUtils.isBlank(entry.getValue())) {
+ throw new IllegalArgumentException("The value of" + key + " is required");
+ }
+ return;
+ }
}
- final CatalogContext catalogContext = CatalogContext.create(options, hadoopConf);
- return CatalogFactory.createCatalog(catalogContext);
+ throw new IllegalArgumentException(key + " is required");
}
}
diff --git a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/config/PaimonConfig.java b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/config/PaimonConfig.java
index 0396e6223af..01207570837 100644
--- a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/config/PaimonConfig.java
+++ b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/config/PaimonConfig.java
@@ -17,18 +17,31 @@
package org.apache.seatunnel.connectors.seatunnel.paimon.config;
+import org.apache.seatunnel.shade.com.google.common.annotations.VisibleForTesting;
+import org.apache.seatunnel.shade.com.google.common.collect.ImmutableList;
+
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.api.sink.SeaTunnelSink;
import org.apache.seatunnel.api.source.SeaTunnelSource;
+import lombok.Getter;
+
import java.io.Serializable;
+import java.util.Arrays;
+import java.util.HashMap;
import java.util.List;
+import java.util.Map;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.seatunnel.shade.com.google.common.base.Preconditions.checkNotNull;
/**
* Utility class to store configuration options, used by {@link SeaTunnelSource} and {@link
* SeaTunnelSink}.
*/
+@Getter
public class PaimonConfig implements Serializable {
public static final Option WAREHOUSE =
@@ -61,9 +74,54 @@ public class PaimonConfig implements Serializable {
.noDefaultValue()
.withDescription("The read columns of the flink table store");
+ @Deprecated
public static final Option HDFS_SITE_PATH =
Options.key("hdfs_site_path")
.stringType()
.noDefaultValue()
.withDescription("The file path of hdfs-site.xml");
+
+ public static final Option