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][io] Debezium sources: Support loading config from secrets #19004

Merged
merged 1 commit into from
Dec 23, 2022
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
Original file line number Diff line number Diff line change
Expand Up @@ -20,12 +20,14 @@

import io.debezium.relational.HistorizedRelationalDatabaseConnectorConfig;
import java.util.Map;
import lombok.extern.slf4j.Slf4j;
import org.apache.commons.lang3.StringUtils;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.io.core.SourceContext;
import org.apache.pulsar.io.kafka.connect.KafkaConnectSource;
import org.apache.pulsar.io.kafka.connect.PulsarKafkaWorkerConfig;

@Slf4j
public abstract class DebeziumSource extends KafkaConnectSource {
private static final String DEFAULT_CONVERTER = "org.apache.kafka.connect.json.JsonConverter";
private static final String DEFAULT_HISTORY = "org.apache.pulsar.io.debezium.PulsarDatabaseHistory";
Expand Down Expand Up @@ -60,11 +62,25 @@ public static String topicNamespace(SourceContext sourceContext) {
+ (StringUtils.isEmpty(namespace) ? TopicName.DEFAULT_NAMESPACE : namespace);
}

public static void tryLoadingConfigSecret(String secretName, Map<String, Object> config, SourceContext context) {
try {
String secret = context.getSecret(secretName);
if (secret != null) {
config.put(secretName, secret);
alpreu marked this conversation as resolved.
Show resolved Hide resolved
log.info("Config key {} set from secret.", secretName);
}
} catch (Exception e) {
log.warn("Failed to read secret {}.", secretName, e);
}
}

public abstract void setDbConnectorTask(Map<String, Object> config) throws Exception;

@Override
public void open(Map<String, Object> config, SourceContext sourceContext) throws Exception {
setDbConnectorTask(config);
tryLoadingConfigSecret("database.user", config, sourceContext);
tryLoadingConfigSecret("database.password", config, sourceContext);

// key.converter
setConfigIfNull(config, PulsarKafkaWorkerConfig.KEY_CONVERTER_CLASS_CONFIG, DEFAULT_CONVERTER);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@

import java.util.Map;
import org.apache.kafka.connect.runtime.TaskConfig;
import org.apache.pulsar.io.core.SourceContext;
import org.apache.pulsar.io.debezium.DebeziumSource;

/**
Expand All @@ -32,4 +33,11 @@ public class DebeziumMongoDbSource extends DebeziumSource {
public void setDbConnectorTask(Map<String, Object> config) throws Exception {
throwExceptionIfConfigNotMatch(config, TaskConfig.TASK_CLASS_CONFIG, DEFAULT_TASK);
}

@Override
public void open(Map<String, Object> config, SourceContext sourceContext) throws Exception {
tryLoadingConfigSecret("mongodb.user", config, sourceContext);
tryLoadingConfigSecret("mongodb.password", config, sourceContext);
super.open(config, sourceContext);
}
}