Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[Improve][Connector-V2] Support hadoop ha and kerberos for paimon sink #6585

Merged
merged 6 commits into from
Apr 8, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
46 changes: 45 additions & 1 deletion docs/en/connector-v2/sink/Paimon.md
Original file line number Diff line number Diff line change
Expand Up @@ -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 | - | |
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You should be compatible with older versions

Copy link
Contributor Author

@dailai dailai Mar 27, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You should be compatible with older versions

Yes, the code is compatible with older versions. I've updated the docs.

| 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

Expand Down Expand Up @@ -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
Expand Down
46 changes: 45 additions & 1 deletion docs/zh/connector-v2/sink/Paimon.md
Original file line number Diff line number Diff line change
Expand Up @@ -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'文件配置 |

## 示例

Expand Down Expand Up @@ -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
Expand Down
9 changes: 8 additions & 1 deletion seatunnel-connectors-v2/connector-paimon/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@
<name>SeaTunnel : Connectors V2 : Paimon</name>

<properties>
<paimon.version>0.6.0-incubating</paimon.version>
<paimon.version>0.7.0-incubating</paimon.version>
</properties>

<dependencies>
Expand All @@ -46,6 +46,13 @@
<version>${paimon.version}</version>
</dependency>

<dependency>
<groupId>org.apache.seatunnel</groupId>
<artifactId>seatunnel-guava</artifactId>
<version>${project.version}</version>
<classifier>optional</classifier>
</dependency>

<dependency>
<groupId>org.apache.seatunnel</groupId>
<artifactId>seatunnel-hadoop3-3.1.4-uber</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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<String, String> optionsMap = new HashMap<>();
final Map<String, String> 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<Map.Entry<String, String>> entryIterator = configuration.iterator();
while (entryIterator.hasNext()) {
Map.Entry<String, String> 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");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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<String> WAREHOUSE =
Expand Down Expand Up @@ -61,9 +74,54 @@ public class PaimonConfig implements Serializable {
.noDefaultValue()
.withDescription("The read columns of the flink table store");

@Deprecated
public static final Option<String> HDFS_SITE_PATH =
Options.key("hdfs_site_path")
.stringType()
.noDefaultValue()
.withDescription("The file path of hdfs-site.xml");

public static final Option<Map<String, String>> HADOOP_CONF =
Options.key("paimon.hadoop.conf")
.mapType()
.defaultValue(new HashMap<>())
.withDescription("Properties in hadoop conf");

public static final Option<String> HADOOP_CONF_PATH =
Options.key("paimon.hadoop.conf-path")
.stringType()
.noDefaultValue()
.withDescription(
"The specified loading path for the 'core-site.xml', 'hdfs-site.xml', 'hive-site.xml' files");

protected String catalogName;
protected String warehouse;
protected String namespace;
protected String table;
protected String hdfsSitePath;
protected Map<String, String> hadoopConfProps;
protected String hadoopConfPath;

public PaimonConfig(ReadonlyConfig readonlyConfig) {
this.catalogName = checkArgumentNotNull(readonlyConfig.get(CATALOG_NAME));
this.warehouse = checkArgumentNotNull(readonlyConfig.get(WAREHOUSE));
this.namespace = checkArgumentNotNull(readonlyConfig.get(DATABASE));
this.table = checkArgumentNotNull(readonlyConfig.get(TABLE));
this.hdfsSitePath = readonlyConfig.get(HDFS_SITE_PATH);
this.hadoopConfProps = readonlyConfig.get(HADOOP_CONF);
this.hadoopConfPath = readonlyConfig.get(HADOOP_CONF_PATH);
}

protected <T> T checkArgumentNotNull(T argument) {
checkNotNull(argument);
return argument;
}

@VisibleForTesting
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we remove this annotation? It is not intended for testing purposes.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@Carl-Zhou-CN
This annotation is meaningful to prevent method permissions from being changed.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok, it's my oversight

public static List<String> stringToList(String value, String regex) {
if (value == null || value.isEmpty()) {
return ImmutableList.of();
}
return Arrays.stream(value.split(regex)).map(String::trim).collect(toList());
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
/*
* 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.
*/

package org.apache.seatunnel.connectors.seatunnel.paimon.config;

import org.apache.hadoop.conf.Configuration;

import java.io.Serializable;

/** Can serializable */
public class PaimonHadoopConfiguration extends Configuration implements Serializable {}
Loading
Loading