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

[feature][datasource] Main add datasource-pulsar #145

Open
wants to merge 3 commits into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -99,7 +99,7 @@ protected AbstractDataSourceClient() {
.get(pluginName.toUpperCase())
.toString());
} catch (Exception e) {
log.warn("datasource " + pluginName + "is error" + ExceptionUtils.getMessage(e));
log.warn("datasource " + pluginName + " is error" + ExceptionUtils.getMessage(e));
}
Thread.currentThread().setContextClassLoader(contextClassLoader);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ public class DatasourceLoadConfig {
"org.slf4j",
"org.apache.log4j",
"org.apache.seatunnel.api",
"org.apache.seatunnel.datasource",
"org.apache.logging",
"org.apache.commons",
"com.fasterxml.jackson"
Expand Down Expand Up @@ -75,6 +76,8 @@ public class DatasourceLoadConfig {
"org.apache.seatunnel.datasource.plugin.tidb.jdbc.TidbJdbcDataSourceFactory");
classLoaderFactoryName.put(
"KAFKA", "org.apache.seatunnel.datasource.plugin.kafka.KafkaDataSourceFactory");
classLoaderFactoryName.put(
"PULSAR", "org.apache.seatunnel.datasource.plugin.pulsar.PulsarDataSourceFactory");
classLoaderFactoryName.put(
"MYSQL-CDC",
"org.apache.seatunnel.datasource.plugin.cdc.mysql.MysqlCDCDataSourceFactory");
Expand Down Expand Up @@ -108,6 +111,7 @@ public class DatasourceLoadConfig {
classLoaderJarName.put("S3", "datasource-s3-");
classLoaderJarName.put("HIVE", "datasource-hive-");
classLoaderJarName.put("KAFKA", "datasource-kafka-");
classLoaderJarName.put("PULSAR", "datasource-pulsar-");
classLoaderJarName.put("STARROCKS", "datasource-starrocks-");
classLoaderJarName.put("S3-REDSHIFT", "datasource-s3redshift-");
classLoaderJarName.put("JDBC-STARROCKS", "datasource-jdbc-starrocks-");
Expand All @@ -124,6 +128,7 @@ public class DatasourceLoadConfig {
"JDBC-SQLServer",
"JDBC-TiDB",
"Kafka",
"Pulsar",
"MySQL-CDC",
"S3",
"SqlServer-CDC",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,13 +65,16 @@
<artifactId>datasource-jdbc-starrocks</artifactId>
<version>${project.version}</version>
</dependency>

<dependency>
<groupId>org.apache.seatunnel</groupId>
<artifactId>datasource-kafka</artifactId>
<version>${project.version}</version>
</dependency>

<dependency>
<groupId>org.apache.seatunnel</groupId>
<artifactId>datasource-pulsar</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.seatunnel</groupId>
<artifactId>datasource-elasticsearch</artifactId>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,136 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--

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.

-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.seatunnel</groupId>
<artifactId>seatunnel-datasource-plugins</artifactId>
<version>${revision}</version>
</parent>

<artifactId>datasource-pulsar</artifactId>

<properties>
<pulsar.version>3.0.0</pulsar.version>
<jetty.version>4.1.86.Final</jetty.version>
<jackson.version>2.12.6</jackson.version>
<codec.version>1.13</codec.version>
</properties>
<dependencies>
<dependency>
<groupId>org.apache.seatunnel</groupId>
<artifactId>datasource-plugins-api</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
<exclusions>
<exclusion>
<groupId>org.apache.commons</groupId>
<artifactId>common-lang3</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-lang3</artifactId>
</dependency>
<!-- https://mvnrepository.com/artifact/com.google.auto.service/auto-service -->
<dependency>
<groupId>com.google.auto.service</groupId>
<artifactId>auto-service</artifactId>
</dependency>
<dependency>
<groupId>org.apache.seatunnel</groupId>
<artifactId>seatunnel-api</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.pulsar</groupId>
<artifactId>pulsar-client-admin</artifactId>
<version>${pulsar.version}</version>
<exclusions>
<exclusion>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-core</artifactId>
</exclusion>
<exclusion>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-core</artifactId>
</exclusion>
<exclusion>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-annotations</artifactId>
</exclusion>
<exclusion>
<groupId>com.fasterxml.jackson.module</groupId>
<artifactId>jackson-module-jaxb-annotations</artifactId>
</exclusion>
<exclusion>
<groupId>commons-codec</groupId>
<artifactId>commons-codec</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.commons</groupId>
<artifactId>commons-compress</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-core</artifactId>
<version>${jackson.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-annotations</artifactId>
<version>${jackson.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>commons-codec</groupId>
<artifactId>commons-codec</artifactId>
<version>${codec.version}</version>
<scope>test</scope>
</dependency>
</dependencies>

<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-dependency-plugin</artifactId>
<configuration>
<skip>${e2e.dependency.skip}</skip>
<appendOutput>true</appendOutput>
</configuration>
</plugin>

<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-shade-plugin</artifactId>
</plugin>
</plugins>
</build>

<scm>
<tag>1.0.0</tag>
</scm>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,154 @@
/*
* 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.datasource.plugin.pulsar;

import org.apache.seatunnel.api.configuration.util.OptionRule;
import org.apache.seatunnel.datasource.plugin.api.DataSourceChannel;
import org.apache.seatunnel.datasource.plugin.api.DataSourcePluginException;
import org.apache.seatunnel.datasource.plugin.api.model.TableField;

import org.apache.commons.collections4.CollectionUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.pulsar.client.admin.PulsarAdmin;
import org.apache.pulsar.client.api.PulsarClientException;

import lombok.NonNull;
import lombok.extern.slf4j.Slf4j;

import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;

import static com.google.common.base.Preconditions.checkArgument;

@Slf4j
public class PulsarDataSourceChannel implements DataSourceChannel {

private static final String TENANT = "public";
private static final String DATABASE = "public/default";

@Override
public OptionRule getDataSourceOptions(@NonNull String pluginName) {
return PulsarOptionRule.optionRule();
}

@Override
public OptionRule getDatasourceMetadataFieldsByDataSourceName(@NonNull String pluginName) {
return PulsarOptionRule.metadataRule();
}

@Override
public List<String> getTables(
@NonNull String pluginName,
Map<String, String> requestParams,
String database,
Map<String, String> option) {
checkArgument(StringUtils.equalsIgnoreCase(database, DATABASE), "database must be default");
try (PulsarAdmin pulsarAdmin = createPulsarAdmin(requestParams)) {
return pulsarAdmin.topics().getList(database);
} catch (Exception ex) {
throw new DataSourcePluginException(
"check Pulsar connectivity failed, " + ex.getMessage(), ex);
}
}

@Override
public List<String> getDatabases(
@NonNull String pluginName, @NonNull Map<String, String> requestParams) {
try (PulsarAdmin pulsarAdmin = createPulsarAdmin(requestParams)) {
return pulsarAdmin.namespaces().getNamespaces(TENANT);
} catch (Exception ex) {
throw new DataSourcePluginException(
"check Pulsar connectivity failed, " + ex.getMessage(), ex);
}
}

@Override
public boolean checkDataSourceConnectivity(
@NonNull String pluginName, @NonNull Map<String, String> requestParams) {
try (PulsarAdmin pulsarAdmin = createPulsarAdmin(requestParams)) {
// just test the connection
List<String> clusters = pulsarAdmin.clusters().getClusters();
return CollectionUtils.isNotEmpty(clusters);
} catch (Exception ex) {
throw new DataSourcePluginException(
"check Pulsar connectivity failed, " + ex.getMessage(), ex);
}
}

@Override
public List<TableField> getTableFields(
@NonNull String pluginName,
@NonNull Map<String, String> requestParams,
@NonNull String database,
@NonNull String table) {
checkArgument(StringUtils.equalsIgnoreCase(database, DATABASE), "database must be default");
return Collections.emptyList();
}

@Override
public Map<String, List<TableField>> getTableFields(
@NonNull String pluginName,
@NonNull Map<String, String> requestParams,
@NonNull String database,
@NonNull List<String> tables) {
checkArgument(StringUtils.equalsIgnoreCase(database, DATABASE), "database must be default");
return Collections.emptyMap();
}

private PulsarAdmin createPulsarAdmin(Map<String, String> requestParams)
throws PulsarClientException {
return PulsarAdmin.builder()
.serviceHttpUrl(requestParams.get("admin.service-url"))
.loadConf(getConfMap(requestParams))
.build();
}

private HashMap<String, Object> getConfMap(Map<String, String> requestParams) {
HashMap<String, Object> confMap = new HashMap<>();
if (requestParams.size() == 0) {
return confMap;
}
if (requestParams.get("pulsar.config") != null) {
String[] lines = requestParams.get("pulsar.config").split(";");
for (String line : lines) {
String[] keyValue = line.split("=");
if (keyValue.length == 2) {
String key = keyValue[0].trim();
String value = keyValue[1].trim();
confMap.put(key, value);
}
}
}
Set<String> pSet = new HashSet<>();
pSet.add("client.service-url");
pSet.add("admin.service-url");
pSet.add("topic");
pSet.add("topic-pattern");
Map<String, Object> filteredParams =
requestParams.entrySet().stream()
.filter(entry -> !pSet.contains(entry.getKey()))
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
confMap.putAll(filteredParams);
return confMap;
}
}