Skip to content

Commit

Permalink
hide kafka-connecter details for easy use debezium connector (apache#…
Browse files Browse the repository at this point in the history
…3825)

currently we explored too much internal config for debezium connector, this PR is to hide some details and make the config easier.
expected pass integration tests.
  • Loading branch information
jiazhai authored and sijie committed Mar 19, 2019
1 parent 7b2ccd1 commit 8fc4f3c
Show file tree
Hide file tree
Showing 15 changed files with 343 additions and 112 deletions.
1 change: 1 addition & 0 deletions distribution/io/src/assemble/io.xml
Original file line number Diff line number Diff line change
Expand Up @@ -64,5 +64,6 @@
<file><source>${basedir}/../../pulsar-io/canal/target/pulsar-io-canal-${project.version}.nar</source></file>
<file><source>${basedir}/../../pulsar-io/netty/target/pulsar-io-netty-${project.version}.nar</source></file>
<file><source>${basedir}/../../pulsar-io/mongo/target/pulsar-io-mongo-${project.version}.nar</source></file>
<file><source>${basedir}/../../pulsar-io/debezium/mysql/target/pulsar-io-debezium-mysql-${project.version}.nar</source></file>
</files>
</assembly>
104 changes: 104 additions & 0 deletions pulsar-io/debezium/core/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,104 @@
<!--
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.pulsar</groupId>
<artifactId>pulsar-io-debezium</artifactId>
<version>2.4.0-SNAPSHOT</version>
</parent>

<artifactId>pulsar-io-debezium-core</artifactId>
<name>Pulsar IO :: Debezium :: Core</name>

<dependencies>

<dependency>
<groupId>${project.groupId}</groupId>
<artifactId>pulsar-io-core</artifactId>
<version>${project.version}</version>
</dependency>

<dependency>
<groupId>io.debezium</groupId>
<artifactId>debezium-core</artifactId>
<version>${debezium.version}</version>
</dependency>

<dependency>
<groupId>${project.groupId}</groupId>
<artifactId>pulsar-io-kafka-connect-adaptor</artifactId>
<version>${project.version}</version>
</dependency>

<dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>kafka_${scala.binary.version}</artifactId>
<version>${kafka-client.version}</version>
</dependency>

<dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>connect-runtime</artifactId>
<version>${kafka-client.version}</version>
</dependency>

<dependency>
<groupId>${project.groupId}</groupId>
<artifactId>pulsar-client-original</artifactId>
<version>${project.version}</version>
</dependency>

<dependency>
<groupId>${project.groupId}</groupId>
<artifactId>pulsar-broker</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>${project.groupId}</groupId>
<artifactId>managed-ledger-original</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>

<dependency>
<groupId>${project.groupId}</groupId>
<artifactId>pulsar-zookeeper-utils</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>

<dependency>
<groupId>${project.groupId}</groupId>
<artifactId>pulsar-broker</artifactId>
<version>${project.version}</version>
<scope>test</scope>
<type>test-jar</type>
</dependency>

</dependencies>

</project>
59 changes: 59 additions & 0 deletions pulsar-io/debezium/mysql/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
<!--
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.pulsar</groupId>
<artifactId>pulsar-io-debezium</artifactId>
<version>2.4.0-SNAPSHOT</version>
</parent>

<artifactId>pulsar-io-debezium-mysql</artifactId>
<name>Pulsar IO :: Debezium :: mysql</name>

<dependencies>

<dependency>
<groupId>${project.groupId}</groupId>
<artifactId>pulsar-io-debezium-core</artifactId>
<version>${project.version}</version>
</dependency>

<dependency>
<groupId>io.debezium</groupId>
<artifactId>debezium-connector-mysql</artifactId>
<version>${debezium.version}</version>
</dependency>

</dependencies>


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

</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,110 @@
/**
* 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.pulsar.io.debezium.mysql;

import java.util.Map;

import io.debezium.connector.mysql.MySqlConnectorConfig;
import lombok.extern.slf4j.Slf4j;
import org.apache.commons.lang3.StringUtils;
import org.apache.kafka.connect.runtime.TaskConfig;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.io.core.SourceContext;
import org.apache.pulsar.io.debezium.PulsarDatabaseHistory;
import org.apache.pulsar.io.kafka.connect.KafkaConnectSource;
import org.apache.pulsar.io.kafka.connect.PulsarKafkaWorkerConfig;

/**
* A pulsar source that runs
*/
@Slf4j
public class DebeziumMysqlSource extends KafkaConnectSource {
static private final String DEFAULT_TASK = "io.debezium.connector.mysql.MySqlConnectorTask";
static private final String DEFAULT_CONVERTER = "org.apache.kafka.connect.json.JsonConverter";
static private final String DEFAULT_HISTORY = "org.apache.pulsar.io.debezium.PulsarDatabaseHistory";
static private final String DEFAULT_OFFSET_TOPIC = "debezium-mysql-offset-topic";
static private final String DEFAULT_HISTORY_TOPIC = "debezium-mysql-history-topic";

private static void throwExceptionIfConfigNotMatch(Map<String, Object> config,
String key,
String value) throws IllegalArgumentException {
Object orig = config.get(key);
if (orig == null) {
config.put(key, value);
return;
}

// throw exception if value not match
if (!orig.equals(value)) {
throw new IllegalArgumentException("Expected " + value + " but has " + orig);
}
}

private static void setConfigIfNull(Map<String, Object> config, String key, String value) {
Object orig = config.get(key);
if (orig == null) {
config.put(key, value);
}
}

// namespace: tenant/namespace
private static String topicNamespace(SourceContext sourceContext) {
String tenant = sourceContext.getTenant();
String namespace = sourceContext.getNamespace();

return (StringUtils.isEmpty(tenant) ? TopicName.PUBLIC_TENANT : tenant) + "/" +
(StringUtils.isEmpty(namespace) ? TopicName.DEFAULT_NAMESPACE : namespace);
}

@Override
public void open(Map<String, Object> config, SourceContext sourceContext) throws Exception {
// connector task
throwExceptionIfConfigNotMatch(config, TaskConfig.TASK_CLASS_CONFIG, DEFAULT_TASK);

// key.converter
setConfigIfNull(config, PulsarKafkaWorkerConfig.KEY_CONVERTER_CLASS_CONFIG, DEFAULT_CONVERTER);
// value.converter
setConfigIfNull(config, PulsarKafkaWorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, DEFAULT_CONVERTER);

// database.history implementation class
setConfigIfNull(config, MySqlConnectorConfig.DATABASE_HISTORY.name(), DEFAULT_HISTORY);

// database.history.pulsar.service.url, this is set as the value of pulsar.service.url if null.
String serviceUrl = (String) config.get(PulsarKafkaWorkerConfig.PULSAR_SERVICE_URL_CONFIG);
if (serviceUrl == null) {
throw new IllegalArgumentException("Pulsar service URL not provided.");
}
setConfigIfNull(config, PulsarDatabaseHistory.SERVICE_URL.name(), serviceUrl);

String topicNamespace = topicNamespace(sourceContext);
// topic.namespace
setConfigIfNull(config, PulsarKafkaWorkerConfig.TOPIC_NAMESPACE_CONFIG, topicNamespace);

String sourceName = sourceContext.getSourceName();
// database.history.pulsar.topic: history topic name
setConfigIfNull(config, PulsarDatabaseHistory.TOPIC.name(),
topicNamespace + "/" + sourceName + "-" + DEFAULT_HISTORY_TOPIC);
// offset.storage.topic: offset topic name
setConfigIfNull(config, PulsarKafkaWorkerConfig.OFFSET_STORAGE_TOPIC_CONFIG,
topicNamespace + "/" + sourceName + "-" + DEFAULT_OFFSET_TOPIC);

super.open(config, sourceContext);
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,22 @@
#
# 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.
#

name: debezium-mysql
description: Debezium MySql Source
sourceClass: org.apache.pulsar.io.debezium.mysql.DebeziumMysqlSource
Original file line number Diff line number Diff line change
Expand Up @@ -19,17 +19,13 @@

tenant: "test"
namespace: "test-namespace"
name: "debezium-kafka-source"
topicName: "kafka-connect-topic"
archive: "connectors/pulsar-io-kafka-connect-adaptor-2.3.0-SNAPSHOT.nar"
name: "debezium-mysql-source"
topicName: "debezium-mysql-topic"
archive: "connectors/pulsar-io-debezium-mysql-2.4.0-SNAPSHOT.nar"

##autoAck: true
parallelism: 1

configs:
## sourceTask
task.class: "io.debezium.connector.mysql.MySqlConnectorTask"

## config for mysql, docker image: debezium/example-mysql:0.8
database.hostname: "localhost"
database.port: "3306"
Expand All @@ -39,15 +35,9 @@ configs:
database.server.name: "dbserver1"
database.whitelist: "inventory"

database.history: "org.apache.pulsar.io.debezium.PulsarDatabaseHistory"
database.history.pulsar.topic: "history-topic"
database.history.pulsar.service.url: "pulsar://127.0.0.1:6650"
## KEY_CONVERTER_CLASS_CONFIG, VALUE_CONVERTER_CLASS_CONFIG
key.converter: "org.apache.kafka.connect.json.JsonConverter"
value.converter: "org.apache.kafka.connect.json.JsonConverter"
## PULSAR_SERVICE_URL_CONFIG
pulsar.service.url: "pulsar://127.0.0.1:6650"
## OFFSET_STORAGE_TOPIC_CONFIG
offset.storage.topic: "offset-topic"
database.history.pulsar.topic: "mysql-history-topic"
offset.storage.topic: "mysql-offset-topic"


Loading

0 comments on commit 8fc4f3c

Please sign in to comment.