|
| 1 | +/** |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, |
| 13 | + * software distributed under the License is distributed on an |
| 14 | + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
| 15 | + * KIND, either express or implied. See the License for the |
| 16 | + * specific language governing permissions and limitations |
| 17 | + * under the License. |
| 18 | + */ |
| 19 | +package org.apache.pulsar.io.debezium.mysql; |
| 20 | + |
| 21 | +import java.util.Map; |
| 22 | + |
| 23 | +import io.debezium.connector.mysql.MySqlConnectorConfig; |
| 24 | +import lombok.extern.slf4j.Slf4j; |
| 25 | +import org.apache.commons.lang3.StringUtils; |
| 26 | +import org.apache.kafka.connect.runtime.TaskConfig; |
| 27 | +import org.apache.pulsar.common.naming.TopicName; |
| 28 | +import org.apache.pulsar.io.core.SourceContext; |
| 29 | +import org.apache.pulsar.io.debezium.PulsarDatabaseHistory; |
| 30 | +import org.apache.pulsar.io.kafka.connect.KafkaConnectSource; |
| 31 | +import org.apache.pulsar.io.kafka.connect.PulsarKafkaWorkerConfig; |
| 32 | + |
| 33 | +/** |
| 34 | + * A pulsar source that runs |
| 35 | + */ |
| 36 | +@Slf4j |
| 37 | +public class DebeziumMysqlSource extends KafkaConnectSource { |
| 38 | + static private final String DEFAULT_TASK = "io.debezium.connector.mysql.MySqlConnectorTask"; |
| 39 | + static private final String DEFAULT_CONVERTER = "org.apache.kafka.connect.json.JsonConverter"; |
| 40 | + static private final String DEFAULT_HISTORY = "org.apache.pulsar.io.debezium.PulsarDatabaseHistory"; |
| 41 | + static private final String DEFAULT_OFFSET_TOPIC = "debezium-mysql-offset-topic"; |
| 42 | + static private final String DEFAULT_HISTORY_TOPIC = "debezium-mysql-history-topic"; |
| 43 | + |
| 44 | + private static void throwExceptionIfConfigNotMatch(Map<String, Object> config, |
| 45 | + String key, |
| 46 | + String value) throws IllegalArgumentException { |
| 47 | + Object orig = config.get(key); |
| 48 | + if (orig == null) { |
| 49 | + config.put(key, value); |
| 50 | + return; |
| 51 | + } |
| 52 | + |
| 53 | + // throw exception if value not match |
| 54 | + if (!orig.equals(value)) { |
| 55 | + throw new IllegalArgumentException("Expected " + value + " but has " + orig); |
| 56 | + } |
| 57 | + } |
| 58 | + |
| 59 | + private static void setConfigIfNull(Map<String, Object> config, String key, String value) { |
| 60 | + Object orig = config.get(key); |
| 61 | + if (orig == null) { |
| 62 | + config.put(key, value); |
| 63 | + } |
| 64 | + } |
| 65 | + |
| 66 | + // namespace: tenant/namespace |
| 67 | + private static String topicNamespace(SourceContext sourceContext) { |
| 68 | + String tenant = sourceContext.getTenant(); |
| 69 | + String namespace = sourceContext.getNamespace(); |
| 70 | + |
| 71 | + return (StringUtils.isEmpty(tenant) ? TopicName.PUBLIC_TENANT : tenant) + "/" + |
| 72 | + (StringUtils.isEmpty(namespace) ? TopicName.DEFAULT_NAMESPACE : namespace); |
| 73 | + } |
| 74 | + |
| 75 | + @Override |
| 76 | + public void open(Map<String, Object> config, SourceContext sourceContext) throws Exception { |
| 77 | + // connector task |
| 78 | + throwExceptionIfConfigNotMatch(config, TaskConfig.TASK_CLASS_CONFIG, DEFAULT_TASK); |
| 79 | + |
| 80 | + // key.converter |
| 81 | + setConfigIfNull(config, PulsarKafkaWorkerConfig.KEY_CONVERTER_CLASS_CONFIG, DEFAULT_CONVERTER); |
| 82 | + // value.converter |
| 83 | + setConfigIfNull(config, PulsarKafkaWorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, DEFAULT_CONVERTER); |
| 84 | + |
| 85 | + // database.history implementation class |
| 86 | + setConfigIfNull(config, MySqlConnectorConfig.DATABASE_HISTORY.name(), DEFAULT_HISTORY); |
| 87 | + |
| 88 | + // database.history.pulsar.service.url, this is set as the value of pulsar.service.url if null. |
| 89 | + String serviceUrl = (String) config.get(PulsarKafkaWorkerConfig.PULSAR_SERVICE_URL_CONFIG); |
| 90 | + if (serviceUrl == null) { |
| 91 | + throw new IllegalArgumentException("Pulsar service URL not provided."); |
| 92 | + } |
| 93 | + setConfigIfNull(config, PulsarDatabaseHistory.SERVICE_URL.name(), serviceUrl); |
| 94 | + |
| 95 | + String topicNamespace = topicNamespace(sourceContext); |
| 96 | + // topic.namespace |
| 97 | + setConfigIfNull(config, PulsarKafkaWorkerConfig.TOPIC_NAMESPACE_CONFIG, topicNamespace); |
| 98 | + |
| 99 | + String sourceName = sourceContext.getSourceName(); |
| 100 | + // database.history.pulsar.topic: history topic name |
| 101 | + setConfigIfNull(config, PulsarDatabaseHistory.TOPIC.name(), |
| 102 | + topicNamespace + "/" + sourceName + "-" + DEFAULT_HISTORY_TOPIC); |
| 103 | + // offset.storage.topic: offset topic name |
| 104 | + setConfigIfNull(config, PulsarKafkaWorkerConfig.OFFSET_STORAGE_TOPIC_CONFIG, |
| 105 | + topicNamespace + "/" + sourceName + "-" + DEFAULT_OFFSET_TOPIC); |
| 106 | + |
| 107 | + super.open(config, sourceContext); |
| 108 | + } |
| 109 | + |
| 110 | +} |
0 commit comments