diff --git a/build.gradle b/build.gradle index b03cebe34d..db2552e774 100644 --- a/build.gradle +++ b/build.gradle @@ -89,7 +89,7 @@ subprojects { } dependencies { implementation platform('com.fasterxml.jackson:jackson-bom:2.15.0') - implementation platform('org.eclipse.jetty:jetty-bom:11.0.17') + implementation platform('org.eclipse.jetty:jetty-bom:9.4.53.v20231009') implementation platform('io.micrometer:micrometer-bom:1.10.5') implementation libs.guava.core implementation libs.slf4j.api @@ -154,13 +154,13 @@ subprojects { } implementation('org.eclipse.jetty:http2-common') { version { - require '11.0.17' + require '9.4.53.v20231009' } because 'Fixes CVE-2023-44487' } implementation('org.eclipse.jetty:http2-server') { version { - require '11.0.17' + require '9.4.53.v20231009' } because 'Fixes CVE-2023-44487' } diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/metrics/PluginMetrics.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/metrics/PluginMetrics.java index 694cabf74f..b20eec607d 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/metrics/PluginMetrics.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/metrics/PluginMetrics.java @@ -8,6 +8,7 @@ import io.micrometer.core.instrument.Counter; import io.micrometer.core.instrument.DistributionSummary; import io.micrometer.core.instrument.Metrics; +import io.micrometer.core.instrument.Tag; import io.micrometer.core.instrument.Timer; import org.opensearch.dataprepper.model.configuration.PluginSetting; @@ -78,8 +79,11 @@ public T gauge(String name, T obj, ToDoubleFunction valueFunction) { return Metrics.gauge(getMeterName(name), obj, valueFunction); } + public T gaugeWithTags(String name, Iterable tags, T obj, ToDoubleFunction valueFunction) { + return Metrics.gauge(getMeterName(name), tags, obj, valueFunction); + } + private String getMeterName(final String name) { return new StringJoiner(MetricNames.DELIMITER).add(metricsPrefix).add(name).toString(); } - } diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/metrics/PluginMetricsTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/metrics/PluginMetricsTest.java index c024af57f6..db6b88d7a0 100644 --- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/metrics/PluginMetricsTest.java +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/metrics/PluginMetricsTest.java @@ -17,6 +17,7 @@ import java.util.StringJoiner; import java.util.concurrent.atomic.AtomicInteger; +import static java.util.Collections.emptyList; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -125,6 +126,18 @@ public void testReferenceGauge() { assertEquals(3, gauge.length()); } + @Test + public void testReferenceGaugeWithTags() { + final String testString = "abc"; + final String gauge = objectUnderTest.gaugeWithTags("gauge", emptyList(), testString, String::length); + assertNotNull( + Metrics.globalRegistry.get(new StringJoiner(MetricNames.DELIMITER) + .add(PIPELINE_NAME).add(PLUGIN_NAME) + .add("gauge").toString()).meter()); + assertEquals(3, gauge.length()); + } + + @Test public void testEmptyPipelineName() { assertThrows( diff --git a/data-prepper-core/build.gradle b/data-prepper-core/build.gradle index ec946e73bf..72d9c47c54 100644 --- a/data-prepper-core/build.gradle +++ b/data-prepper-core/build.gradle @@ -20,7 +20,7 @@ dependencies { testImplementation project(':data-prepper-plugins:common').sourceSets.test.output implementation 'com.fasterxml.jackson.core:jackson-databind' implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' - implementation "org.reflections:reflections:0.10.2" + implementation libs.reflections.core implementation 'io.micrometer:micrometer-core' implementation 'io.micrometer:micrometer-registry-prometheus' implementation 'io.micrometer:micrometer-registry-cloudwatch2' diff --git a/data-prepper-plugins/common/build.gradle b/data-prepper-plugins/common/build.gradle index 6dfa2e0f8f..2744f9c998 100644 --- a/data-prepper-plugins/common/build.gradle +++ b/data-prepper-plugins/common/build.gradle @@ -17,7 +17,7 @@ dependencies { implementation libs.commons.lang3 implementation libs.bouncycastle.bcprov implementation libs.bouncycastle.bcpkix - implementation 'org.reflections:reflections:0.10.2' + implementation libs.reflections.core implementation 'io.micrometer:micrometer-core' testImplementation testLibs.junit.vintage implementation 'org.apache.parquet:parquet-common:1.13.1' diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/NoopSink.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/NoopSink.java new file mode 100644 index 0000000000..f184bd0f40 --- /dev/null +++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/NoopSink.java @@ -0,0 +1,30 @@ +package org.opensearch.dataprepper.plugins.sink; + +import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.sink.Sink; + +import java.util.Collection; + +@DataPrepperPlugin(name = "noop", pluginType = Sink.class) +public class NoopSink implements Sink> { + @Override + public void output(Collection> records) { + // empty by design. + } + + @Override + public void shutdown() { + // empty by design. + } + + @Override + public void initialize() { + // empty by design. + } + + @Override + public boolean isReady() { + return true; + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/README.md b/data-prepper-plugins/kafka-connect-plugins/README.md new file mode 100644 index 0000000000..505222b04a --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/README.md @@ -0,0 +1,278 @@ +# Kafka Connect Source + +This is a source plugin that start a Kafka Connect and Connectors. Please note that the Kafka Connect Source has to work with Kafka Buffer. + +## Usages +To get started with Kafka Connect source, +config the kafka cluster in `data-prepper.config.yaml` configuration +```yaml +ssl: false +extensions: + kafka_connect_config: + worker_properties: + client_id: client + group_id: group + config_storage_topic: pipeline-configs + offset_storage_topic: pipeline-offsets + status_storage_topic: pipeline-status + kafka_cluster_config: + encryption: + type: ssl + insecure: true + aws: + region: "us-east-1" + msk: + broker_connection_type: public + arn: "msk-arn" + authentication: + asl: + aws_msk_iam: default +``` +for Local: +```yaml +ssl: false +extensions: + kafka_connect_config: + worker_properties: + client_id: client + group_id: group + config_storage_topic: pipeline-configs + offset_storage_topic: pipeline-offsets + status_storage_topic: pipeline-status + kafka_cluster_config: + bootstrap_servers: + - localhost:9092 + encryption: + type: none +``` +create the following `pipeline.yaml` configuration: +```yaml +mysql-pipeline: + source: + mysql: + hostname: localhost + credentials: + plaintext: + username: username + password: password + tables: + - topic_prefix: prefix1 + table_name: dbname.tableName1 + - topic_prefix: prefix2 + table_name: dbname.tableName2 + sink: + - noop: + +mongodb-pipeline: + mongodb: + hostname: localhost + credentials: + plaintext: + username: username + password: password + collections: + - topic_prefix: prefix1 + collection_name: dbname.collection1 + - topic_prefix: prefix2 + collection_name: dbname.collection2 + sink: + - noop: + +postgres-pipeline: + postgresql: + hostname: localhost + credentials: + plaintext: + username: username + password: password + tables: + - topic_prefix: prefix1 + database_name: dbname + table_name: public.tableName1 + sink: + - noop: + +sink-pipeline: + source: + kafka: + topics: + - name: prefix1.dbname.collection1 + group_id: mongodb-group + auto_offset_reset: earliest + serde_format: json + - name: prefix2.dbname.collection2 + group_id: mongodb-group + auto_offset_reset: earliest + serde_format: json + - name: prefix1.public.tableName1 + group_id: postgres-group + auto_offset_reset: earliest + serde_format: json + - name: prefix1.dbname.tableName1 + group_id: mysql-group + auto_offset_reset: earliest + serde_format: json + - name: prefix2.dbname.tableName2 + group_id: mysql-group + auto_offset_reset: earliest + serde_format: json + sink: + - stdout: +``` + + +## Configurations + +### Kafka Connect Config +| Options | Required | Default | Type | Description | +|-----------------------|----------|---------|--------|--------------------------------------------------| +| worker_properties | Yes | | | Worker Properties | +| connect_timeout_ms | No | 60000 | Long | The timeout used to detect if Kafka connect is running by data-prepper | +| connector_timeout_ms | No | 30000 | Long | The timeout used to detect if Connectors are in a running state. | + +#### Worker Properties +| Option | Required | Default | Type | Description | +|------------------------------|----------|---------|--------|----------------------------------------------------------------------------------------------------------| +| group_id | YES | | String | A unique string that identifies the connector clusters in OSI/OSDP node that this Worker belongs to. | +| config_storage_topic | YES | | String | The name of the topic where connector and task configuration data are stored. | +| offset_storage_topic | YES | | String | The name of the topic where offsets are stored. | +| status_storage_topic | YES | | String | The name of the topic where connectors and their tasks status are stored. | +| client_id | NO | | String | An ID string to pass to the server when making requests. | +| offset_storage_partitions | No | 25 | int | The number of partitions used when KafkaConnect creates the topic used to store connector offsets. Enter -1 to use the default number of partitions configured in the Kafka broker. | +| offset_flush_interval.ms | No | 60000 | Long | Interval at which to try committing offsets for tasks. By default, connectors commit offsets per 60 secs. | +| offset_flush_timeout_ms | No | 5000 | Long | Maximum number of milliseconds to wait for records to flush and partition offset data to be committed to offset storage before cancelling the process and restoring the offset data to be committed in a future attempt. | +| status_storage_partitions | No | 5 | int | The number of partitions used when Connect creates the topic used to store connector and task status updates. Enter -1 to use the default number of partitions configured in the Kafka broker. | +| heartbeat_interval_ms | No | 3000 | Long | Heartbeats are used to ensure that the Worker’s session stays active and to facilitate rebalancing when new members join or leave the group. | +| session_timeout_ms | No | 30000 | Long | The timeout used to detect failures. Heartbeat must be sent to the broker before this time expires. | + +### MySQL +| Option | Required | Default | Type | Description | +|-----------------|----------|---------|--------------|---------------------------------------------------------------------------------------------------------------------| +| hostname | YES | | String | The hostname of MySQL. | +| port | NO | 3306 | String | The port of MySQL. | +| snapshot_mode | NO | initial | String | MySQL snapshot mode. | +| credentials | YES | | Credentials | The Credentials to access the database. | +| tables | YES | | List\ | The tables to ingest CDC data. | +| force_update | NO | FALSE | Boolean | When restarting or updating a pipeline, whether to force all connectors to update their config even if the connector name already exists. By default, if the connector name exists, the config will not be updated. The connector name is .. | + +Snapshot Mode [ref](https://debezium.io/documentation/reference/stable/connectors/mysql.html#mysql-property-snapshot-mode) + +#### Table +| Option | Required | Default | Type | Description | +|--------------|----------|---------|--------|-------------------------------------------------------| +| topic_prefix | YES | | String | Unique name that identifies the connector. | +| table_name | YES | | String | The table name to ingest, using *database.tableName* format. | + +### Postgresql +| Option | Required | Default | Type | Description | +|----------------|----------|----------|--------------|---------------------------------------------------------------------------------------------------------------------| +| hostname | YES | | String | The hostname of MySQL. | +| port | NO | 5432 | String | The port of MySQL. | +| plugin_name | NO | pgoutput | ENUM | The name of the PostgreSQL logical decoding plug-in installed on the PostgreSQL server. Supported values are `decoderbufs` and `pgoutput`. | +| snapshot_mode | NO | initial | String | PostgreSQL snapshot mode. | +| credentials | YES | | Credentials | The Credentials to access the database. | +| tables | YES | | List\ | The tables to ingest CDC data. | +| force_update | NO | FALSE | Boolean | When restarting or updating a pipeline, whether to force all connectors to update their config even if the connector name already exists. By default, if the connector name exists, the config will not be updated. The connector name is `.`. | + +Snapshot Mode [ref](https://debezium.io/documentation/reference/stable/connectors/postgresql.html#postgresql-property-snapshot-mode) + +#### Table +| Option | Required | Default | Type | Description | +|----------------|----------|---------|--------|---------------------------------------------------------------------------------------------------------------------| +| topic_prefix | YES | | String | Unique name that identifies the connector. | +| database_name | YES | | String | The name of the PostgreSQL database from which to stream the changes. | +| table_name | YES | | String | The table name to ingest, using *schemaName.tableName* format. | + +### MongoDB +| Option | Required | Default | Type | Description | +|-----------------|----------|----------|--------------|---------------------------------------------------------------------------------------------------------------------| +| hostname | YES | | String | The hostname of MySQL. | +| port | NO | 27017 | String | The port of MySQL. | +| ssl | NO | FALSE | Boolean | Connector will use SSL to connect to MongoDB instances. | +| snapshot_mode | NO | initial | String | MongoDB snapshot mode. | +| credentials | YES | | Credentials | The Credentials to access the database. | +| collections | YES | | List\ | The collections to ingest CDC data. | +| force_update | NO | FALSE | Boolean | When restarting or updating a pipeline, whether to force all connectors to update their config even if the connector name already exists. By default, if the connector name exists, the config will not be updated. The connector name is `.`. | +Snapshot Mode [ref](https://debezium.io/documentation/reference/stable/connectors/mongodb.html#mongodb-property-snapshot-mode) + +#### Collection + +| Option | Required | Default | Type | Description | +|------------------|----------|---------|--------|---------------------------------------------------------------------------------------------------------------------| +| topic_prefix | YES | | String | Unique name that identifies the connector. | +| collection_name | YES | | String | The table name to ingest, using *database.collectionName* format. | + + +### Credentials +PlainText username and password +```yaml +credentials: + plaintext: + username: username + password: password +``` +Leverage AWS Secret Manager for username and password +```yaml +credentials: + secret_manager: + region: us-east-1 #required aws region + secretId: secretId #required secret manager secretId + sts_role_arn: sts-role #optional +``` +Note: +* `sts_role_arn` (Optional) : The AWS STS role to assume for requests to Secret Manager. Defaults to null, which will use the [standard SDK behavior for credentials](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/credentials.html). + +### Kafka Cluster Configuration + +The Kafka Cluster Configuration must be set in data-prepper's configuration yaml. + +```yaml +kafka_cluster_config: + bootstrap_servers: + - localhost:9092 + encryption: + type: ssl + insecure: true + aws: + sts_role_arn: sts-role-arn + region: us-east-1 + msk: + broker_connection_type: public + arn: msk-arn + authentication: + sasl: + aws_msk_iam: default +``` +* `bootstrap_servers` (Optional) Required if `aws` and `msk` is not configured. +* `encryption` (Required) https://opensearch.org/docs/latest/data-prepper/pipelines/configuration/sources/kafka/#encryption +* `aws` (Optional) https://opensearch.org/docs/latest/data-prepper/pipelines/configuration/sources/kafka/#aws +* `authentication` (Optional) https://opensearch.org/docs/latest/data-prepper/pipelines/configuration/sources/kafka/#authentication + +# Metrics + +## Kafka Connect Metrics +Kafka Connect Worker Metrics + +- `task-count`: Number of tasks that have run in this worker +- `connector-count`: The number of connectors that have run in this worker +- `connector-startup-attempts-total`: Total number of connector startups that this worker has attempted +- `connector-startup-success-total`: Total number of connector starts that succeeded +- `connector-startup-failure-total`: Total number of connector starts that failed +- `task-startup-attempts-total`: Total number of task startups that the worker has attempted +- `task-startup-success-total`: Total number of task starts that succeeded +- `task-startup-failure-total`: Total number of task starts that failed + +## Connector Metrics +Each connector contains following metrics: +- `source-record-write-total`: Number of records output from the transformations and written to Kafka for the task belonging to the named source connector in the worker (since the task was last restarted) +- `source-record-write-rate`: After transformations are applied, this is the average per-second number of records output from the transformations and written to Kafka for the task belonging to the named source connector in the worker (excludes any records filtered out by the transformations) +- `source-record-poll-total`: This is the number of records produced or polled by the task belonging to the named source connector (database.tableName) in the worker (since the task was last restarted) +- `source-record-poll-rate`: This is the average per-second number of records produced or polled by the task belonging to the named source connector (database.tableName) in the worker +- `source-record-active-count-max`: Maximum number of records polled by the task but not yet completely written to Kafka +- `source-record-active-count-avg`: Average number of records polled by the task but not yet completely written to Kafka +- `source-record-active-count`: Most recent number of records polled by the task but not yet completely written to Kafka + +## Developer Guide +This plugin is compatible with Java 14. See +- [CONTRIBUTING](https://github.com/opensearch-project/data-prepper/blob/main/CONTRIBUTING.md) +- [monitoring](https://github.com/opensearch-project/data-prepper/blob/main/docs/monitoring.md) diff --git a/data-prepper-plugins/kafka-connect-plugins/build.gradle b/data-prepper-plugins/kafka-connect-plugins/build.gradle new file mode 100644 index 0000000000..54a93ca3e5 --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/build.gradle @@ -0,0 +1,67 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +plugins { + id 'java' +} + +configurations.all { + exclude group: 'org.apache.zookeeper', module: 'zookeeper' +} + +dependencies { + implementation project(':data-prepper-plugins:common') + implementation project(':data-prepper-plugins:kafka-plugins') + implementation 'org.apache.kafka:connect-runtime:3.5.1' + implementation 'software.amazon.awssdk:sts' + implementation 'software.amazon.awssdk:secretsmanager' + implementation 'javax.validation:validation-api:2.0.1.Final' + implementation libs.reflections.core + implementation 'io.micrometer:micrometer-core' + implementation ('io.confluent:kafka-schema-registry:7.5.0') { + exclude group: 'org.glassfish.jersey.containers', module: 'jersey-container-servlet' + exclude group: 'org.glassfish.jersey.inject', module: 'jersey-hk2' + exclude group: 'org.glassfish.jersey.ext', module: 'jersey-bean-validation' + } + // Common Debezium Connector + implementation 'io.debezium:debezium-api:2.3.0.Final' + implementation 'io.debezium:debezium-core:2.3.0.Final' + implementation 'io.debezium:debezium-storage-kafka:2.3.0.Final' + implementation 'io.debezium:debezium-storage-file:2.3.0.Final' + // Debezium MySQL Connector + implementation 'org.antlr:antlr4-runtime:4.10.1' + implementation 'io.debezium:debezium-connector-mysql:2.3.0.Final' + implementation 'io.debezium:debezium-ddl-parser:2.3.0.Final' + implementation 'com.zendesk:mysql-binlog-connector-java:0.28.1' + implementation 'com.mysql:mysql-connector-j:8.0.33' + implementation 'com.github.luben:zstd-jni:1.5.0-2' + // Debezium Postgres connector + implementation 'io.debezium:debezium-connector-postgres:2.3.0.Final' + implementation 'org.postgresql:postgresql:42.5.1' + implementation 'com.google.protobuf:protobuf-java:3.19.6' + // Debezium Mongodb connector + implementation 'io.debezium:debezium-connector-mongodb:2.3.0.Final' + implementation 'org.mongodb:mongodb-driver-core:4.7.1' + implementation 'org.mongodb:mongodb-driver-sync:4.7.1' + implementation 'org.mongodb:bson:4.7.1' + runtimeOnly 'org.mongodb:bson-record-codec:4.7.1' + // test + testImplementation project(':data-prepper-test-common') + testImplementation project(':data-prepper-core') + testImplementation 'org.yaml:snakeyaml:2.0' + testImplementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.14.2' + testImplementation testLibs.mockito.inline +} + +jacocoTestCoverageVerification { + dependsOn jacocoTestReport + violationRules { + rule { //in addition to core projects rule + limit { + minimum = 0.90 + } + } + } +} \ No newline at end of file diff --git a/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/configuration/ConnectorConfig.java b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/configuration/ConnectorConfig.java new file mode 100644 index 0000000000..b92e78cdd1 --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/configuration/ConnectorConfig.java @@ -0,0 +1,37 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.opensearch.dataprepper.plugins.kafkaconnect.util.Connector; + +import java.util.List; +import java.util.Properties; + +public abstract class ConnectorConfig { + @JsonProperty("force_update") + public Boolean forceUpdate = false; + private String bootstrapServers; + private Properties authProperties; + + public abstract List buildConnectors(); + + public Properties getAuthProperties() { + return this.authProperties; + } + + public void setAuthProperties(Properties authProperties) { + this.authProperties = authProperties; + } + + public String getBootstrapServers() { + return this.bootstrapServers; + } + + public void setBootstrapServers(String bootStrapServers) { + this.bootstrapServers = bootStrapServers; + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/configuration/CredentialsConfig.java b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/configuration/CredentialsConfig.java new file mode 100644 index 0000000000..c00ffb8357 --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/configuration/CredentialsConfig.java @@ -0,0 +1,91 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.configuration; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.opensearch.dataprepper.plugins.kafkaconnect.util.SecretManagerHelper; + +import java.util.Map; + +public class CredentialsConfig { + private final String username; + private final String password; + + @JsonCreator + public CredentialsConfig(@JsonProperty("plaintext") final PlainText plainText, + @JsonProperty("secret_manager") final SecretManager secretManager) { + if (plainText != null && secretManager != null) { + throw new IllegalArgumentException("plaintext and secret_manager cannot both be set"); + } + if (plainText != null) { + if (plainText.username == null || plainText.password == null) { + throw new IllegalArgumentException("user and password must be set for plaintext credentials"); + } + this.username = plainText.username; + this.password = plainText.password; + } else if (secretManager != null) { + if (secretManager.secretId == null || secretManager.region == null) { + throw new IllegalArgumentException("secretId and region must be set for aws credential type"); + } + final Map secretMap = this.getSecretValueMap(secretManager.stsRoleArn, secretManager.region, secretManager.secretId); + if (!secretMap.containsKey("username") || !secretMap.containsKey("password")) { + throw new RuntimeException("username or password missing in secret manager."); + } + this.username = secretMap.get("username"); + this.password = secretMap.get("password"); + } else { + throw new IllegalArgumentException("plaintext or secret_manager must be set"); + } + } + + private Map getSecretValueMap(String stsRoleArn, String region, String secretId) { + ObjectMapper objectMapper = new ObjectMapper(); + try { + final String secretValue = SecretManagerHelper.getSecretValue(stsRoleArn, region, secretId); + return objectMapper.readValue(secretValue, new TypeReference<>() {}); + } catch (Exception e) { + throw new RuntimeException("Failed to get credentials.", e); + } + } + + public String getUsername() { + return username; + } + + public String getPassword() { + return password; + } + + static class PlainText { + private String username; + private String password; + + @JsonCreator + public PlainText(@JsonProperty("username") String username, + @JsonProperty("password") String password) { + this.username = username; + this.password = password; + } + } + + static class SecretManager { + private String region; + private String secretId; + private String stsRoleArn; + + @JsonCreator + public SecretManager(@JsonProperty("sts_role_arn") String stsRoleArn, + @JsonProperty("region") String region, + @JsonProperty("secretId") String secretId) { + this.stsRoleArn = stsRoleArn; + this.region = region; + this.secretId = secretId; + } + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/configuration/MongoDBConfig.java b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/configuration/MongoDBConfig.java new file mode 100644 index 0000000000..ab8476fd96 --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/configuration/MongoDBConfig.java @@ -0,0 +1,91 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.NotNull; +import org.opensearch.dataprepper.plugins.kafkaconnect.util.Connector; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class MongoDBConfig extends ConnectorConfig { + public static final String CONNECTOR_CLASS = "io.debezium.connector.mongodb.MongoDbConnector"; + private static final String MONGODB_CONNECTION_STRING_FORMAT = "mongodb://%s:%s/?replicaSet=rs0&directConnection=true"; + private static final String DEFAULT_PORT = "27017"; + private static final String DEFAULT_SNAPSHOT_MODE = "initial"; + private static final Boolean SSL_ENABLED = false; + private static final Boolean SSL_INVALID_HOST_ALLOWED = false; + private static final String DEFAULT_SNAPSHOT_FETCH_SIZE = "1000"; + @JsonProperty("hostname") + @NotNull + private String hostname; + @JsonProperty("port") + private String port = DEFAULT_PORT; + @JsonProperty("credentials") + private CredentialsConfig credentialsConfig; + @JsonProperty("snapshot_mode") + private String snapshotMode = DEFAULT_SNAPSHOT_MODE; + @JsonProperty("snapshot_fetch_size") + private String snapshotFetchSize = DEFAULT_SNAPSHOT_FETCH_SIZE; + @JsonProperty("collections") + private List collections = new ArrayList<>(); + @JsonProperty("ssl") + private Boolean ssl = SSL_ENABLED; + @JsonProperty("ssl_invalid_host_allowed") + private Boolean sslInvalidHostAllowed = SSL_INVALID_HOST_ALLOWED; + + @Override + public List buildConnectors() { + return collections.stream().map(collection -> { + final String connectorName = collection.getTopicPrefix() + "." + collection.getCollectionName(); + final Map config = buildConfig(collection); + return new Connector(connectorName, config, this.forceUpdate); + }).collect(Collectors.toList()); + } + + private Map buildConfig(final CollectionConfig collection) { + Map config = new HashMap<>(); + config.put("connector.class", CONNECTOR_CLASS); + config.put("mongodb.connection.string", String.format(MONGODB_CONNECTION_STRING_FORMAT, hostname, port)); + config.put("mongodb.user", credentialsConfig.getUsername()); + config.put("mongodb.password", credentialsConfig.getPassword()); + config.put("snapshot.mode", snapshotMode); + config.put("snapshot.fetch.size", snapshotFetchSize); + config.put("topic.prefix", collection.getTopicPrefix()); + config.put("collection.include.list", collection.getCollectionName()); + config.put("mongodb.ssl.enabled", ssl.toString()); + config.put("mongodb.ssl.invalid.hostname.allowed", sslInvalidHostAllowed.toString()); + // Non-configurable properties used to transform CDC data before sending to Kafka. + config.put("transforms", "unwrap"); + config.put("transforms.unwrap.type", "io.debezium.connector.mongodb.transforms.ExtractNewDocumentState"); + config.put("transforms.unwrap.drop.tombstones", "true"); + config.put("transforms.unwrap.delete.handling.mode", "rewrite"); + config.put("transforms.unwrap.add.fields", "op,rs,collection,source.ts_ms,source.db,source.snapshot,ts_ms"); + return config; + } + + private static class CollectionConfig { + @JsonProperty("topic_prefix") + @NotNull + private String topicPrefix; + + @JsonProperty("collection") + @NotNull + private String collectionName; + + public String getCollectionName() { + return collectionName; + } + + public String getTopicPrefix() { + return topicPrefix; + } + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/configuration/MySQLConfig.java b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/configuration/MySQLConfig.java new file mode 100644 index 0000000000..ea9d0f9f95 --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/configuration/MySQLConfig.java @@ -0,0 +1,109 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.NotNull; +import org.apache.kafka.connect.runtime.WorkerConfig; +import org.opensearch.dataprepper.plugins.kafkaconnect.util.Connector; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.stream.Collectors; + +public class MySQLConfig extends ConnectorConfig { + public static final String CONNECTOR_CLASS = "io.debezium.connector.mysql.MySqlConnector"; + private static final String SCHEMA_HISTORY_PRODUCER_PREFIX = "schema.history.internal.producer."; + private static final String SCHEMA_HISTORY_CONSUMER_PREFIX = "schema.history.internal.consumer."; + private static final String TOPIC_DEFAULT_PARTITIONS = "10"; + private static final String TOPIC_DEFAULT_REPLICATION_FACTOR = "-1"; + private static final String SCHEMA_HISTORY = "schemahistory"; + private static final String DEFAULT_SNAPSHOT_MODE = "initial"; + private static final String DEFAULT_PORT = "3306"; + + @JsonProperty("hostname") + @NotNull + private String hostname; + @JsonProperty("port") + private String port = DEFAULT_PORT; + @JsonProperty("credentials") + private CredentialsConfig credentialsConfig; + @JsonProperty("snapshot_mode") + private String snapshotMode = DEFAULT_SNAPSHOT_MODE; + @JsonProperty("tables") + private List tables = new ArrayList<>(); + + @Override + public List buildConnectors() { + return tables.stream().map(table -> { + final String connectorName = table.getTopicPrefix() + "." + table.getTableName(); + final Map config = buildConfig(table, connectorName); + return new Connector(connectorName, config, this.forceUpdate); + }).collect(Collectors.toList()); + } + + private Map buildConfig(final TableConfig table, final String connectorName) { + int databaseServerId = Math.abs(connectorName.hashCode()); + final Map config = new HashMap<>(); + final Properties authProperties = this.getAuthProperties(); + if (authProperties != null) { + authProperties.forEach((k, v) -> { + if (k == WorkerConfig.BOOTSTRAP_SERVERS_CONFIG) { + this.setBootstrapServers(v.toString()); + return; + } + if (v instanceof Class) { + config.put(SCHEMA_HISTORY_PRODUCER_PREFIX + k, ((Class) v).getName()); + config.put(SCHEMA_HISTORY_CONSUMER_PREFIX + k, ((Class) v).getName()); + return; + } + config.put(SCHEMA_HISTORY_PRODUCER_PREFIX + k, v.toString()); + config.put(SCHEMA_HISTORY_CONSUMER_PREFIX + k, v.toString()); + }); + } + config.put("topic.creation.default.partitions", TOPIC_DEFAULT_PARTITIONS); + config.put("topic.creation.default.replication.factor", TOPIC_DEFAULT_REPLICATION_FACTOR); + config.put("connector.class", CONNECTOR_CLASS); + config.put("database.hostname", hostname); + config.put("database.port", port); + config.put("database.user", credentialsConfig.getUsername()); + config.put("database.password", credentialsConfig.getPassword()); + config.put("snapshot.mode", snapshotMode); + config.put("topic.prefix", table.getTopicPrefix()); + config.put("table.include.list", table.getTableName()); + config.put("schema.history.internal.kafka.bootstrap.servers", this.getBootstrapServers()); + config.put("schema.history.internal.kafka.topic", String.join(".", List.of(table.getTopicPrefix(), table.getTableName(), SCHEMA_HISTORY))); + config.put("database.server.id", Integer.toString(databaseServerId)); + // Non-configurable properties used to transform CDC data before sending to Kafka. + config.put("transforms", "unwrap"); + config.put("transforms.unwrap.type", "io.debezium.transforms.ExtractNewRecordState"); + config.put("transforms.unwrap.drop.tombstones", "true"); + config.put("transforms.unwrap.delete.handling.mode", "rewrite"); + config.put("transforms.unwrap.add.fields", "op,table,source.ts_ms,source.db,source.snapshot,ts_ms"); + return config; + } + + private static class TableConfig { + @JsonProperty("topic_prefix") + @NotNull + private String topicPrefix; + + @JsonProperty("table") + @NotNull + private String tableName; + + public String getTableName() { + return tableName; + } + + public String getTopicPrefix() { + return topicPrefix; + } + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/configuration/PostgreSQLConfig.java b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/configuration/PostgreSQLConfig.java new file mode 100644 index 0000000000..205cfcd7a4 --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/configuration/PostgreSQLConfig.java @@ -0,0 +1,124 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.configuration; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.NotNull; +import org.opensearch.dataprepper.plugins.kafkaconnect.util.Connector; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class PostgreSQLConfig extends ConnectorConfig { + public static final String CONNECTOR_CLASS = "io.debezium.connector.postgresql.PostgresConnector"; + private static final String TOPIC_DEFAULT_PARTITIONS = "10"; + private static final String TOPIC_DEFAULT_REPLICATION_FACTOR = "-1"; + private static final String DEFAULT_PORT = "5432"; + private static final String DEFAULT_SNAPSHOT_MODE = "initial"; + private static final PluginName DEFAULT_DECODING_PLUGIN = PluginName.PGOUTPUT; // default plugin for Aurora PostgreSQL + @JsonProperty("hostname") + @NotNull + private String hostname; + @JsonProperty("port") + private String port = DEFAULT_PORT; + /** + * The name of the PostgreSQL logical decoding plug-in installed on the PostgreSQL server. + * Supported values are decoderbufs, and pgoutput. + */ + @JsonProperty("plugin_name") + private PluginName pluginName = DEFAULT_DECODING_PLUGIN; + @JsonProperty("credentials") + private CredentialsConfig credentialsConfig; + @JsonProperty("snapshot_mode") + private String snapshotMode = DEFAULT_SNAPSHOT_MODE; + @JsonProperty("tables") + private List tables = new ArrayList<>(); + + @Override + public List buildConnectors() { + return tables.stream().map(table -> { + final String connectorName = table.getTopicPrefix() + "." + table.getTableName(); + final Map config = buildConfig(table); + return new Connector(connectorName, config, this.forceUpdate); + }).collect(Collectors.toList()); + } + + private Map buildConfig(final TableConfig tableName) { + Map config = new HashMap<>(); + config.put("topic.creation.default.partitions", TOPIC_DEFAULT_PARTITIONS); + config.put("topic.creation.default.replication.factor", TOPIC_DEFAULT_REPLICATION_FACTOR); + config.put("connector.class", CONNECTOR_CLASS); + config.put("plugin.name", pluginName.type); + config.put("database.hostname", hostname); + config.put("database.port", port); + config.put("database.user", credentialsConfig.getUsername()); + config.put("database.password", credentialsConfig.getPassword()); + config.put("snapshot.mode", snapshotMode); + config.put("topic.prefix", tableName.getTopicPrefix()); + config.put("database.dbname", tableName.getDatabaseName()); + config.put("table.include.list", tableName.getTableName()); + // Non-configurable properties used to transform CDC data before sending to Kafka. + config.put("transforms", "unwrap"); + config.put("transforms.unwrap.type", "io.debezium.transforms.ExtractNewRecordState"); + config.put("transforms.unwrap.drop.tombstones", "true"); + config.put("transforms.unwrap.delete.handling.mode", "rewrite"); + config.put("transforms.unwrap.add.fields", "op,table,source.ts_ms,source.db,source.snapshot,ts_ms"); + return config; + } + + public enum PluginName { + DECODERBUFS("decoderbufs"), + PGOUTPUT("pgoutput"); + + private static final Map OPTIONS_MAP = Arrays.stream(PostgreSQLConfig.PluginName.values()) + .collect(Collectors.toMap( + value -> value.type, + value -> value + )); + + private final String type; + + PluginName(final String type) { + this.type = type; + } + + @JsonCreator + public static PostgreSQLConfig.PluginName fromTypeValue(final String type) { + return OPTIONS_MAP.get(type.toLowerCase()); + } + } + + private static class TableConfig { + @JsonProperty("database") + @NotNull + private String databaseName; + + @JsonProperty("topic_prefix") + @NotNull + private String topicPrefix; + + @JsonProperty("table") + @NotNull + private String tableName; + + public String getDatabaseName() { + return databaseName; + } + + public String getTableName() { + return tableName; + } + + public String getTopicPrefix() { + return topicPrefix; + } + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/DefaultKafkaConnectConfigSupplier.java b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/DefaultKafkaConnectConfigSupplier.java new file mode 100644 index 0000000000..39e7f6609a --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/DefaultKafkaConnectConfigSupplier.java @@ -0,0 +1,18 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.extension; + +public class DefaultKafkaConnectConfigSupplier implements KafkaConnectConfigSupplier { + private final KafkaConnectConfig kafkaConnectConfig; + public DefaultKafkaConnectConfigSupplier(KafkaConnectConfig kafkaConnectConfig) { + this.kafkaConnectConfig = kafkaConnectConfig; + } + + @Override + public KafkaConnectConfig getConfig() { + return this.kafkaConnectConfig; + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/KafkaConnectConfig.java b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/KafkaConnectConfig.java new file mode 100644 index 0000000000..fbed48d949 --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/KafkaConnectConfig.java @@ -0,0 +1,96 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.extension; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.opensearch.dataprepper.plugins.kafka.configuration.AuthConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.AwsConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.EncryptionConfig; +import org.opensearch.dataprepper.plugins.kafka.util.KafkaClusterAuthConfig; + +import java.time.Duration; +import java.util.List; +import java.util.Objects; +import java.util.Properties; + +public class KafkaConnectConfig implements KafkaClusterAuthConfig { + private static final long CONNECTOR_TIMEOUT_MS = 360000L; // 360 seconds + private static final long CONNECT_TIMEOUT_MS = 60000L; // 60 seconds + + @JsonProperty("worker_properties") + private WorkerProperties workerProperties = new WorkerProperties(); + + @JsonProperty("connect_start_timeout") + private Duration connectStartTimeout = Duration.ofMillis(CONNECT_TIMEOUT_MS); + + @JsonProperty("connector_start_timeout") + private Duration connectorStartTimeout = Duration.ofMillis(CONNECTOR_TIMEOUT_MS); + + @JsonProperty("bootstrap_servers") + private List bootstrapServers; + + private AuthConfig authConfig; + private EncryptionConfig encryptionConfig; + private AwsConfig awsConfig; + + public Duration getConnectStartTimeout() { + return connectStartTimeout; + } + + public Duration getConnectorStartTimeout() { + return connectorStartTimeout; + } + + public void setBootstrapServers(final List bootstrapServers) { + this.bootstrapServers = bootstrapServers; + if (Objects.nonNull(bootstrapServers)) { + this.workerProperties.setBootstrapServers(String.join(",", bootstrapServers));; + } + } + + public void setAuthProperties(final Properties authProperties) { + this.workerProperties.setAuthProperties(authProperties); + } + + public void setAuthConfig(AuthConfig authConfig) { + this.authConfig = authConfig; + } + + public void setAwsConfig(AwsConfig awsConfig) { + this.awsConfig = awsConfig; + } + + public void setEncryptionConfig(EncryptionConfig encryptionConfig) { + this.encryptionConfig = encryptionConfig; + } + + public WorkerProperties getWorkerProperties() { + return workerProperties; + } + + @Override + public AwsConfig getAwsConfig() { + return awsConfig; + } + + @Override + public AuthConfig getAuthConfig() { + return authConfig; + } + + @Override + public EncryptionConfig getEncryptionConfig() { + return encryptionConfig; + } + + @Override + public List getBootstrapServers() { + if (Objects.nonNull(bootstrapServers)) { + return bootstrapServers; + } + return null; + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/KafkaConnectConfigExtension.java b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/KafkaConnectConfigExtension.java new file mode 100644 index 0000000000..18c6aee682 --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/KafkaConnectConfigExtension.java @@ -0,0 +1,30 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.extension; + +import org.opensearch.dataprepper.model.annotations.DataPrepperExtensionPlugin; +import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; +import org.opensearch.dataprepper.model.plugin.ExtensionPlugin; +import org.opensearch.dataprepper.model.plugin.ExtensionPoints; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@DataPrepperExtensionPlugin(modelType = KafkaConnectConfig.class, rootKeyJsonPath = "/kafka_connect_config") +public class KafkaConnectConfigExtension implements ExtensionPlugin { + private static final Logger LOG = LoggerFactory.getLogger(KafkaConnectConfigExtension.class); + private DefaultKafkaConnectConfigSupplier defaultKafkaConnectConfigSupplier; + + @DataPrepperPluginConstructor + public KafkaConnectConfigExtension(final KafkaConnectConfig kafkaConnectConfig) { + this.defaultKafkaConnectConfigSupplier = new DefaultKafkaConnectConfigSupplier(kafkaConnectConfig); + } + + @Override + public void apply(ExtensionPoints extensionPoints) { + LOG.info("Applying Kafka Connect Config Extension."); + extensionPoints.addExtensionProvider(new KafkaConnectConfigProvider(this.defaultKafkaConnectConfigSupplier)); + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/KafkaConnectConfigProvider.java b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/KafkaConnectConfigProvider.java new file mode 100644 index 0000000000..2e2f5e01f6 --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/KafkaConnectConfigProvider.java @@ -0,0 +1,27 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.extension; + +import org.opensearch.dataprepper.model.plugin.ExtensionProvider; + +import java.util.Optional; + +public class KafkaConnectConfigProvider implements ExtensionProvider { + private final KafkaConnectConfigSupplier kafkaConnectConfigSupplier; + public KafkaConnectConfigProvider(KafkaConnectConfigSupplier kafkaConnectConfigSupplier) { + this.kafkaConnectConfigSupplier = kafkaConnectConfigSupplier; + } + + @Override + public Optional provideInstance(Context context) { + return Optional.of(this.kafkaConnectConfigSupplier); + } + + @Override + public Class supportedClass() { + return KafkaConnectConfigSupplier.class; + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/KafkaConnectConfigSupplier.java b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/KafkaConnectConfigSupplier.java new file mode 100644 index 0000000000..c5805378d9 --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/KafkaConnectConfigSupplier.java @@ -0,0 +1,10 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.extension; + +public interface KafkaConnectConfigSupplier { + KafkaConnectConfig getConfig(); +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/WorkerProperties.java b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/WorkerProperties.java new file mode 100644 index 0000000000..1de8fff29c --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/WorkerProperties.java @@ -0,0 +1,213 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.extension; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.kafka.connect.runtime.WorkerConfig; + +import java.time.Duration; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +public class WorkerProperties { + private static final String KEY_CONVERTER = "org.apache.kafka.connect.json.JsonConverter"; + private static final String KEY_CONVERTER_SCHEMAS_ENABLE = "false"; + private static final String VALUE_CONVERTER_SCHEMAS_ENABLE = "false"; + private static final String VALUE_CONVERTER = "org.apache.kafka.connect.json.JsonConverter"; + private static final Integer OFFSET_STORAGE_PARTITIONS = 25; + private static final Long OFFSET_FLUSH_INTERVAL_MS = 60000L; + private static final Long OFFSET_FLUSH_TIMEOUT_MS = 5000L; + private static final Integer STATUS_STORAGE_PARTITIONS = 5; + private static final Long HEARTBEAT_INTERVAL_MS = 3000L; + private static final Long SESSION_TIMEOUT_MS = 30000L; + private static final long CONNECTOR_REBALANCE_DELAY_MS = 300000L; // 300 seconds + private static final String DEFAULT_GROUP_ID = "localGroup"; + private static final String DEFAULT_CLIENT_ID = "localClient"; + private static final String DEFAULT_CONFIG_STORAGE_TOPIC = "config-storage-topic"; + private static final String DEFAULT_OFFSET_STORAGE_TOPIC = "offset-storage-topic"; + private static final String DEFAULT_STATUS_STORAGE_TOPIC = "status-storage-topic"; + private final Integer offsetStorageReplicationFactor = -1; + private final Integer configStorageReplicationFactor = -1; + private final Integer statusStorageReplicationFactor = -1; + private String keyConverter = KEY_CONVERTER; + private String keyConverterSchemasEnable = KEY_CONVERTER_SCHEMAS_ENABLE; + private String valueConverter = VALUE_CONVERTER; + private String valueConverterSchemasEnable = VALUE_CONVERTER_SCHEMAS_ENABLE; + + @JsonProperty("group_id") + private String groupId = DEFAULT_GROUP_ID; + @JsonProperty("config_storage_topic") + private String configStorageTopic = DEFAULT_CONFIG_STORAGE_TOPIC; + @JsonProperty("offset_storage_topic") + private String offsetStorageTopic = DEFAULT_OFFSET_STORAGE_TOPIC; + @JsonProperty("status_storage_topic") + private String statusStorageTopic = DEFAULT_STATUS_STORAGE_TOPIC; + @JsonProperty("client_id") + private String clientId = DEFAULT_CLIENT_ID; + @JsonProperty("offset_storage_partitions") + private Integer offsetStoragePartitions = OFFSET_STORAGE_PARTITIONS; + @JsonProperty("offset_flush_interval") + private Duration offsetFlushInterval = Duration.ofMillis(OFFSET_FLUSH_INTERVAL_MS); + @JsonProperty("offset_flush_timeout") + private Duration offsetFlushTimeout = Duration.ofMillis(OFFSET_FLUSH_TIMEOUT_MS); + @JsonProperty("status_storage_partitions") + private Integer statusStoragePartitions = STATUS_STORAGE_PARTITIONS; + @JsonProperty("heartbeat_interval") + private Duration heartBeatInterval = Duration.ofMillis(HEARTBEAT_INTERVAL_MS); + @JsonProperty("session_timeout") + private Duration sessionTimeout = Duration.ofMillis(SESSION_TIMEOUT_MS); + @JsonProperty("connector_rebalance_max_delay") + private Duration connectorRebalanceDelay = Duration.ofMillis(CONNECTOR_REBALANCE_DELAY_MS); + private String keyConverterSchemaRegistryUrl; + private String valueConverterSchemaRegistryUrl; + private String bootstrapServers; + private Properties authProperties; + + public String getKeyConverter() { + return keyConverter; + } + + public String getKeyConverterSchemasEnable() { + return keyConverterSchemasEnable; + } + + public String getKeyConverterSchemaRegistryUrl() { + return keyConverterSchemaRegistryUrl; + } + + public String getValueConverter() { + return valueConverter; + } + + public String getValueConverterSchemasEnable() { + return valueConverterSchemasEnable; + } + + public String getValueConverterSchemaRegistryUrl() { + return valueConverterSchemaRegistryUrl; + } + + public Integer getOffsetStoragePartitions() { + return offsetStoragePartitions; + } + + public Long getOffsetFlushInterval() { + return offsetFlushInterval.toMillis(); + } + + public Long getOffsetFlushTimeout() { + return offsetFlushTimeout.toMillis(); + } + + public Long getRebalanceMaxDelay() { + return connectorRebalanceDelay.toMillis(); + } + + public Integer getStatusStoragePartitions() { + return statusStoragePartitions; + } + + public Long getHeartBeatInterval() { + return heartBeatInterval.toMillis(); + } + + public Long getSessionTimeout() { + return sessionTimeout.toMillis(); + } + + public String getBootstrapServers() { + return bootstrapServers; + } + + public void setBootstrapServers(final String bootstrapServers) { + this.bootstrapServers = bootstrapServers; + } + + public String getGroupId() { + return groupId; + } + + public String getClientId() { + return clientId; + } + + public String getConfigStorageTopic() { + return configStorageTopic; + } + + public Integer getConfigStorageReplicationFactor() { + return configStorageReplicationFactor; + } + + public String getOffsetStorageTopic() { + return offsetStorageTopic; + } + + public Integer getOffsetStorageReplicationFactor() { + return offsetStorageReplicationFactor; + } + + public String getStatusStorageTopic() { + return statusStorageTopic; + } + + public Integer getStatusStorageReplicationFactor() { + return statusStorageReplicationFactor; + } + + public void setAuthProperties(Properties authProperties) { + this.authProperties = authProperties; + } + + public Map buildKafkaConnectPropertyMap() { + final String producerPrefix = "producer."; + Map workerProps = new HashMap<>(); + if (authProperties != null) { + authProperties.forEach((k, v) -> { + if (k == WorkerConfig.BOOTSTRAP_SERVERS_CONFIG) { + this.setBootstrapServers(v.toString()); + return; + } + if (v instanceof Class) { + workerProps.put(k.toString(), ((Class) v).getName()); + workerProps.put(producerPrefix + k, ((Class) v).getName()); + return; + } + workerProps.put(k.toString(), v.toString()); + workerProps.put(producerPrefix + k, v.toString()); + }); + } + workerProps.put("bootstrap.servers", this.getBootstrapServers()); + workerProps.put("group.id", this.getGroupId()); + workerProps.put("client.id", this.getClientId()); + workerProps.put("offset.storage.topic", this.getOffsetStorageTopic()); + workerProps.put("offset.storage.replication.factor", this.getOffsetStorageReplicationFactor().toString()); + workerProps.put("config.storage.topic", this.getConfigStorageTopic()); + workerProps.put("config.storage.replication.factor", this.getConfigStorageReplicationFactor().toString()); + workerProps.put("status.storage.topic", this.getStatusStorageTopic()); + workerProps.put("status.storage.replication.factor", this.getStatusStorageReplicationFactor().toString()); + workerProps.put("key.converter", this.getKeyConverter()); + workerProps.put("key.converter.schemas.enable", this.getKeyConverterSchemasEnable()); + if (this.getKeyConverterSchemaRegistryUrl() != null) { + workerProps.put("key.converter.schema.registry.url", this.getKeyConverterSchemaRegistryUrl()); + } + workerProps.put("value.converter", this.getValueConverter()); + workerProps.put("value.converter.schemas.enable", this.getValueConverterSchemasEnable()); + if (this.getValueConverterSchemaRegistryUrl() != null) { + workerProps.put("value.converter.schema.registry.url", this.getValueConverterSchemaRegistryUrl()); + } + workerProps.put("offset.storage.partitions", this.getOffsetStoragePartitions().toString()); + workerProps.put("offset.flush.interval.ms", this.getOffsetFlushInterval().toString()); + workerProps.put("offset.flush.timeout.ms", this.getOffsetFlushTimeout().toString()); + workerProps.put("status.storage.partitions", this.getStatusStoragePartitions().toString()); + workerProps.put("heartbeat.interval.ms", this.getHeartBeatInterval().toString()); + workerProps.put("session.timeout.ms", this.getSessionTimeout().toString()); + workerProps.put("scheduled.rebalance.max.delay.ms", this.getRebalanceMaxDelay().toString()); + return workerProps; + } +} + diff --git a/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/meter/KafkaConnectMetrics.java b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/meter/KafkaConnectMetrics.java new file mode 100644 index 0000000000..db7250c259 --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/meter/KafkaConnectMetrics.java @@ -0,0 +1,168 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.meter; + +import io.micrometer.core.instrument.Tag; +import io.micrometer.core.instrument.Tags; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.management.MBeanServer; +import javax.management.MBeanServerFactory; +import javax.management.MalformedObjectNameException; +import javax.management.ObjectName; +import java.lang.management.ManagementFactory; +import java.util.List; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.function.BiFunction; + +import static java.util.Collections.emptyList; + +public class KafkaConnectMetrics { + private static final Logger LOG = LoggerFactory.getLogger(KafkaConnectMetrics.class); + private static final String JMX_DOMAIN = "kafka.connect"; + private static final String CONNECT_WORKER_METRICS_NAME = "connect-worker-metrics"; + private static final List CONNECT_WORKER_METRICS_LIST = List.of( + "task-count", + "connector-count", + "connector-startup-attempts-total", + "connector-startup-success-total", + "connector-startup-failure-total", + "task-startup-attempts-total", + "task-startup-success-total", + "task-startup-failure-total" + ); + private static final String SOURCE_TASK_METRICS_NAME = "source-task-metrics"; + private static final List SOURCE_TASK_METRICS_LIST = List.of( + "source-record-write-total", + "source-record-write-rate", + "source-record-poll-total", + "source-record-poll-rate", + "source-record-active-count-max", + "source-record-active-count-avg", + "source-record-active-count" + ); + private static final String CLIENT_ID_KEY = "client-id"; + private static final String CLIENT_ID = "client.id"; + private static final String NODE_ID_KEY = "node-id"; + private static final String NODE_ID = "node.id"; + private static final String CONNECTOR = "connector"; + private static final String TASK = "task"; + + private final PluginMetrics pluginMetrics; + + private final MBeanServer mBeanServer; + + private final Iterable tags; + + + public KafkaConnectMetrics(final PluginMetrics pluginMetrics) { + this(pluginMetrics, emptyList()); + } + + public KafkaConnectMetrics(final PluginMetrics pluginMetrics, + final Iterable tags) { + this(pluginMetrics, getMBeanServer(), tags); + } + + public KafkaConnectMetrics(final PluginMetrics pluginMetrics, + final MBeanServer mBeanServer, + final Iterable tags) { + this.pluginMetrics = pluginMetrics; + this.mBeanServer = mBeanServer; + this.tags = tags; + } + + private static MBeanServer getMBeanServer() { + List mBeanServers = MBeanServerFactory.findMBeanServer(null); + if (!mBeanServers.isEmpty()) { + return mBeanServers.get(0); + } + return ManagementFactory.getPlatformMBeanServer(); + } + + private static String sanitize(String value) { + return value.replaceAll("-", "."); + } + + public void bindConnectMetrics() { + registerMetricsEventually(CONNECT_WORKER_METRICS_NAME, (o, tags) -> { + CONNECT_WORKER_METRICS_LIST.forEach( + (metricName) -> registerFunctionGaugeForObject(o, metricName, tags) + ); + return null; + }); + } + + public void bindConnectorMetrics() { + registerMetricsEventually(SOURCE_TASK_METRICS_NAME, (o, tags) -> { + SOURCE_TASK_METRICS_LIST.forEach( + (metricName) -> registerFunctionGaugeForObject(o, metricName, tags) + ); + return null; + }); + } + + private void registerMetricsEventually(String type, + BiFunction perObject) { + try { + Set objs = mBeanServer.queryNames(new ObjectName(JMX_DOMAIN + ":type=" + type + ",*"), null); + if (!objs.isEmpty()) { + for (ObjectName o : objs) { + perObject.apply(o, Tags.concat(tags, nameTag(o))); + } + } + } catch (MalformedObjectNameException e) { + throw new RuntimeException("Error registering Kafka Connect JMX based metrics", e); + } + } + + private Iterable nameTag(ObjectName name) { + Tags tags = Tags.empty(); + + String clientId = name.getKeyProperty(CLIENT_ID_KEY); + if (clientId != null) { + tags = Tags.concat(tags, CLIENT_ID, clientId); + } + + String nodeId = name.getKeyProperty(NODE_ID_KEY); + if (nodeId != null) { + tags = Tags.concat(tags, NODE_ID, nodeId); + } + + String connectorName = name.getKeyProperty(CONNECTOR); + if (connectorName != null) { + tags = Tags.concat(tags, CONNECTOR, connectorName); + } + + String taskName = name.getKeyProperty(TASK); + if (taskName != null) { + tags = Tags.concat(tags, TASK, taskName); + } + + return tags; + } + + private void registerFunctionGaugeForObject(ObjectName o, String jmxMetricName, Tags allTags) { + pluginMetrics.gaugeWithTags( + sanitize(jmxMetricName), + allTags, + mBeanServer, + s -> safeDouble(() -> s.getAttribute(o, jmxMetricName)) + ); + } + + private double safeDouble(Callable callable) { + try { + if (callable.call() == null) return Double.NaN; + return Double.parseDouble(callable.call().toString()); + } catch (Exception e) { + return Double.NaN; + } + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/source/KafkaConnectSource.java b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/source/KafkaConnectSource.java new file mode 100644 index 0000000000..96ec7faca6 --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/source/KafkaConnectSource.java @@ -0,0 +1,104 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.source; + +import org.apache.kafka.connect.runtime.WorkerConfig; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.configuration.PipelineDescription; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.source.Source; +import org.opensearch.dataprepper.plugins.kafka.extension.KafkaClusterConfigSupplier; +import org.opensearch.dataprepper.plugins.kafka.util.KafkaSecurityConfigurer; +import org.opensearch.dataprepper.plugins.kafkaconnect.configuration.ConnectorConfig; +import org.opensearch.dataprepper.plugins.kafkaconnect.extension.KafkaConnectConfig; +import org.opensearch.dataprepper.plugins.kafkaconnect.extension.KafkaConnectConfigSupplier; +import org.opensearch.dataprepper.plugins.kafkaconnect.extension.WorkerProperties; +import org.opensearch.dataprepper.plugins.kafkaconnect.util.Connector; +import org.opensearch.dataprepper.plugins.kafkaconnect.util.KafkaConnect; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Properties; + +/** + * The abstraction of the kafka connect source. + * The kafka connect and connectors are configured and runs async here. + */ +@SuppressWarnings("deprecation") +public abstract class KafkaConnectSource implements Source> { + private static final Logger LOG = LoggerFactory.getLogger(KafkaConnectSource.class); + private final ConnectorConfig connectorConfig; + private final KafkaConnectConfig kafkaConnectConfig; + private final String pipelineName; + private KafkaConnect kafkaConnect; + + public KafkaConnectSource(final ConnectorConfig connectorConfig, + final PluginMetrics pluginMetrics, + final PipelineDescription pipelineDescription, + final KafkaClusterConfigSupplier kafkaClusterConfigSupplier, + final KafkaConnectConfigSupplier kafkaConnectConfigSupplier) { + if (kafkaClusterConfigSupplier == null || kafkaConnectConfigSupplier == null) { + throw new IllegalArgumentException("Extensions: KafkaClusterConfig and KafkaConnectConfig cannot be null"); + } + this.connectorConfig = connectorConfig; + this.pipelineName = pipelineDescription.getPipelineName(); + this.kafkaConnectConfig = kafkaConnectConfigSupplier.getConfig(); + this.updateConfig(kafkaClusterConfigSupplier); + this.kafkaConnect = KafkaConnect.getPipelineInstance( + pipelineName, + pluginMetrics, + kafkaConnectConfig.getConnectStartTimeout(), + kafkaConnectConfig.getConnectorStartTimeout()); + } + + @Override + public void start(Buffer> buffer) { + LOG.info("Starting Kafka Connect Source for pipeline: {}", pipelineName); + // Please make sure buildWokerProperties is always first to execute. + final WorkerProperties workerProperties = this.kafkaConnectConfig.getWorkerProperties(); + Map workerProps = workerProperties.buildKafkaConnectPropertyMap(); + if (workerProps.get(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG) == null || workerProps.get(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG).isEmpty()) { + throw new IllegalArgumentException("Bootstrap Servers cannot be null or empty"); + } + final List connectors = this.connectorConfig.buildConnectors(); + kafkaConnect.addConnectors(connectors); + kafkaConnect.initialize(workerProps); + kafkaConnect.start(); + } + + @Override + public void stop() { + LOG.info("Stopping Kafka Connect Source for pipeline: {}", pipelineName); + kafkaConnect.stop(); + } + + private void updateConfig(final KafkaClusterConfigSupplier kafkaClusterConfigSupplier) { + if (kafkaConnectConfig.getBootstrapServers() == null) { + this.kafkaConnectConfig.setBootstrapServers(kafkaClusterConfigSupplier.getBootStrapServers()); + } + if (kafkaConnectConfig.getAuthConfig() == null) { + kafkaConnectConfig.setAuthConfig(kafkaClusterConfigSupplier.getAuthConfig()); + } + if (kafkaConnectConfig.getAwsConfig() == null) { + kafkaConnectConfig.setAwsConfig(kafkaClusterConfigSupplier.getAwsConfig()); + } + if (kafkaConnectConfig.getEncryptionConfig() == null) { + kafkaConnectConfig.setEncryptionConfig(kafkaClusterConfigSupplier.getEncryptionConfig()); + } + Properties authProperties = new Properties(); + KafkaSecurityConfigurer.setAuthProperties(authProperties, kafkaConnectConfig, LOG); + this.kafkaConnectConfig.setAuthProperties(authProperties); + // Update Connector Config + if (Objects.nonNull(kafkaConnectConfig.getBootstrapServers())) { + this.connectorConfig.setBootstrapServers(String.join(",", kafkaConnectConfig.getBootstrapServers())); + } + this.connectorConfig.setAuthProperties(authProperties); + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/source/MongoDBSource.java b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/source/MongoDBSource.java new file mode 100644 index 0000000000..a6fcd45679 --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/source/MongoDBSource.java @@ -0,0 +1,32 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.source; + +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; +import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; +import org.opensearch.dataprepper.model.configuration.PipelineDescription; +import org.opensearch.dataprepper.model.source.Source; +import org.opensearch.dataprepper.plugins.kafka.extension.KafkaClusterConfigSupplier; +import org.opensearch.dataprepper.plugins.kafkaconnect.configuration.MongoDBConfig; +import org.opensearch.dataprepper.plugins.kafkaconnect.extension.KafkaConnectConfigSupplier; + +/** + * The starting point of the mysql source which ingest CDC data using Kafka Connect and Debezium Connector. + */ +@SuppressWarnings("deprecation") +@DataPrepperPlugin(name = "mongodb", pluginType = Source.class, pluginConfigurationType = MongoDBConfig.class) +public class MongoDBSource extends KafkaConnectSource { + + @DataPrepperPluginConstructor + public MongoDBSource(final MongoDBConfig mongoDBConfig, + final PluginMetrics pluginMetrics, + final PipelineDescription pipelineDescription, + final KafkaClusterConfigSupplier kafkaClusterConfigSupplier, + final KafkaConnectConfigSupplier kafkaConnectConfigSupplier) { + super(mongoDBConfig, pluginMetrics, pipelineDescription, kafkaClusterConfigSupplier, kafkaConnectConfigSupplier); + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/source/MySQLSource.java b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/source/MySQLSource.java new file mode 100644 index 0000000000..5dbcd16f4e --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/source/MySQLSource.java @@ -0,0 +1,32 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.source; + +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; +import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; +import org.opensearch.dataprepper.model.configuration.PipelineDescription; +import org.opensearch.dataprepper.model.source.Source; +import org.opensearch.dataprepper.plugins.kafka.extension.KafkaClusterConfigSupplier; +import org.opensearch.dataprepper.plugins.kafkaconnect.configuration.MySQLConfig; +import org.opensearch.dataprepper.plugins.kafkaconnect.extension.KafkaConnectConfigSupplier; + +/** + * The starting point of the mysql source which ingest CDC data using Kafka Connect and Debezium Connector. + */ +@SuppressWarnings("deprecation") +@DataPrepperPlugin(name = "mysql", pluginType = Source.class, pluginConfigurationType = MySQLConfig.class) +public class MySQLSource extends KafkaConnectSource { + + @DataPrepperPluginConstructor + public MySQLSource(final MySQLConfig mySQLConfig, + final PluginMetrics pluginMetrics, + final PipelineDescription pipelineDescription, + final KafkaClusterConfigSupplier kafkaClusterConfigSupplier, + final KafkaConnectConfigSupplier kafkaConnectConfigSupplier) { + super(mySQLConfig, pluginMetrics, pipelineDescription, kafkaClusterConfigSupplier, kafkaConnectConfigSupplier); + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/source/PostgreSQLSource.java b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/source/PostgreSQLSource.java new file mode 100644 index 0000000000..7e7c24d08c --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/source/PostgreSQLSource.java @@ -0,0 +1,32 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.source; + +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; +import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; +import org.opensearch.dataprepper.model.configuration.PipelineDescription; +import org.opensearch.dataprepper.model.source.Source; +import org.opensearch.dataprepper.plugins.kafka.extension.KafkaClusterConfigSupplier; +import org.opensearch.dataprepper.plugins.kafkaconnect.configuration.PostgreSQLConfig; +import org.opensearch.dataprepper.plugins.kafkaconnect.extension.KafkaConnectConfigSupplier; + +/** + * The starting point of the mysql source which ingest CDC data using Kafka Connect and Debezium Connector. + */ +@SuppressWarnings("deprecation") +@DataPrepperPlugin(name = "postgresql", pluginType = Source.class, pluginConfigurationType = PostgreSQLConfig.class) +public class PostgreSQLSource extends KafkaConnectSource { + + @DataPrepperPluginConstructor + public PostgreSQLSource(final PostgreSQLConfig postgreSQLConfig, + final PluginMetrics pluginMetrics, + final PipelineDescription pipelineDescription, + final KafkaClusterConfigSupplier kafkaClusterConfigSupplier, + final KafkaConnectConfigSupplier kafkaConnectConfigSupplier) { + super(postgreSQLConfig, pluginMetrics, pipelineDescription, kafkaClusterConfigSupplier, kafkaConnectConfigSupplier); + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/util/Connector.java b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/util/Connector.java new file mode 100644 index 0000000000..a413a89bbd --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/util/Connector.java @@ -0,0 +1,33 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.util; + +import java.util.Map; + +public class Connector { + private final String name; + private final Map config; + private final Boolean allowReplace; + + public Connector(final String name, final Map config, final Boolean allowReplace) { + this.name = name; + this.config = config; + this.allowReplace = allowReplace; + } + + public String getName() { + return this.name; + } + + public Map getConfig() { + config.putIfAbsent("name", name); + return config; + } + + public Boolean getAllowReplace() { + return allowReplace; + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/util/KafkaConnect.java b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/util/KafkaConnect.java new file mode 100644 index 0000000000..3a1e82bace --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/util/KafkaConnect.java @@ -0,0 +1,375 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.util; + +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.connect.connector.policy.ConnectorClientConfigOverridePolicy; +import org.apache.kafka.connect.errors.AlreadyExistsException; +import org.apache.kafka.connect.errors.NotFoundException; +import org.apache.kafka.connect.json.JsonConverter; +import org.apache.kafka.connect.json.JsonConverterConfig; +import org.apache.kafka.connect.runtime.Connect; +import org.apache.kafka.connect.runtime.Worker; +import org.apache.kafka.connect.runtime.WorkerConfig; +import org.apache.kafka.connect.runtime.WorkerConfigTransformer; +import org.apache.kafka.connect.runtime.distributed.DistributedConfig; +import org.apache.kafka.connect.runtime.distributed.DistributedHerder; +import org.apache.kafka.connect.runtime.distributed.NotLeaderException; +import org.apache.kafka.connect.runtime.isolation.Plugins; +import org.apache.kafka.connect.runtime.rest.ConnectRestServer; +import org.apache.kafka.connect.runtime.rest.RestClient; +import org.apache.kafka.connect.runtime.rest.RestServer; +import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; +import org.apache.kafka.connect.storage.ConfigBackingStore; +import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.connect.storage.KafkaConfigBackingStore; +import org.apache.kafka.connect.storage.KafkaOffsetBackingStore; +import org.apache.kafka.connect.storage.KafkaStatusBackingStore; +import org.apache.kafka.connect.storage.StatusBackingStore; +import org.apache.kafka.connect.util.ConnectUtils; +import org.apache.kafka.connect.util.SharedTopicAdmin; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.plugins.kafkaconnect.meter.KafkaConnectMetrics; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.URI; +import java.time.Clock; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +import static org.apache.kafka.clients.CommonClientConfigs.CLIENT_ID_CONFIG; + +/** + * The KafkaConnect infra. + * Unique with single instance for each pipeline. + */ +public class KafkaConnect { + private static final Logger LOG = LoggerFactory.getLogger(KafkaConnect.class); + private static volatile Map instanceMap = new HashMap<>(); + private static final long RETRY_INTERVAL_MS = 3000L; // 3 seconds + private static final int LATCH_WAIT_TIME = 1; // 1 minute + private static final String RUNNING = "RUNNING"; + private final Map connectorMap; + private final KafkaConnectMetrics kafkaConnectMetrics; + private final Time time = Time.SYSTEM; + private final Clock clock = Clock.systemUTC(); + private DistributedHerder herder; + private RestServer rest; + private Connect connect; + private final long connectTimeoutMs; // 60 seconds + private final long connectorTimeoutMs; // 30 seconds + + private KafkaConnect(final PluginMetrics pluginMetrics, + final Duration connectTimeout, + final Duration connectorTimeout) { + this.connectorMap = new HashMap<>(); + this.kafkaConnectMetrics = new KafkaConnectMetrics(pluginMetrics); + this.connectTimeoutMs = connectTimeout.toMillis(); + this.connectorTimeoutMs = connectorTimeout.toMillis(); + } + + /** + * For Testing + */ + public KafkaConnect(final DistributedHerder herder, + final RestServer rest, + final Connect connect, + final KafkaConnectMetrics kafkaConnectMetrics) { + this.connectorMap = new HashMap<>(); + this.herder = herder; + this.rest = rest; + this.connect = connect; + this.kafkaConnectMetrics = kafkaConnectMetrics; + this.connectTimeoutMs = 60000L; + this.connectorTimeoutMs = 30000L; + } + + public static KafkaConnect getPipelineInstance(final String pipelineName, + final PluginMetrics pluginMetrics, + final Duration connectTimeout, + final Duration connectorTimeout) { + KafkaConnect instance = instanceMap.get(pipelineName); + if (instance == null) { + synchronized (KafkaConnect.class) { + instance = new KafkaConnect(pluginMetrics, connectTimeout, connectorTimeout); + instanceMap.put(pipelineName, instance); + } + } + return instance; + } + + public synchronized void initialize(Map workerProps) { + DistributedConfig config = new DistributedConfig(workerProps); + RestClient restClient = new RestClient(config); + this.rest = new ConnectRestServer(config.rebalanceTimeout(), restClient, workerProps); + this.herder = initHerder(workerProps, config, restClient); + this.connect = new Connect(herder, (ConnectRestServer) rest); + } + + /** + * Add connectors to the Kafka Connect. + * This must be done before the start() is called. + * + * @param connectors connectors to be added. + */ + public void addConnectors(List connectors) { + connectors.forEach(connector -> { + this.connectorMap.put(connector.getName(), connector); + }); + } + + /** + * Start the kafka connect. + * Will add all connectors, and cleanup unused connectors at once. + */ + public synchronized void start() { + if (this.connect == null) { + throw new RuntimeException("Please initialize Kafka Connect first."); + } + if (this.connect.isRunning()) { + LOG.info("Kafka Connect is running, will not start again"); + return; + } + LOG.info("Starting Kafka Connect"); + try { + this.rest.initializeServer(); + this.connect.start(); + waitForConnectRunning(); + this.kafkaConnectMetrics.bindConnectMetrics(); + this.initConnectors(); + } catch (Exception e) { + LOG.error("Failed to start Connect", e); + this.connect.stop(); + throw new RuntimeException(e); + } + } + + /** + * Stop the Kafka Connect. + */ + public void stop() { + if (this.connect == null) { + LOG.info("Kafka Connect is running, will not start again"); + return; + } + LOG.info("Stopping Kafka Connect"); + this.connect.stop(); + } + + private DistributedHerder initHerder(Map workerProps, DistributedConfig config, RestClient restClient) { + LOG.info("Scanning for plugin classes. This might take a moment ..."); + Plugins plugins = new Plugins(workerProps); + plugins.compareAndSwapWithDelegatingLoader(); + String kafkaClusterId = config.kafkaClusterId(); + LOG.info("Kafka cluster ID: {}", kafkaClusterId); + + URI advertisedUrl = rest.advertisedUrl(); + String workerId = advertisedUrl.getHost() + ":" + advertisedUrl.getPort(); + + String clientIdBase = ConnectUtils.clientIdBase(config); + + // Create the admin client to be shared by all backing stores. + Map adminProps = new HashMap<>(config.originals()); + ConnectUtils.addMetricsContextProperties(adminProps, config, kafkaClusterId); + adminProps.put(CLIENT_ID_CONFIG, clientIdBase + "shared-admin"); + SharedTopicAdmin sharedAdmin = new SharedTopicAdmin(adminProps); + + KafkaOffsetBackingStore offsetBackingStore = new KafkaOffsetBackingStore(sharedAdmin, () -> clientIdBase, + plugins.newInternalConverter(true, JsonConverter.class.getName(), + Collections.singletonMap(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, "false"))); + offsetBackingStore.configure(config); + + ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy = plugins.newPlugin( + config.getString(WorkerConfig.CONNECTOR_CLIENT_POLICY_CLASS_CONFIG), + config, ConnectorClientConfigOverridePolicy.class); + + Worker worker = new Worker(workerId, time, plugins, config, offsetBackingStore, connectorClientConfigOverridePolicy); + WorkerConfigTransformer configTransformer = worker.configTransformer(); + + Converter internalValueConverter = worker.getInternalValueConverter(); + StatusBackingStore statusBackingStore = new KafkaStatusBackingStore(time, internalValueConverter, sharedAdmin, clientIdBase); + statusBackingStore.configure(config); + + ConfigBackingStore configBackingStore = new KafkaConfigBackingStore( + internalValueConverter, + config, + configTransformer, + sharedAdmin, + clientIdBase); + + // Pass the shared admin to the distributed herder as an additional AutoCloseable object that should be closed when the + // herder is stopped. This is easier than having to track and own the lifecycle ourselves. + return new DistributedHerder(config, time, worker, + kafkaClusterId, statusBackingStore, configBackingStore, + advertisedUrl.toString(), restClient, connectorClientConfigOverridePolicy, + Collections.emptyList(), sharedAdmin); + } + + /** + * + * @throws InterruptedException + */ + private void waitForConnectRunning() throws InterruptedException { + long startTime = clock.millis(); + boolean isRunning = false; + while (clock.millis() - startTime < connectTimeoutMs) { + LOG.info("Waiting Kafka Connect running"); + isRunning = this.connect.isRunning(); + if (isRunning) break; + TimeUnit.MILLISECONDS.sleep(RETRY_INTERVAL_MS); + } + if (!isRunning) { + throw new RuntimeException("Timed out waiting for Kafka Connect running"); + } + LOG.info("Kafka Connect is running"); + } + + /** + * Initialize connectors. + * The Kafka Connectors are managed in orders: + * 1. Delete Connectors not in pipeline configurations + * 2. Register Connectors + * 3. Wait for all connectors in running state. + * 4. Bind connectors' metrics + */ + private void initConnectors() throws InterruptedException { + this.deleteConnectors(); + this.registerConnectors(); + this.waitForConnectorsRunning(); + this.kafkaConnectMetrics.bindConnectorMetrics(); + } + + /** + * Register Connector to Kafka Connect. + * Designed as private method to prevent register the connector after connect is started. + */ + private void registerConnectors() throws InterruptedException { + CountDownLatch connectorLatch = new CountDownLatch(connectorMap.size()); + List exceptionMessages = new ArrayList<>(); + connectorMap.forEach((connectorName, connector) -> { + herder.connectorConfig(connectorName, (e, config) -> { + boolean shouldUpdate; + if (config == null) { + shouldUpdate = true; + } else { + shouldUpdate = connector.getAllowReplace() || (!config.equals(connector.getConfig())); + } + herder.putConnectorConfig(connectorName, connector.getConfig(), shouldUpdate, (error, result) -> { + if (error != null) { + if (error instanceof NotLeaderException || error instanceof AlreadyExistsException) { + LOG.info(error.getMessage()); + } else { + LOG.error("Failed to put connector config: {}", connectorName); + exceptionMessages.add(error.getMessage()); + } + } else { + // Handle the successful registration + LOG.info("Success put connector config: {}", connectorName); + } + connectorLatch.countDown(); + }); + }); + }); + // Block and wait for all tasks to complete + if (!connectorLatch.await(LATCH_WAIT_TIME, TimeUnit.MINUTES)) { + throw new RuntimeException("Timed out waiting for initConnectors"); + } else { + if (!exceptionMessages.isEmpty()) { + throw new RuntimeException(String.join(", ", exceptionMessages)); + } + LOG.info("InitConnectors completed"); + } + } + + /** + * Delete Connectors from Kafka Connect. + * Designed as private method to prevent delete the connector after connect is started. + */ + private void deleteConnectors() throws InterruptedException { + Collection connectorsToDelete = this.herder.connectors() + .stream() + .filter(connectorName -> !connectorMap.containsKey(connectorName)) + .collect(Collectors.toList()); + List exceptionMessages = new ArrayList<>(); + CountDownLatch deleteLatch = new CountDownLatch(connectorsToDelete.size()); + connectorsToDelete.forEach(connectorName -> { + herder.deleteConnectorConfig(connectorName, (error, result) -> { + if (error != null) { + if (error instanceof NotLeaderException || error instanceof NotFoundException) { + LOG.info(error.getMessage()); + } else { + LOG.error("Failed to delete connector config: {}", connectorName); + exceptionMessages.add(error.getMessage()); + } + } else { + // Handle the successful registration + LOG.info("Success delete connector config: {}", connectorName); + } + deleteLatch.countDown(); + }); + }); + // Block and wait for all tasks to complete + if (!deleteLatch.await(LATCH_WAIT_TIME, TimeUnit.MINUTES)) { + throw new RuntimeException("Timed out waiting for deleteConnectors"); + } else { + if (!exceptionMessages.isEmpty()) { + throw new RuntimeException(String.join(", ", exceptionMessages)); + } + LOG.info("deleteConnectors completed"); + } + } + + private void waitForConnectorsRunning() throws InterruptedException { + LOG.info("Waiting for connectors to be running"); + Set connectorNames = this.connectorMap.keySet(); + List exceptionMessages = new ArrayList<>(); + CountDownLatch countDownLatch = new CountDownLatch(connectorNames.size()); + connectorNames.parallelStream().forEach(connectorName -> { + long startTime = clock.millis(); + boolean isRunning = false; + while (clock.millis() - startTime < connectorTimeoutMs) { + try { + ConnectorStateInfo info = herder.connectorStatus(connectorName); + if (RUNNING.equals(info.connector().state())) { + // Connector is running, decrement the latch count + isRunning = true; + break; + } + } catch (Exception e) { + LOG.info(e.getMessage()); + } + try { + TimeUnit.MILLISECONDS.sleep(RETRY_INTERVAL_MS); + } catch (InterruptedException e) { + break; + } + } + countDownLatch.countDown(); + if (!isRunning) { + exceptionMessages.add(String.format("Connector %s is not running in desired period of time", connectorName)); + } + }); + // Block and wait for all tasks to complete + if (!countDownLatch.await(LATCH_WAIT_TIME, TimeUnit.MINUTES)) { + throw new RuntimeException("Timed out waiting for running state check"); + } else { + if (!exceptionMessages.isEmpty()) { + throw new RuntimeException(String.join(", ", exceptionMessages)); + } + LOG.info("All connectors are running"); + } + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/util/SecretManagerHelper.java b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/util/SecretManagerHelper.java new file mode 100644 index 0000000000..a4a31c52bb --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafkaconnect/util/SecretManagerHelper.java @@ -0,0 +1,58 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.util; + +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.core.retry.RetryPolicy; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.secretsmanager.SecretsManagerClient; +import software.amazon.awssdk.services.secretsmanager.model.GetSecretValueRequest; +import software.amazon.awssdk.services.secretsmanager.model.GetSecretValueResponse; +import software.amazon.awssdk.services.sts.StsClient; +import software.amazon.awssdk.services.sts.auth.StsAssumeRoleCredentialsProvider; +import software.amazon.awssdk.services.sts.model.AssumeRoleRequest; + +import java.util.UUID; + +public class SecretManagerHelper { + private static final String SESSION_PREFIX = "data-prepper-secretmanager-session"; + public static String getSecretValue(final String stsRoleArn, final String region, final String secretId) { + AwsCredentialsProvider credentialsProvider = DefaultCredentialsProvider.create(); + ClientOverrideConfiguration clientOverrideConfiguration = ClientOverrideConfiguration + .builder() + .retryPolicy(RetryPolicy.defaultRetryPolicy()) + .build(); + + if (stsRoleArn != null && !stsRoleArn.isEmpty()) { + String sessionName = SESSION_PREFIX + UUID.randomUUID(); + StsClient stsClient = StsClient.builder() + .overrideConfiguration(clientOverrideConfiguration) + .region(Region.of(region)) + .credentialsProvider(credentialsProvider) + .build(); + AssumeRoleRequest assumeRoleRequest = AssumeRoleRequest + .builder() + .roleArn(stsRoleArn) + .roleSessionName(sessionName) + .build(); + credentialsProvider = StsAssumeRoleCredentialsProvider + .builder() + .stsClient(stsClient) + .refreshRequest(assumeRoleRequest) + .build(); + } + SecretsManagerClient secretsManagerClient = SecretsManagerClient.builder() + .overrideConfiguration(clientOverrideConfiguration) + .credentialsProvider(credentialsProvider) + .region(Region.of(region)) + .build(); + final GetSecretValueRequest request = GetSecretValueRequest.builder().secretId(secretId).build(); + final GetSecretValueResponse response = secretsManagerClient.getSecretValue(request); + return response.secretString(); + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/configuration/CredentialConfigTest.java b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/configuration/CredentialConfigTest.java new file mode 100644 index 0000000000..e36af1e63a --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/configuration/CredentialConfigTest.java @@ -0,0 +1,87 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.configuration; + +import org.junit.jupiter.api.Test; +import org.mockito.MockedStatic; +import org.opensearch.dataprepper.plugins.kafkaconnect.util.SecretManagerHelper; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.mockStatic; + +public class CredentialConfigTest { + private final String testUserName = "testUser"; + private final String testPassword = "testPassword"; + private final String testStsRole = "testRole"; + private final String testRegion = "testRegion"; + private final String testSecretId = "testSecritId"; + + @Test + void test_credential_config_plaintext() { + CredentialsConfig credentialsConfig = new CredentialsConfig( + new CredentialsConfig.PlainText(testUserName, testPassword), null); + assertThat(credentialsConfig.getUsername(), is(testUserName)); + assertThat(credentialsConfig.getPassword(), is(testPassword)); + } + + @Test + void test_credential_config_plaintext_invalid() { + assertThrows(IllegalArgumentException.class, () -> new CredentialsConfig( + new CredentialsConfig.PlainText(null, null), null)); + assertThrows(IllegalArgumentException.class, () -> new CredentialsConfig( + new CredentialsConfig.PlainText(testUserName, null), null)); + assertThrows(IllegalArgumentException.class, () -> new CredentialsConfig( + new CredentialsConfig.PlainText(null, testPassword), null)); + } + + @Test + void test_credential_config_secret_manager() { + final String expectedSecret = "{\"username\":\"expectedUsername\",\"password\":\"expectedPassword\"}"; + try (MockedStatic mockedStatic = mockStatic(SecretManagerHelper.class)) { + mockedStatic.when(() -> SecretManagerHelper.getSecretValue(testStsRole, testRegion, testSecretId)).thenReturn(expectedSecret); + CredentialsConfig credentialsConfig = new CredentialsConfig( + null, new CredentialsConfig.SecretManager(testStsRole, testRegion, testSecretId)); + assertThat(credentialsConfig.getUsername(), is("expectedUsername")); + assertThat(credentialsConfig.getPassword(), is("expectedPassword")); + } + } + + @Test + void test_credential_config_failure_on_secret_manager() { + try (MockedStatic mockedStatic = mockStatic(SecretManagerHelper.class)) { + mockedStatic.when(() -> SecretManagerHelper.getSecretValue(testStsRole, testRegion, testSecretId)).thenThrow(new RuntimeException()); + assertThrows(RuntimeException.class, () -> new CredentialsConfig( + null, new CredentialsConfig.SecretManager(testStsRole, testRegion, testSecretId))); + final String invalidSecret = "{}"; + mockedStatic.when(() -> SecretManagerHelper.getSecretValue(testStsRole, testRegion, testSecretId)).thenReturn(invalidSecret); + assertThrows(RuntimeException.class, () -> new CredentialsConfig( + null, new CredentialsConfig.SecretManager(testStsRole, testRegion, testSecretId))); + } + } + + @Test + void test_credential_config_secret_manager_invalid_input() { + assertThrows(IllegalArgumentException.class, () -> new CredentialsConfig( + null, new CredentialsConfig.SecretManager(null, null, null))); + assertThrows(IllegalArgumentException.class, () -> new CredentialsConfig( + null, new CredentialsConfig.SecretManager(null, null, testSecretId))); + assertThrows(IllegalArgumentException.class, () -> new CredentialsConfig( + null, new CredentialsConfig.SecretManager(null, testRegion, null))); + } + + @Test + void test_invalid_credential_config() { + // Must be set + assertThrows(IllegalArgumentException.class, () -> new CredentialsConfig(null, null)); + // Cannot both set + assertThrows(IllegalArgumentException.class, () -> new CredentialsConfig( + new CredentialsConfig.PlainText(testUserName, testPassword), + new CredentialsConfig.SecretManager(testStsRole, testRegion, testSecretId) + )); + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/configuration/MongoDBConfigTest.java b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/configuration/MongoDBConfigTest.java new file mode 100644 index 0000000000..bf6492441c --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/configuration/MongoDBConfigTest.java @@ -0,0 +1,65 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.configuration; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.plugins.kafkaconnect.util.Connector; +import org.yaml.snakeyaml.Yaml; + +import java.io.FileReader; +import java.io.IOException; +import java.io.Reader; +import java.io.StringReader; +import java.util.Map; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.notNullValue; + +public class MongoDBConfigTest { + + @Test + public void test_get_mongodb_connectors() throws IOException { + MongoDBConfig testConfig = buildTestConfig("sample-mongodb-pipeline.yaml"); + assertThat(testConfig, notNullValue()); + assertThat(testConfig.buildConnectors(), notNullValue()); + assertThat(testConfig.buildConnectors().size(), is(1)); + // verify Connector + Connector mongodbConnector = testConfig.buildConnectors().get(0); + assertThat(mongodbConnector, instanceOf(Connector.class)); + final Map actualConfig = mongodbConnector.getConfig(); + assertThat(actualConfig.get("connector.class"), is(MongoDBConfig.CONNECTOR_CLASS)); + assertThat(actualConfig.get("mongodb.connection.string"), is("mongodb://localhost:27017/?replicaSet=rs0&directConnection=true")); + assertThat(actualConfig.get("mongodb.user"), is("debezium")); + assertThat(actualConfig.get("mongodb.password"), is("dbz")); + assertThat(actualConfig.get("snapshot.mode"), is("initial")); + assertThat(actualConfig.get("topic.prefix"), is("prefix1")); + assertThat(actualConfig.get("collection.include.list"), is("test.customers")); + assertThat(actualConfig.get("mongodb.ssl.enabled"), is("false")); + } + + private MongoDBConfig buildTestConfig(final String resourceFileName) throws IOException { + //Added to load Yaml file - Start + Yaml yaml = new Yaml(); + FileReader fileReader = new FileReader(getClass().getClassLoader().getResource(resourceFileName).getFile()); + Object data = yaml.load(fileReader); + if (data instanceof Map) { + Map propertyMap = (Map) data; + Map logPipelineMap = (Map) propertyMap.get("log-pipeline"); + Map sourceMap = (Map) logPipelineMap.get("source"); + Map kafkaConnectConfigMap = (Map) sourceMap.get("mongodb"); + ObjectMapper mapper = new ObjectMapper(); + mapper.registerModule(new JavaTimeModule()); + String json = mapper.writeValueAsString(kafkaConnectConfigMap); + Reader reader = new StringReader(json); + return mapper.readValue(reader, MongoDBConfig.class); + } + return null; + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/configuration/MySQLConfigTest.java b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/configuration/MySQLConfigTest.java new file mode 100644 index 0000000000..fa4d3526da --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/configuration/MySQLConfigTest.java @@ -0,0 +1,76 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.configuration; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.plugins.kafkaconnect.util.Connector; +import org.yaml.snakeyaml.Yaml; + +import java.io.FileReader; +import java.io.IOException; +import java.io.Reader; +import java.io.StringReader; +import java.util.Map; +import java.util.Properties; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.notNullValue; + +public class MySQLConfigTest { + @Test + public void test_get_mysql_connectors() throws IOException { + final String bootstrapServers = "localhost:9092"; + final Properties authProperties = new Properties(); + authProperties.put("bootstrap.servers", bootstrapServers); + authProperties.put("testClass", this.getClass()); + authProperties.put("testKey", "testValue"); + MySQLConfig testConfig = buildTestConfig("sample-mysql-pipeline.yaml"); + assertThat(testConfig, notNullValue()); + assertThat(testConfig.buildConnectors(), notNullValue()); + assertThat(testConfig.buildConnectors().size(), is(1)); + // verify Connector + testConfig.setAuthProperties(authProperties); + Connector mysqlConnector = testConfig.buildConnectors().get(0); + assertThat(mysqlConnector, instanceOf(Connector.class)); + final Map actualConfig = mysqlConnector.getConfig(); + assertThat(actualConfig.get("connector.class"), is(MySQLConfig.CONNECTOR_CLASS)); + assertThat(actualConfig.get("database.hostname"), is("localhost")); + assertThat(actualConfig.get("database.port"), is("3306")); + assertThat(actualConfig.get("database.user"), is("debezium")); + assertThat(actualConfig.get("database.password"), is("dbz")); + assertThat(actualConfig.get("snapshot.mode"), is("initial")); + assertThat(actualConfig.get("topic.prefix"), is("prefix1")); + assertThat(actualConfig.get("table.include.list"), is("inventory.customers")); + assertThat(actualConfig.get("schema.history.internal.kafka.bootstrap.servers"), is(bootstrapServers)); + assertThat(actualConfig.get("schema.history.internal.producer.testKey"), is(authProperties.getProperty("testKey"))); + assertThat(actualConfig.get("schema.history.internal.consumer.testKey"), is(authProperties.getProperty("testKey"))); + assertThat(actualConfig.get("schema.history.internal.producer.testClass"), is(this.getClass().getName())); + assertThat(actualConfig.get("schema.history.internal.consumer.testClass"), is(this.getClass().getName())); + } + + private MySQLConfig buildTestConfig(final String resourceFileName) throws IOException { + //Added to load Yaml file - Start + Yaml yaml = new Yaml(); + FileReader fileReader = new FileReader(getClass().getClassLoader().getResource(resourceFileName).getFile()); + Object data = yaml.load(fileReader); + if (data instanceof Map) { + Map propertyMap = (Map) data; + Map logPipelineMap = (Map) propertyMap.get("log-pipeline"); + Map sourceMap = (Map) logPipelineMap.get("source"); + Map kafkaConnectConfigMap = (Map) sourceMap.get("mysql"); + ObjectMapper mapper = new ObjectMapper(); + mapper.registerModule(new JavaTimeModule()); + String json = mapper.writeValueAsString(kafkaConnectConfigMap); + Reader reader = new StringReader(json); + return mapper.readValue(reader, MySQLConfig.class); + } + return null; + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/configuration/PostgreSQLConfigTest.java b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/configuration/PostgreSQLConfigTest.java new file mode 100644 index 0000000000..036cda30aa --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/configuration/PostgreSQLConfigTest.java @@ -0,0 +1,67 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.configuration; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.plugins.kafkaconnect.util.Connector; +import org.yaml.snakeyaml.Yaml; + +import java.io.FileReader; +import java.io.IOException; +import java.io.Reader; +import java.io.StringReader; +import java.util.Map; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.notNullValue; + +public class PostgreSQLConfigTest { + @Test + public void test_get_postgresql_connectors() throws IOException { + PostgreSQLConfig testConfig = buildTestConfig("sample-postgres-pipeline.yaml"); + assertThat(testConfig, notNullValue()); + assertThat(testConfig.buildConnectors(), notNullValue()); + assertThat(testConfig.buildConnectors().size(), is(1)); + // verify Connector + Connector postgresqlConnector = testConfig.buildConnectors().get(0); + assertThat(postgresqlConnector, instanceOf(Connector.class)); + assertThat(postgresqlConnector.getName(), is("psql.public.customers")); + final Map actualConfig = postgresqlConnector.getConfig(); + assertThat(actualConfig.get("connector.class"), is(PostgreSQLConfig.CONNECTOR_CLASS)); + assertThat(actualConfig.get("plugin.name"), is("pgoutput")); + assertThat(actualConfig.get("database.hostname"), is("localhost")); + assertThat(actualConfig.get("database.port"), is("5432")); + assertThat(actualConfig.get("database.user"), is("debezium")); + assertThat(actualConfig.get("database.password"), is("dbz")); + assertThat(actualConfig.get("snapshot.mode"), is("initial")); + assertThat(actualConfig.get("topic.prefix"), is("psql")); + assertThat(actualConfig.get("database.dbname"), is("postgres")); + assertThat(actualConfig.get("table.include.list"), is("public.customers")); + } + + private PostgreSQLConfig buildTestConfig(final String resourceFileName) throws IOException { + //Added to load Yaml file - Start + Yaml yaml = new Yaml(); + FileReader fileReader = new FileReader(getClass().getClassLoader().getResource(resourceFileName).getFile()); + Object data = yaml.load(fileReader); + if (data instanceof Map) { + Map propertyMap = (Map) data; + Map logPipelineMap = (Map) propertyMap.get("log-pipeline"); + Map sourceMap = (Map) logPipelineMap.get("source"); + Map kafkaConnectConfigMap = (Map) sourceMap.get("postgresql"); + ObjectMapper mapper = new ObjectMapper(); + mapper.registerModule(new JavaTimeModule()); + String json = mapper.writeValueAsString(kafkaConnectConfigMap); + Reader reader = new StringReader(json); + return mapper.readValue(reader, PostgreSQLConfig.class); + } + return null; + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/DefaultKafkaConnectConfigSupplierTest.java b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/DefaultKafkaConnectConfigSupplierTest.java new file mode 100644 index 0000000000..d4bfb32504 --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/DefaultKafkaConnectConfigSupplierTest.java @@ -0,0 +1,29 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.extension; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; + +@ExtendWith(MockitoExtension.class) +public class DefaultKafkaConnectConfigSupplierTest { + @Mock + private KafkaConnectConfig kafkaConnectConfig; + + private DefaultKafkaConnectConfigSupplier createObjectUnderTest() { + return new DefaultKafkaConnectConfigSupplier(kafkaConnectConfig); + } + + @Test + void test_get_config() { + assertThat(createObjectUnderTest().getConfig(), equalTo(kafkaConnectConfig)); + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/KafkaConnectConfigExtensionTest.java b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/KafkaConnectConfigExtensionTest.java new file mode 100644 index 0000000000..e2a705978b --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/KafkaConnectConfigExtensionTest.java @@ -0,0 +1,44 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.extension; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.plugin.ExtensionPoints; +import org.opensearch.dataprepper.model.plugin.ExtensionProvider; + +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.Mockito.verify; + +@ExtendWith(MockitoExtension.class) +public class KafkaConnectConfigExtensionTest { + @Mock + private ExtensionPoints extensionPoints; + + @Mock + private KafkaConnectConfig kafkaConnectConfig; + + private KafkaConnectConfigExtension createObjectUnderTest() { + return new KafkaConnectConfigExtension(kafkaConnectConfig); + } + + @Test + void apply_should_addExtensionProvider() { + createObjectUnderTest().apply(extensionPoints); + final ArgumentCaptor extensionProviderArgumentCaptor = + ArgumentCaptor.forClass(ExtensionProvider.class); + + verify(extensionPoints).addExtensionProvider(extensionProviderArgumentCaptor.capture()); + + final ExtensionProvider actualExtensionProvider = extensionProviderArgumentCaptor.getValue(); + + assertThat(actualExtensionProvider, instanceOf(KafkaConnectConfigProvider.class)); + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/KafkaConnectConfigProviderTest.java b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/KafkaConnectConfigProviderTest.java new file mode 100644 index 0000000000..3162f0d193 --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/KafkaConnectConfigProviderTest.java @@ -0,0 +1,52 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.extension; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.plugin.ExtensionProvider; + +import java.util.Optional; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.CoreMatchers.sameInstance; +import static org.hamcrest.MatcherAssert.assertThat; + +@ExtendWith(MockitoExtension.class) +public class KafkaConnectConfigProviderTest { + @Mock + private KafkaConnectConfigSupplier kafkaConnectConfigSupplier; + + @Mock + private ExtensionProvider.Context context; + + private KafkaConnectConfigProvider createObjectUnderTest() { + return new KafkaConnectConfigProvider(kafkaConnectConfigSupplier); + } + + @Test + void supportedClass_returns_kafkaConnectConfigSupplier() { + assertThat(createObjectUnderTest().supportedClass(), equalTo(KafkaConnectConfigSupplier.class)); + } + + @Test + void provideInstance_returns_the_kafkaConnectConfigSupplier_from_the_constructor() { + final KafkaConnectConfigProvider objectUnderTest = createObjectUnderTest(); + + final Optional optionalKafkaConnectConfigSupplier = objectUnderTest.provideInstance(context); + assertThat(optionalKafkaConnectConfigSupplier, notNullValue()); + assertThat(optionalKafkaConnectConfigSupplier.isPresent(), equalTo(true)); + assertThat(optionalKafkaConnectConfigSupplier.get(), equalTo(kafkaConnectConfigSupplier)); + + final Optional anotherOptionalKafkaConnectConfigSupplier = objectUnderTest.provideInstance(context); + assertThat(anotherOptionalKafkaConnectConfigSupplier, notNullValue()); + assertThat(anotherOptionalKafkaConnectConfigSupplier.isPresent(), equalTo(true)); + assertThat(anotherOptionalKafkaConnectConfigSupplier.get(), sameInstance(optionalKafkaConnectConfigSupplier.get())); + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/KafkaConnectConfigTest.java b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/KafkaConnectConfigTest.java new file mode 100644 index 0000000000..672cb43903 --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/extension/KafkaConnectConfigTest.java @@ -0,0 +1,124 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.extension; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; +import org.hamcrest.CoreMatchers; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.model.types.ByteCount; +import org.opensearch.dataprepper.parser.ByteCountDeserializer; +import org.opensearch.dataprepper.parser.DataPrepperDurationDeserializer; +import org.opensearch.dataprepper.parser.model.DataPrepperConfiguration; +import org.opensearch.dataprepper.plugins.kafka.configuration.AuthConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.AwsConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.EncryptionConfig; + +import java.io.File; +import java.io.IOException; +import java.io.Reader; +import java.io.StringReader; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.nullValue; + +public class KafkaConnectConfigTest { + + private static SimpleModule simpleModule = new SimpleModule() + .addDeserializer(Duration.class, new DataPrepperDurationDeserializer()) + .addDeserializer(ByteCount.class, new ByteCountDeserializer()); + private static ObjectMapper OBJECT_MAPPER = new ObjectMapper(new YAMLFactory()).registerModule(simpleModule); + + private KafkaConnectConfig makeConfig(String filePath) throws IOException { + final File configurationFile = new File(filePath); + final DataPrepperConfiguration dataPrepperConfiguration = OBJECT_MAPPER.readValue(configurationFile, DataPrepperConfiguration.class); + assertThat(dataPrepperConfiguration, CoreMatchers.notNullValue()); + assertThat(dataPrepperConfiguration.getPipelineExtensions(), CoreMatchers.notNullValue()); + final Map kafkaConnectConfigMap = (Map) dataPrepperConfiguration.getPipelineExtensions().getExtensionMap().get("kafka_connect_config"); + String json = OBJECT_MAPPER.writeValueAsString(kafkaConnectConfigMap); + Reader reader = new StringReader(json); + return OBJECT_MAPPER.readValue(reader, KafkaConnectConfig.class); + } + + @Test + public void test_config_setter_getter() throws IOException { + KafkaConnectConfig testConfig = makeConfig("src/test/resources/sample-data-prepper-config-with-kafka-connect-config-extension.yaml"); + AuthConfig authConfig = new AuthConfig(); + AwsConfig awsConfig = new AwsConfig(); + EncryptionConfig encryptionConfig = new EncryptionConfig(); + List bootstrapServer = List.of("testhost:123"); + testConfig.setAuthConfig(authConfig); + testConfig.setAwsConfig(awsConfig); + testConfig.setEncryptionConfig(encryptionConfig); + testConfig.setBootstrapServers(bootstrapServer); + assertThat(testConfig.getAuthConfig(), is(authConfig)); + assertThat(testConfig.getAwsConfig(), is(awsConfig)); + assertThat(testConfig.getEncryptionConfig(), is(encryptionConfig)); + assertThat(testConfig.getBootstrapServers(), is(bootstrapServer)); + assertThat(testConfig.getConnectorStartTimeout().getSeconds(), is(3L)); + assertThat(testConfig.getConnectStartTimeout().getSeconds(), is(3L)); + } + + @Test + public void test_config_get_worker_properties() throws IOException { + final String bootstrapServers = "localhost:9092"; + final Properties authProperties = new Properties(); + authProperties.put("bootstrap.servers", bootstrapServers); + authProperties.put("testClass", KafkaConnectConfigTest.class); + authProperties.put("testKey", "testValue"); + KafkaConnectConfig testConfig = makeConfig("src/test/resources/sample-data-prepper-config-with-kafka-connect-config-extension.yaml"); + testConfig.setAuthProperties(authProperties); + // verify WorkerProperties + assertThat(testConfig.getWorkerProperties(), notNullValue()); + Map workerProperties = testConfig.getWorkerProperties().buildKafkaConnectPropertyMap(); + assertThat(workerProperties.get("bootstrap.servers"), is(bootstrapServers)); + assertThat(workerProperties.get("group.id"), is("test-group")); + assertThat(workerProperties.get("client.id"), is("test-client")); + assertThat(workerProperties.get("offset.storage.topic"), is("test-offsets")); + assertThat(workerProperties.get("config.storage.topic"), is("test-configs")); + assertThat(workerProperties.get("status.storage.topic"), is("test-status")); + assertThat(workerProperties.get("key.converter"), is("org.apache.kafka.connect.json.JsonConverter")); + assertThat(workerProperties.get("value.converter"), is("org.apache.kafka.connect.json.JsonConverter")); + assertThat(workerProperties.get("offset.storage.partitions"), is("2")); + assertThat(workerProperties.get("offset.flush.interval.ms"), is("6000")); + assertThat(workerProperties.get("offset.flush.timeout.ms"), is("500")); + assertThat(workerProperties.get("status.storage.partitions"), is("1")); + assertThat(workerProperties.get("heartbeat.interval.ms"), is("300")); + assertThat(workerProperties.get("session.timeout.ms"), is("3000")); + assertThat(workerProperties.get("scheduled.rebalance.max.delay.ms"), is("60000")); + assertThat(workerProperties.get("testClass"), is(this.getClass().getName())); + assertThat(workerProperties.get("producer.testClass"), is(this.getClass().getName())); + assertThat(workerProperties.get("testKey"), is(authProperties.getProperty("testKey"))); + assertThat(workerProperties.get("producer.testKey"), is(authProperties.getProperty("testKey"))); + } + + @Test + public void test_config_default_worker_properties() throws IOException { + KafkaConnectConfig testConfig = makeConfig("src/test/resources/sample-data-prepper-config-with-default-kafka-connect-config-extension.yaml"); + assertThat(testConfig, notNullValue()); + assertThat(testConfig.getConnectStartTimeout().getSeconds(), is(60L)); + assertThat(testConfig.getConnectorStartTimeout().getSeconds(), is(360L)); + assertThat(testConfig.getBootstrapServers(), nullValue()); + WorkerProperties testWorkerProperties = testConfig.getWorkerProperties(); + assertThat(testWorkerProperties, notNullValue()); + Map workerProperties = testWorkerProperties.buildKafkaConnectPropertyMap(); + assertThat(workerProperties.get("bootstrap.servers"), nullValue()); + assertThat(workerProperties.get("offset.storage.partitions"), is("25")); + assertThat(workerProperties.get("offset.flush.interval.ms"), is("60000")); + assertThat(workerProperties.get("offset.flush.timeout.ms"), is("5000")); + assertThat(workerProperties.get("status.storage.partitions"), is("5")); + assertThat(workerProperties.get("heartbeat.interval.ms"), is("3000")); + assertThat(workerProperties.get("session.timeout.ms"), is("30000")); + assertThat(workerProperties.get("scheduled.rebalance.max.delay.ms"), is("300000")); + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/meter/KafkaConnectMetricsTest.java b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/meter/KafkaConnectMetricsTest.java new file mode 100644 index 0000000000..55ef90db5d --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/meter/KafkaConnectMetricsTest.java @@ -0,0 +1,69 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.meter; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.opensearch.dataprepper.metrics.PluginMetrics; + +import javax.management.MBeanServer; +import javax.management.MalformedObjectNameException; +import javax.management.ObjectName; +import java.util.Set; + +import static java.util.Collections.emptyList; +import static java.util.Collections.emptySet; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.notNullValue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class KafkaConnectMetricsTest { + @Mock + private PluginMetrics pluginMetrics; + + @Mock + private MBeanServer mBeanServer; + + private Iterable tags = emptyList(); + + @BeforeEach + void setUp() throws Exception { + pluginMetrics = mock(PluginMetrics.class); + mBeanServer = mock(MBeanServer.class); + lenient().when(mBeanServer.getAttribute(any(), any())).thenReturn(1); + } + + @Test + void testConstructor() { + assertThat(new KafkaConnectMetrics(pluginMetrics), notNullValue()); + when(mBeanServer.queryNames(any(), any())).thenReturn(emptySet()); + assertThat(new KafkaConnectMetrics(pluginMetrics, tags), notNullValue()); + } + + @Test + void testBindConnectMetrics() throws MalformedObjectNameException { + final KafkaConnectMetrics kafkaConnectMetrics = new KafkaConnectMetrics(pluginMetrics, mBeanServer, tags); + when(mBeanServer.queryNames(any(), any())).thenReturn(Set.of(new ObjectName("test:*"))); + kafkaConnectMetrics.bindConnectMetrics(); + verify(mBeanServer).queryNames(any(), any()); + verify(pluginMetrics, atLeastOnce()).gaugeWithTags(any(), any(), any(), any()); + } + + @Test + void testBindConnectorMetrics() throws MalformedObjectNameException { + final KafkaConnectMetrics kafkaConnectMetrics = new KafkaConnectMetrics(pluginMetrics, mBeanServer, tags); + when(mBeanServer.queryNames(any(), any())).thenReturn(Set.of(new ObjectName("test:type=test,connector=test,client-id=test1,node-id=test1,task=task1"))); + kafkaConnectMetrics.bindConnectorMetrics(); + verify(mBeanServer).queryNames(any(), any()); + verify(pluginMetrics, atLeastOnce()).gaugeWithTags(any(), any(), any(), any()); + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/source/KafkaConnectSourceTest.java b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/source/KafkaConnectSourceTest.java new file mode 100644 index 0000000000..0ce37e2227 --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/source/KafkaConnectSourceTest.java @@ -0,0 +1,184 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.source; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.MockedStatic; +import org.mockito.junit.jupiter.MockitoExtension; +import org.mockito.stubbing.Answer; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.configuration.PipelineDescription; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.kafka.configuration.AuthConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.AwsConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.EncryptionConfig; +import org.opensearch.dataprepper.plugins.kafka.extension.KafkaClusterConfigSupplier; +import org.opensearch.dataprepper.plugins.kafka.util.KafkaSecurityConfigurer; +import org.opensearch.dataprepper.plugins.kafkaconnect.configuration.MySQLConfig; +import org.opensearch.dataprepper.plugins.kafkaconnect.extension.KafkaConnectConfig; +import org.opensearch.dataprepper.plugins.kafkaconnect.extension.KafkaConnectConfigSupplier; +import org.opensearch.dataprepper.plugins.kafkaconnect.extension.WorkerProperties; +import org.opensearch.dataprepper.plugins.kafkaconnect.util.KafkaConnect; + +import java.util.Collections; +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +public class KafkaConnectSourceTest { + private final String TEST_PIPELINE_NAME = "test_pipeline"; + private KafkaConnectSource kafkaConnectSource; + + @Mock + private MySQLConfig mySQLConfig; + + @Mock + private KafkaConnectConfig kafkaConnectConfig; + + @Mock + private PluginMetrics pluginMetrics; + + @Mock + private PipelineDescription pipelineDescription; + + @Mock + private Buffer> buffer; + + @Mock + private KafkaConnect kafkaConnect; + + @Mock + private KafkaClusterConfigSupplier kafkaClusterConfigSupplier; + + @Mock + private KafkaConnectConfigSupplier kafkaConnectConfigSupplier; + + private String bootstrapServers = "localhost:9092"; + + public KafkaConnectSource createSourceUnderTest() { + return new MySQLSource(mySQLConfig, pluginMetrics, pipelineDescription, kafkaClusterConfigSupplier, kafkaConnectConfigSupplier); + } + + @BeforeEach + void setUp() { + WorkerProperties workerProperties = new WorkerProperties(); + workerProperties.setBootstrapServers(bootstrapServers); + kafkaConnectConfigSupplier = mock(KafkaConnectConfigSupplier.class); + lenient().when(kafkaConnectConfigSupplier.getConfig()).thenReturn(kafkaConnectConfig); + lenient().when(kafkaConnectConfig.getWorkerProperties()).thenReturn(workerProperties); + lenient().when(mySQLConfig.buildConnectors()).thenReturn(Collections.emptyList()); + + pipelineDescription = mock(PipelineDescription.class); + lenient().when(pipelineDescription.getPipelineName()).thenReturn(TEST_PIPELINE_NAME); + pluginMetrics = mock(PluginMetrics.class); + } + + @Test + void testStartKafkaConnectSource() { + try (MockedStatic mockedStatic = mockStatic(KafkaConnect.class); + MockedStatic mockedSecurityConfigurer = mockStatic(KafkaSecurityConfigurer.class)) { + mockedSecurityConfigurer.when(() -> KafkaSecurityConfigurer.setAuthProperties(any(), any(), any())).thenAnswer((Answer) invocation -> null); + kafkaConnect = mock(KafkaConnect.class); + doNothing().when(kafkaConnect).addConnectors(any()); + doNothing().when(kafkaConnect).start(); + doNothing().when(kafkaConnect).stop(); + // Set up the mock behavior for the static method getInstance() + mockedStatic.when(() -> KafkaConnect.getPipelineInstance(any(), any(), any(), any())).thenReturn(kafkaConnect); + kafkaConnectSource = createSourceUnderTest(); + kafkaConnectSource.start(buffer); + verify(kafkaConnect).addConnectors(any()); + verify(kafkaConnect).start(); + try { + Thread.sleep(10); + } catch (Exception e) { + e.printStackTrace(); + } + kafkaConnectSource.stop(); + verify(kafkaConnect).stop(); + } + } + + @Test + void testStartKafkaConnectSourceError() { + WorkerProperties workerProperties = new WorkerProperties(); + workerProperties.setBootstrapServers(null); + lenient().when(kafkaConnectConfig.getWorkerProperties()).thenReturn(workerProperties); + try (MockedStatic mockedStatic = mockStatic(KafkaConnect.class); + MockedStatic mockedSecurityConfigurer = mockStatic(KafkaSecurityConfigurer.class)) { + mockedSecurityConfigurer.when(() -> KafkaSecurityConfigurer.setAuthProperties(any(), any(), any())).thenAnswer((Answer) invocation -> null); + kafkaConnect = mock(KafkaConnect.class); + // Set up the mock behavior for the static method getInstance() + mockedStatic.when(() -> KafkaConnect.getPipelineInstance(any(), any(), any(), any())).thenReturn(kafkaConnect); + kafkaConnectSource = createSourceUnderTest(); + assertThrows(IllegalArgumentException.class, () -> kafkaConnectSource.start(buffer)); + } + } + + @Test + void test_updateConfig_using_kafkaClusterConfigExtension() { + final List bootstrapServers = List.of("localhost:9092"); + final AuthConfig authConfig = mock(AuthConfig.class); + final AwsConfig awsConfig = mock(AwsConfig.class); + final EncryptionConfig encryptionConfig = mock(EncryptionConfig.class); + doNothing().when(kafkaConnectConfig).setBootstrapServers(any()); + doNothing().when(kafkaConnectConfig).setAuthConfig(any()); + doNothing().when(kafkaConnectConfig).setAwsConfig(any()); + doNothing().when(kafkaConnectConfig).setEncryptionConfig(any()); + when(kafkaConnectConfig.getAuthConfig()).thenReturn(null); + when(kafkaConnectConfig.getAwsConfig()).thenReturn(null); + when(kafkaConnectConfig.getEncryptionConfig()).thenReturn(null); + when(kafkaConnectConfig.getBootstrapServers()).thenReturn(null); + when(kafkaClusterConfigSupplier.getBootStrapServers()).thenReturn(bootstrapServers); + when(kafkaClusterConfigSupplier.getAuthConfig()).thenReturn(authConfig); + when(kafkaClusterConfigSupplier.getAwsConfig()).thenReturn(awsConfig); + when(kafkaClusterConfigSupplier.getEncryptionConfig()).thenReturn(encryptionConfig); + try (MockedStatic mockedStatic = mockStatic(KafkaSecurityConfigurer.class)) { + mockedStatic.when(() -> KafkaSecurityConfigurer.setAuthProperties(any(), any(), any())).thenAnswer((Answer) invocation -> null); + kafkaConnectSource = createSourceUnderTest(); + verify(kafkaConnectConfig).setBootstrapServers(bootstrapServers); + verify(kafkaConnectConfig).setAuthConfig(authConfig); + verify(kafkaConnectConfig).setAwsConfig(awsConfig); + verify(kafkaConnectConfig).setEncryptionConfig(encryptionConfig); + } + } + + @Test + void test_updateConfig_not_using_kafkaClusterConfigExtension() { + final List bootstrapServers = List.of("localhost:9092"); + final AuthConfig authConfig = mock(AuthConfig.class); + final AwsConfig awsConfig = mock(AwsConfig.class); + final EncryptionConfig encryptionConfig = mock(EncryptionConfig.class); + lenient().doNothing().when(kafkaConnectConfig).setBootstrapServers(any()); + lenient().doNothing().when(kafkaConnectConfig).setAuthConfig(any()); + lenient().doNothing().when(kafkaConnectConfig).setAwsConfig(any()); + lenient().doNothing().when(kafkaConnectConfig).setEncryptionConfig(any()); + lenient().when(kafkaConnectConfig.getAuthConfig()).thenReturn(authConfig); + lenient().when(kafkaConnectConfig.getAwsConfig()).thenReturn(awsConfig); + lenient().when(kafkaConnectConfig.getEncryptionConfig()).thenReturn(encryptionConfig); + lenient().when(kafkaConnectConfig.getBootstrapServers()).thenReturn(bootstrapServers); + try (MockedStatic mockedStatic = mockStatic(KafkaSecurityConfigurer.class)) { + mockedStatic.when(() -> KafkaSecurityConfigurer.setAuthProperties(any(), any(), any())).thenAnswer((Answer) invocation -> null); + kafkaConnectSource = createSourceUnderTest(); + verify(kafkaConnectConfig, never()).setBootstrapServers(any()); + verify(kafkaConnectConfig, never()).setAuthConfig(any()); + verify(kafkaConnectConfig, never()).setAwsConfig(any()); + verify(kafkaConnectConfig, never()).setEncryptionConfig(any()); + } + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/source/MongoDBSourceTest.java b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/source/MongoDBSourceTest.java new file mode 100644 index 0000000000..e7d00f7d96 --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/source/MongoDBSourceTest.java @@ -0,0 +1,33 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.source; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.configuration.PipelineDescription; +import org.opensearch.dataprepper.plugins.kafkaconnect.configuration.MongoDBConfig; + +import static org.junit.jupiter.api.Assertions.assertThrows; + +@ExtendWith(MockitoExtension.class) +public class MongoDBSourceTest { + @Mock + private MongoDBConfig mongoDBConfig; + + @Mock + private PluginMetrics pluginMetrics; + + @Mock + private PipelineDescription pipelineDescription; + + @Test + void testConstructorValidations() { + assertThrows(IllegalArgumentException.class, () -> new MongoDBSource(mongoDBConfig, pluginMetrics, pipelineDescription, null, null)); + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/source/MySQLSourceTest.java b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/source/MySQLSourceTest.java new file mode 100644 index 0000000000..3c0fbb0046 --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/source/MySQLSourceTest.java @@ -0,0 +1,33 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.source; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.configuration.PipelineDescription; +import org.opensearch.dataprepper.plugins.kafkaconnect.configuration.MySQLConfig; + +import static org.junit.jupiter.api.Assertions.assertThrows; + +@ExtendWith(MockitoExtension.class) +public class MySQLSourceTest { + @Mock + private MySQLConfig mySQLConfig; + + @Mock + private PluginMetrics pluginMetrics; + + @Mock + private PipelineDescription pipelineDescription; + + @Test + void testConstructorValidations() { + assertThrows(IllegalArgumentException.class, () -> new MySQLSource(mySQLConfig, pluginMetrics, pipelineDescription, null, null)); + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/source/PostgreSQLSourceTest.java b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/source/PostgreSQLSourceTest.java new file mode 100644 index 0000000000..2cdf8973cf --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/source/PostgreSQLSourceTest.java @@ -0,0 +1,33 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.source; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.configuration.PipelineDescription; +import org.opensearch.dataprepper.plugins.kafkaconnect.configuration.PostgreSQLConfig; + +import static org.junit.jupiter.api.Assertions.assertThrows; + +@ExtendWith(MockitoExtension.class) +public class PostgreSQLSourceTest { + @Mock + private PostgreSQLConfig postgreSQLConfig; + + @Mock + private PluginMetrics pluginMetrics; + + @Mock + private PipelineDescription pipelineDescription; + + @Test + void testConstructorValidations() { + assertThrows(IllegalArgumentException.class, () -> new PostgreSQLSource(postgreSQLConfig, pluginMetrics, pipelineDescription, null, null)); + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/util/ConnectorTest.java b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/util/ConnectorTest.java new file mode 100644 index 0000000000..f8d377ae37 --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/util/ConnectorTest.java @@ -0,0 +1,28 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.util; + +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; + +public class ConnectorTest { + @Test + void testGettersOfConnector() { + final String name = "connectorName"; + final Boolean allowReplace = false; + final Map config = new HashMap<>(); + final Connector connector = new Connector(name, config, allowReplace); + assertThat(connector.getName(), is(name)); + assertThat(connector.getConfig(), is(config)); + assertThat(connector.getConfig().get("name"), is(name)); + assertThat(connector.getAllowReplace(), is(allowReplace)); + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/util/KafkaConnectTest.java b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/util/KafkaConnectTest.java new file mode 100644 index 0000000000..eb13027378 --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/util/KafkaConnectTest.java @@ -0,0 +1,342 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.util; + +import org.apache.kafka.connect.connector.policy.ConnectorClientConfigOverridePolicy; +import org.apache.kafka.connect.errors.AlreadyExistsException; +import org.apache.kafka.connect.errors.NotFoundException; +import org.apache.kafka.connect.runtime.Connect; +import org.apache.kafka.connect.runtime.Herder; +import org.apache.kafka.connect.runtime.Worker; +import org.apache.kafka.connect.runtime.WorkerConfigTransformer; +import org.apache.kafka.connect.runtime.distributed.DistributedConfig; +import org.apache.kafka.connect.runtime.distributed.DistributedHerder; +import org.apache.kafka.connect.runtime.distributed.NotLeaderException; +import org.apache.kafka.connect.runtime.isolation.Plugins; +import org.apache.kafka.connect.runtime.rest.ConnectRestServer; +import org.apache.kafka.connect.runtime.rest.RestClient; +import org.apache.kafka.connect.runtime.rest.RestServer; +import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo; +import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; +import org.apache.kafka.connect.runtime.rest.entities.ConnectorType; +import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.connect.storage.KafkaOffsetBackingStore; +import org.apache.kafka.connect.util.Callback; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.mockito.MockedConstruction; +import org.mockito.MockedStatic; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.plugins.kafkaconnect.extension.WorkerProperties; +import org.opensearch.dataprepper.plugins.kafkaconnect.meter.KafkaConnectMetrics; + +import java.net.URI; +import java.time.Clock; +import java.time.Duration; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.not; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockConstruction; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class KafkaConnectTest { + private static final String TEST_PIPELINE_NAME = "test"; + private static final WorkerProperties DEFAULT_WORDER_PROPERTY = new WorkerProperties(); + private static final long TEST_CONNECTOR_TIMEOUT_MS = 360000L; // 360 seconds + private static final long TEST_CONNECT_TIMEOUT_MS = 60000L; // 60 seconds + private static final Duration TEST_CONNECTOR_TIMEOUT = Duration.ofMillis(TEST_CONNECTOR_TIMEOUT_MS); + private static final Duration TEST_CONNECT_TIMEOUT = Duration.ofMillis(TEST_CONNECT_TIMEOUT_MS); + @Mock + private KafkaConnectMetrics kafkaConnectMetrics; + + @Mock + private PluginMetrics pluginMetrics; + + @Mock + private DistributedHerder distributedHerder; + + @Mock + private RestServer rest; + + @Mock + private Connect connect; + + + @BeforeEach + void setUp() throws Exception { + kafkaConnectMetrics = mock(KafkaConnectMetrics.class); + distributedHerder = mock(DistributedHerder.class); + rest = mock(RestServer.class); + connect = mock(Connect.class); + DEFAULT_WORDER_PROPERTY.setBootstrapServers("localhost:9002"); + + lenient().when(connect.isRunning()).thenReturn(false).thenReturn(true); + lenient().when(distributedHerder.connectors()).thenReturn(new ArrayList<>()); + ConnectorStateInfo runningState = new ConnectorStateInfo("newConnector", new ConnectorStateInfo.ConnectorState("RUNNING", "worker", "msg"), new ArrayList<>(), ConnectorType.SOURCE); + lenient().when(distributedHerder.connectorStatus(any())).thenReturn(runningState); + lenient().doAnswer(invocation -> { + Callback> callback = invocation.getArgument(1); + // Simulate a successful completion + callback.onCompletion(null, null); + return null; + }).when(distributedHerder).connectorConfig(any(), any(Callback.class)); + lenient().doAnswer(invocation -> { + Callback> callback = invocation.getArgument(3); + // Simulate a successful completion + callback.onCompletion(null, null); + return null; + }).when(distributedHerder).putConnectorConfig(any(String.class), any(Map.class), any(Boolean.class), any(Callback.class)); + lenient().doAnswer(invocation -> { + Callback> callback = invocation.getArgument(1); + // Simulate a successful completion + callback.onCompletion(null, null); + return null; + }).when(distributedHerder).deleteConnectorConfig(any(), any(Callback.class)); + } + + @Test + void testInitializeKafkaConnectWithSingletonForSamePipeline() { + final KafkaConnect kafkaConnect = KafkaConnect.getPipelineInstance(TEST_PIPELINE_NAME, pluginMetrics, TEST_CONNECT_TIMEOUT, TEST_CONNECTOR_TIMEOUT); + final KafkaConnect sameConnect = KafkaConnect.getPipelineInstance(TEST_PIPELINE_NAME, pluginMetrics, TEST_CONNECT_TIMEOUT, TEST_CONNECTOR_TIMEOUT); + assertThat(sameConnect, is(kafkaConnect)); + final String anotherPipeline = "anotherPipeline"; + final KafkaConnect anotherKafkaConnect = KafkaConnect.getPipelineInstance(anotherPipeline, pluginMetrics, TEST_CONNECT_TIMEOUT, TEST_CONNECTOR_TIMEOUT); + assertThat(anotherKafkaConnect, not(kafkaConnect)); + } + + @Test + void testInitializeKafkaConnect() { + Map workerProps = DEFAULT_WORDER_PROPERTY.buildKafkaConnectPropertyMap(); + try (MockedConstruction mockedConfig = mockConstruction(DistributedConfig.class, (mock, context) -> { + when(mock.kafkaClusterId()).thenReturn("test-cluster-id"); + when(mock.getString(any())).thenReturn("test-string"); + }); + MockedConstruction mockedRestClient = mockConstruction(RestClient.class); + MockedConstruction mockedHerder = mockConstruction(DistributedHerder.class); + MockedConstruction mockedRestServer = mockConstruction(ConnectRestServer.class, (mock, context) -> { + when(mock.advertisedUrl()).thenReturn(URI.create("localhost:9002")); + }); + MockedConstruction mockedPlugin = mockConstruction(Plugins.class, (mock, context) -> { + ClassLoader classLoader = mock(ClassLoader.class); + ConnectorClientConfigOverridePolicy connectorPolicy = mock(ConnectorClientConfigOverridePolicy.class); + when(mock.compareAndSwapWithDelegatingLoader()).thenReturn(classLoader); + when(mock.newPlugin(any(), any(), any())).thenReturn(connectorPolicy); + }); + MockedConstruction mockedWorker = mockConstruction(Worker.class, (mock, context) -> { + WorkerConfigTransformer configTransformer = mock(WorkerConfigTransformer.class); + Converter converter = mock(Converter.class); + when(mock.configTransformer()).thenReturn(configTransformer); + when(mock.getInternalValueConverter()).thenReturn(converter); + }); + MockedConstruction mockedOffsetStore = mockConstruction(KafkaOffsetBackingStore.class, (mock, context) -> { + doNothing().when(mock).configure(any()); + }) + ) { + final KafkaConnect kafkaConnect = KafkaConnect.getPipelineInstance(TEST_PIPELINE_NAME, pluginMetrics, TEST_CONNECT_TIMEOUT, TEST_CONNECTOR_TIMEOUT); + kafkaConnect.initialize(workerProps); + } + } + + @Test + void testStartKafkaConnectSuccess() { + final KafkaConnect kafkaConnect = new KafkaConnect(distributedHerder, rest, connect, kafkaConnectMetrics); + doNothing().when(rest).initializeServer(); + doNothing().when(connect).start(); + kafkaConnect.start(); + verify(rest).initializeServer(); + verify(connect).start(); + } + + @Test + void testStartKafkaConnectFail() { + final KafkaConnect kafkaConnect = new KafkaConnect(distributedHerder, rest, connect, kafkaConnectMetrics); + doNothing().when(rest).initializeServer(); + doThrow(new RuntimeException()).when(connect).start(); + doNothing().when(connect).stop(); + assertThrows(RuntimeException.class, kafkaConnect::start); + verify(connect, times(1)).stop(); + + // throw exception immediately if connect is null + final KafkaConnect kafkaConnect2 = new KafkaConnect(distributedHerder, rest, null, kafkaConnectMetrics); + assertThrows(RuntimeException.class, kafkaConnect2::start); + } + + @Test + void testStartKafkaConnectFailTimeout() { + doNothing().when(rest).initializeServer(); + doNothing().when(connect).start(); + doNothing().when(connect).stop(); + when(connect.isRunning()).thenReturn(false); + try (MockedStatic mockedStatic = mockStatic(Clock.class)) { + final Clock clock = mock(Clock.class); + mockedStatic.when(() -> Clock.systemUTC()).thenReturn(clock); + when(clock.millis()).thenReturn(0L).thenReturn(TEST_CONNECT_TIMEOUT_MS + 1); + final KafkaConnect kafkaConnect = new KafkaConnect(distributedHerder, rest, connect, kafkaConnectMetrics); + assertThrows(RuntimeException.class, kafkaConnect::start); + verify(rest).initializeServer(); + verify(connect).start(); + verify(connect).stop(); + verify(clock, times(2)).millis(); + } + } + + @Test + void testStartKafkaConnectWithConnectRunningAlready() { + final KafkaConnect kafkaConnect = new KafkaConnect(distributedHerder, rest, connect, kafkaConnectMetrics); + when(connect.isRunning()).thenReturn(true); + kafkaConnect.start(); + verify(rest, never()).initializeServer(); + verify(connect, never()).start(); + } + + @Test + void testStopKafkaConnect() { + final KafkaConnect kafkaConnect = new KafkaConnect(distributedHerder, rest, connect, kafkaConnectMetrics); + kafkaConnect.stop(); + verify(connect).stop(); + // should ignore stop if connect is null + final KafkaConnect kafkaConnect2 = new KafkaConnect(distributedHerder, rest, null, kafkaConnectMetrics); + kafkaConnect2.stop(); + } + + @Test + void testInitConnectorsWhenStartKafkaConnectSuccess() { + final String oldConnectorName = "oldConnector"; + final Connector newConnector = mock(Connector.class); + final String newConnectorName = "newConnector"; + final Map newConnectorConfig = new HashMap<>(); + when(newConnector.getName()).thenReturn(newConnectorName); + when(newConnector.getConfig()).thenReturn(newConnectorConfig); + when(distributedHerder.connectors()).thenReturn(List.of(oldConnectorName)); + + final KafkaConnect kafkaConnect = new KafkaConnect(distributedHerder, rest, connect, kafkaConnectMetrics); + kafkaConnect.addConnectors(List.of(newConnector)); + kafkaConnect.start(); + verify(distributedHerder).putConnectorConfig(eq(newConnectorName), eq(newConnectorConfig), eq(true), any(Callback.class)); + verify(distributedHerder).deleteConnectorConfig(eq(oldConnectorName), any(Callback.class)); + } + + @Test + void testInitConnectorsWithoutConnectorConfigChange() { + final Connector newConnector = mock(Connector.class); + final String newConnectorName = "newConnector"; + final Map newConnectorConfig = new HashMap<>(); + when(newConnector.getName()).thenReturn(newConnectorName); + when(newConnector.getConfig()).thenReturn(newConnectorConfig); + when(newConnector.getAllowReplace()).thenReturn(false); + doAnswer(invocation -> { + Callback> callback = invocation.getArgument(1); + // Simulate a successful completion + callback.onCompletion(null, newConnectorConfig); + return null; + }).when(distributedHerder).connectorConfig(any(), any(Callback.class)); + + final KafkaConnect kafkaConnect = new KafkaConnect(distributedHerder, rest, connect, kafkaConnectMetrics); + kafkaConnect.addConnectors(List.of(newConnector)); + kafkaConnect.start(); + verify(distributedHerder).putConnectorConfig(eq(newConnectorName), eq(newConnectorConfig), eq(false), any(Callback.class)); + } + + @Test + void testInitConnectorsErrorsWhenDeleteConnector() { + final String oldConnectorName = "oldConnector"; + when(distributedHerder.connectors()).thenReturn(List.of(oldConnectorName)); + final KafkaConnect kafkaConnect = new KafkaConnect(distributedHerder, rest, connect, kafkaConnectMetrics); + doAnswer(invocation -> { + Callback> callback = invocation.getArgument(1); + // Simulate a successful completion + callback.onCompletion(new RuntimeException(), null); + return null; + }).when(distributedHerder).deleteConnectorConfig(eq(oldConnectorName), any(Callback.class)); + assertThrows(RuntimeException.class, kafkaConnect::start); + // NotLeaderException or NotFoundException should be ignored. + doAnswer(invocation -> { + Callback> callback = invocation.getArgument(1); + callback.onCompletion(new NotLeaderException("Only Leader can delete.", "leaderUrl"), null); + return null; + }).when(distributedHerder).deleteConnectorConfig(eq(oldConnectorName), any(Callback.class)); + kafkaConnect.start(); + doAnswer(invocation -> { + Callback> callback = invocation.getArgument(1); + // Simulate a successful completion + callback.onCompletion(new NotFoundException("Not Found"), null); + return null; + }).when(distributedHerder).deleteConnectorConfig(eq(oldConnectorName), any(Callback.class)); + kafkaConnect.start(); + } + + @Test + void testInitConnectorsErrorsWhenPutConnector() { + final Connector newConnector = mock(Connector.class); + final String newConnectorName = "newConnector"; + final Map newConnectorConfig = new HashMap<>(); + when(newConnector.getName()).thenReturn(newConnectorName); + when(newConnector.getConfig()).thenReturn(newConnectorConfig); + final KafkaConnect kafkaConnect = new KafkaConnect(distributedHerder, rest, connect, kafkaConnectMetrics); + kafkaConnect.addConnectors(List.of(newConnector)); + // RuntimeException should be thrown + doAnswer(invocation -> { + Callback> callback = invocation.getArgument(3); + callback.onCompletion(new RuntimeException(), null); + return null; + }).when(distributedHerder).putConnectorConfig(eq(newConnectorName), eq(newConnectorConfig), eq(true), any(Callback.class)); + assertThrows(RuntimeException.class, kafkaConnect::start); + // NotLeaderException or NotFoundException should be ignored. + doAnswer(invocation -> { + Callback> callback = invocation.getArgument(3); + callback.onCompletion(new NotLeaderException("not leader", "leaderUrl"), null); + return null; + }).when(distributedHerder).putConnectorConfig(eq(newConnectorName), eq(newConnectorConfig), eq(true), any(Callback.class)); + kafkaConnect.start(); + doAnswer(invocation -> { + Callback> callback = invocation.getArgument(3); + callback.onCompletion(new AlreadyExistsException("Already added"), null); + return null; + }).when(distributedHerder).putConnectorConfig(eq(newConnectorName), eq(newConnectorConfig), eq(true), any(Callback.class)); + kafkaConnect.start(); + } + + @Test + void testInitConnectorsErrorsWhenConnectorsNotRunning() { + // should throw exception if connector failed in Running state for 30 seconds + final Connector newConnector = mock(Connector.class); + final String newConnectorName = "newConnector"; + final Map newConnectorConfig = new HashMap<>(); + when(newConnector.getName()).thenReturn(newConnectorName); + when(newConnector.getConfig()).thenReturn(newConnectorConfig); + when(distributedHerder.connectorStatus(eq(newConnectorName))).thenReturn(null); + + try (MockedStatic mockedStatic = mockStatic(Clock.class)) { + final Clock clock = mock(Clock.class); + mockedStatic.when(() -> Clock.systemUTC()).thenReturn(clock); + when(clock.millis()).thenReturn(0L).thenReturn(0L).thenReturn(0L).thenReturn(0L).thenReturn(TEST_CONNECTOR_TIMEOUT_MS + 1); + final KafkaConnect kafkaConnect = new KafkaConnect(distributedHerder, rest, connect, kafkaConnectMetrics); + kafkaConnect.addConnectors(List.of(newConnector)); + assertThrows(RuntimeException.class, kafkaConnect::start); + verify(distributedHerder, times(1)).connectorStatus(any()); + verify(distributedHerder).putConnectorConfig(eq(newConnectorName), eq(newConnectorConfig), eq(true), any(Callback.class)); + verify(clock, times(5)).millis(); + } + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/util/SecretManagerHelperTest.java b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/util/SecretManagerHelperTest.java new file mode 100644 index 0000000000..207d651108 --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafkaconnect/util/SecretManagerHelperTest.java @@ -0,0 +1,85 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafkaconnect.util; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.mockito.MockedStatic; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.services.secretsmanager.SecretsManagerClient; +import software.amazon.awssdk.services.secretsmanager.SecretsManagerClientBuilder; +import software.amazon.awssdk.services.secretsmanager.model.GetSecretValueRequest; +import software.amazon.awssdk.services.secretsmanager.model.GetSecretValueResponse; +import software.amazon.awssdk.services.sts.StsClient; +import software.amazon.awssdk.services.sts.StsClientBuilder; +import software.amazon.awssdk.services.sts.auth.StsAssumeRoleCredentialsProvider; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class SecretManagerHelperTest { + private final String expectedSecretString = "expectedSecret"; + private final String testStsRole = "testRole"; + private final String testRegion = "testRegion"; + private final String testSecretId = "testSecritId"; + @Mock + private SecretsManagerClientBuilder secretsManagerClientBuilder; + @Mock + private SecretsManagerClient secretsManagerClient; + @Mock + private GetSecretValueResponse getSecretValueResponse; + + @BeforeEach + void setup() { + secretsManagerClientBuilder = mock(SecretsManagerClientBuilder.class); + secretsManagerClient = mock(SecretsManagerClient.class); + getSecretValueResponse = mock(GetSecretValueResponse.class); + lenient().when(secretsManagerClientBuilder.overrideConfiguration(any(ClientOverrideConfiguration.class))).thenReturn(secretsManagerClientBuilder); + lenient().when(secretsManagerClientBuilder.credentialsProvider(any(AwsCredentialsProvider.class))).thenReturn(secretsManagerClientBuilder); + lenient().when(secretsManagerClientBuilder.region(any())).thenReturn(secretsManagerClientBuilder); + lenient().when(secretsManagerClientBuilder.build()).thenReturn(secretsManagerClient); + lenient().when(secretsManagerClient.getSecretValue(any(GetSecretValueRequest.class))).thenReturn(getSecretValueResponse); + lenient().when(getSecretValueResponse.secretString()).thenReturn(expectedSecretString); + } + + @Test + void test_get_secret_without_sts() { + try (MockedStatic mockedStatic = mockStatic(SecretsManagerClient.class)) { + mockedStatic.when(() -> SecretsManagerClient.builder()).thenReturn(secretsManagerClientBuilder); + String result = SecretManagerHelper.getSecretValue("", testRegion, testSecretId); + assertThat(result, is(expectedSecretString)); + verify(secretsManagerClientBuilder, times(1)).credentialsProvider(any(AwsCredentialsProvider.class)); + } + } + + @Test + void test_get_secret_with_sts() { + try (MockedStatic mockedSts = mockStatic(StsClient.class); + MockedStatic mockedStatic = mockStatic(SecretsManagerClient.class)) { + StsClient stsClient = mock(StsClient.class); + StsClientBuilder stsClientBuilder = mock(StsClientBuilder.class); + when(stsClientBuilder.overrideConfiguration(any(ClientOverrideConfiguration.class))).thenReturn(stsClientBuilder); + when(stsClientBuilder.credentialsProvider(any(AwsCredentialsProvider.class))).thenReturn(stsClientBuilder); + when(stsClientBuilder.region(any())).thenReturn(stsClientBuilder); + when(stsClientBuilder.build()).thenReturn(stsClient); + + mockedSts.when(() -> StsClient.builder()).thenReturn(stsClientBuilder); + mockedStatic.when(() -> SecretsManagerClient.builder()).thenReturn(secretsManagerClientBuilder); + String result = SecretManagerHelper.getSecretValue(testStsRole, testRegion, testSecretId); + assertThat(result, is(expectedSecretString)); + verify(secretsManagerClientBuilder, times(1)).credentialsProvider(any(StsAssumeRoleCredentialsProvider.class)); + } + } +} diff --git a/data-prepper-plugins/kafka-connect-plugins/src/test/resources/sample-data-prepper-config-with-default-kafka-connect-config-extension.yaml b/data-prepper-plugins/kafka-connect-plugins/src/test/resources/sample-data-prepper-config-with-default-kafka-connect-config-extension.yaml new file mode 100644 index 0000000000..753d4f9f5d --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/test/resources/sample-data-prepper-config-with-default-kafka-connect-config-extension.yaml @@ -0,0 +1,8 @@ +extensions: + kafka_connect_config: + worker_properties: + group_id: test-group + client_id: test-client + config_storage_topic: test-configs + offset_storage_topic: test-offsets + status_storage_topic: test-status \ No newline at end of file diff --git a/data-prepper-plugins/kafka-connect-plugins/src/test/resources/sample-data-prepper-config-with-kafka-connect-config-extension.yaml b/data-prepper-plugins/kafka-connect-plugins/src/test/resources/sample-data-prepper-config-with-kafka-connect-config-extension.yaml new file mode 100644 index 0000000000..e41c7d04fe --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/test/resources/sample-data-prepper-config-with-kafka-connect-config-extension.yaml @@ -0,0 +1,19 @@ +extensions: + kafka_connect_config: + bootstrap_servers: + - test:123 + connect_start_timeout: 3000ms + connector_start_timeout: 3s + worker_properties: + group_id: test-group + client_id: test-client + config_storage_topic: test-configs + offset_storage_topic: test-offsets + status_storage_topic: test-status + offset_storage_partitions: 2 #optional and default is 25 + offset_flush_interval: 6s #optional and default is 60000 (60s) + offset_flush_timeout: 500ms #optional and default is 5000 (5s) + status_storage_partitions: 1 #optional and default is 5 + heartbeat_interval: 300ms #optional and default is 3000 (3s) + session_timeout: 3s #optional and default is 30000 (30s) + connector_rebalance_max_delay: 60s \ No newline at end of file diff --git a/data-prepper-plugins/kafka-connect-plugins/src/test/resources/sample-mongodb-pipeline.yaml b/data-prepper-plugins/kafka-connect-plugins/src/test/resources/sample-mongodb-pipeline.yaml new file mode 100644 index 0000000000..ef4a9cc829 --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/test/resources/sample-mongodb-pipeline.yaml @@ -0,0 +1,13 @@ +log-pipeline: + source: + mongodb: + hostname: localhost + credentials: + plaintext: + username: debezium + password: dbz + collections: + - topic_prefix: prefix1 + collection: test.customers + sink: + - noop: \ No newline at end of file diff --git a/data-prepper-plugins/kafka-connect-plugins/src/test/resources/sample-mysql-pipeline.yaml b/data-prepper-plugins/kafka-connect-plugins/src/test/resources/sample-mysql-pipeline.yaml new file mode 100644 index 0000000000..bdbaeff015 --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/test/resources/sample-mysql-pipeline.yaml @@ -0,0 +1,13 @@ +log-pipeline: + source: + mysql: + hostname: localhost + credentials: + plaintext: + username: debezium + password: dbz + tables: + - topic_prefix: prefix1 + table: inventory.customers + sink: + - noop: \ No newline at end of file diff --git a/data-prepper-plugins/kafka-connect-plugins/src/test/resources/sample-postgres-pipeline.yaml b/data-prepper-plugins/kafka-connect-plugins/src/test/resources/sample-postgres-pipeline.yaml new file mode 100644 index 0000000000..c843e3ccff --- /dev/null +++ b/data-prepper-plugins/kafka-connect-plugins/src/test/resources/sample-postgres-pipeline.yaml @@ -0,0 +1,14 @@ +log-pipeline: + source: + postgresql: + hostname: localhost + credentials: + plaintext: + username: debezium + password: dbz + tables: + - topic_prefix: psql + database: postgres + table: public.customers + sink: + - noop: \ No newline at end of file diff --git a/data-prepper-plugins/kafka-plugins/build.gradle b/data-prepper-plugins/kafka-plugins/build.gradle index 1a9a79e9b5..3d9e815870 100644 --- a/data-prepper-plugins/kafka-plugins/build.gradle +++ b/data-prepper-plugins/kafka-plugins/build.gradle @@ -17,11 +17,13 @@ dependencies { implementation 'com.fasterxml.jackson.core:jackson-databind' implementation 'io.micrometer:micrometer-core' implementation libs.commons.lang3 - implementation 'io.confluent:kafka-avro-serializer:7.3.3' - implementation 'io.confluent:kafka-schema-registry-client:7.3.3' - implementation 'io.confluent:kafka-avro-serializer:7.3.3' - implementation 'io.confluent:kafka-schema-registry-client:7.3.3' - implementation 'io.confluent:kafka-schema-registry:7.3.3:tests' + implementation 'io.confluent:kafka-avro-serializer:7.4.0' + implementation 'io.confluent:kafka-schema-registry-client:7.4.0' + implementation ('io.confluent:kafka-schema-registry:7.4.0:tests') { + exclude group: 'org.glassfish.jersey.containers', module: 'jersey-container-servlet' + exclude group: 'org.glassfish.jersey.inject', module: 'jersey-hk2' + exclude group: 'org.glassfish.jersey.ext', module: 'jersey-bean-validation' + } implementation 'software.amazon.awssdk:sts' implementation 'software.amazon.awssdk:auth' implementation 'software.amazon.awssdk:kafka' diff --git a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceJsonTypeIT.java b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceJsonTypeIT.java index ec82ecebf0..491caf06fd 100644 --- a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceJsonTypeIT.java +++ b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceJsonTypeIT.java @@ -30,6 +30,7 @@ import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaKeyMode; import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; +import org.opensearch.dataprepper.plugins.kafka.extension.KafkaClusterConfigSupplier; import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -77,6 +78,9 @@ public class KafkaSourceJsonTypeIT { @Mock private EncryptionConfig encryptionConfig; + @Mock + private KafkaClusterConfigSupplier kafkaClusterConfigSupplier; + @Mock private TopicConfig jsonTopic; @@ -92,7 +96,7 @@ public class KafkaSourceJsonTypeIT { private String testGroup; public KafkaSource createObjectUnderTest() { - return new KafkaSource(sourceConfig, pluginMetrics, acknowledgementSetManager, pipelineDescription); + return new KafkaSource(sourceConfig, pluginMetrics, acknowledgementSetManager, pipelineDescription, kafkaClusterConfigSupplier); } @BeforeEach diff --git a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceMultipleAuthTypeIT.java b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceMultipleAuthTypeIT.java index 0556c13cf4..693c7dc8af 100644 --- a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceMultipleAuthTypeIT.java +++ b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceMultipleAuthTypeIT.java @@ -28,6 +28,7 @@ import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.PlainTextAuthConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; +import org.opensearch.dataprepper.plugins.kafka.extension.KafkaClusterConfigSupplier; import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat; import java.time.Duration; @@ -80,6 +81,9 @@ public class KafkaSourceMultipleAuthTypeIT { @Mock private EncryptionConfig encryptionConfig; + @Mock + private KafkaClusterConfigSupplier kafkaClusterConfigSupplier; + private TopicConfig jsonTopic; private TopicConfig avroTopic; @@ -97,7 +101,7 @@ public class KafkaSourceMultipleAuthTypeIT { private String kafkaPassword; public KafkaSource createObjectUnderTest() { - return new KafkaSource(sourceConfig, pluginMetrics, acknowledgementSetManager, pipelineDescription); + return new KafkaSource(sourceConfig, pluginMetrics, acknowledgementSetManager, pipelineDescription, kafkaClusterConfigSupplier); } @BeforeEach diff --git a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/MskGlueRegistryMultiTypeIT.java b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/MskGlueRegistryMultiTypeIT.java index 19d2e755e2..4091a66966 100644 --- a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/MskGlueRegistryMultiTypeIT.java +++ b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/MskGlueRegistryMultiTypeIT.java @@ -41,6 +41,7 @@ import org.opensearch.dataprepper.plugins.kafka.configuration.SchemaConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.SchemaRegistryType; import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; +import org.opensearch.dataprepper.plugins.kafka.extension.KafkaClusterConfigSupplier; import java.io.File; import java.io.IOException; @@ -100,6 +101,9 @@ public class MskGlueRegistryMultiTypeIT { @Mock private EncryptionConfig encryptionConfig; + @Mock + private KafkaClusterConfigSupplier kafkaClusterConfigSupplier; + private KafkaSource kafkaSource; private TopicConfig jsonTopic; private TopicConfig avroTopic; @@ -125,7 +129,7 @@ public class MskGlueRegistryMultiTypeIT { public KafkaSource createObjectUnderTest() { - return new KafkaSource(sourceConfig, pluginMetrics, acknowledgementSetManager, pipelineDescription); + return new KafkaSource(sourceConfig, pluginMetrics, acknowledgementSetManager, pipelineDescription, kafkaClusterConfigSupplier); } @BeforeEach diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/EncryptionConfig.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/EncryptionConfig.java index a6642e5afe..826c2edadf 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/EncryptionConfig.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/EncryptionConfig.java @@ -1,3 +1,8 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + package org.opensearch.dataprepper.plugins.kafka.configuration; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaConnectionConfig.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaConnectionConfig.java index 09d6cf3376..cbd0848fac 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaConnectionConfig.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaConnectionConfig.java @@ -1,8 +1,10 @@ package org.opensearch.dataprepper.plugins.kafka.configuration; +import org.opensearch.dataprepper.plugins.kafka.util.KafkaClusterAuthConfig; + import java.util.Collection; -public interface KafkaConnectionConfig { +public interface KafkaConnectionConfig extends KafkaClusterAuthConfig { Collection getBootstrapServers(); AuthConfig getAuthConfig(); AwsConfig getAwsConfig(); diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaSourceConfig.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaSourceConfig.java index 060878fb17..ba95e75723 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaSourceConfig.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaSourceConfig.java @@ -88,6 +88,10 @@ public AuthConfig getAuthConfig() { return authConfig; } + public void setAuthConfig(AuthConfig authConfig) { + this.authConfig = authConfig; + } + public EncryptionConfig getEncryptionConfig() { if (Objects.isNull(encryptionConfig)) { return new EncryptionConfig(); @@ -95,11 +99,19 @@ public EncryptionConfig getEncryptionConfig() { return encryptionConfig; } + public void setEncryptionConfig(EncryptionConfig encryptionConfig) { + this.encryptionConfig = encryptionConfig; + } + + public EncryptionConfig getEncryptionConfigRaw() { + return encryptionConfig; + } + public AwsConfig getAwsConfig() { return awsConfig; } - public void setAuthConfig(AuthConfig authConfig) { - this.authConfig = authConfig; + public void setAwsConfig(AwsConfig awsConfig) { + this.awsConfig = awsConfig; } } diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/extension/DefaultKafkaClusterConfigSupplier.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/extension/DefaultKafkaClusterConfigSupplier.java new file mode 100644 index 0000000000..4378dac1ef --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/extension/DefaultKafkaClusterConfigSupplier.java @@ -0,0 +1,39 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.extension; + +import org.opensearch.dataprepper.plugins.kafka.configuration.AuthConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.AwsConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.EncryptionConfig; + +import java.util.List; + +public class DefaultKafkaClusterConfigSupplier implements KafkaClusterConfigSupplier { + private final KafkaClusterConfig kafkaClusterConfig; + public DefaultKafkaClusterConfigSupplier(KafkaClusterConfig kafkaClusterConfig) { + this.kafkaClusterConfig = kafkaClusterConfig; + } + + @Override + public List getBootStrapServers() { + return kafkaClusterConfig.getBootStrapServers(); + } + + @Override + public AuthConfig getAuthConfig() { + return kafkaClusterConfig.getAuthConfig(); + } + + @Override + public AwsConfig getAwsConfig() { + return kafkaClusterConfig.getAwsConfig(); + } + + @Override + public EncryptionConfig getEncryptionConfig() { + return kafkaClusterConfig.getEncryptionConfig(); + } +} diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/extension/KafkaClusterConfig.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/extension/KafkaClusterConfig.java new file mode 100644 index 0000000000..567ee7a631 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/extension/KafkaClusterConfig.java @@ -0,0 +1,46 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.extension; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.Valid; +import org.opensearch.dataprepper.plugins.kafka.configuration.AuthConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.AwsConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.EncryptionConfig; + +import java.util.List; + +public class KafkaClusterConfig { + @JsonProperty("bootstrap_servers") + private List bootstrapServers; + + @Valid + @JsonProperty("authentication") + private AuthConfig authConfig; + + @JsonProperty("encryption") + private EncryptionConfig encryptionConfig; + + @JsonProperty("aws") + @Valid + private AwsConfig awsConfig; + + public List getBootStrapServers() { + return bootstrapServers; + } + + public AuthConfig getAuthConfig() { + return authConfig; + } + + public AwsConfig getAwsConfig() { + return awsConfig; + } + + public EncryptionConfig getEncryptionConfig() { + return encryptionConfig; + } +} diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/extension/KafkaClusterConfigExtension.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/extension/KafkaClusterConfigExtension.java new file mode 100644 index 0000000000..2635a645df --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/extension/KafkaClusterConfigExtension.java @@ -0,0 +1,29 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.extension; + +import org.opensearch.dataprepper.model.annotations.DataPrepperExtensionPlugin; +import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; +import org.opensearch.dataprepper.model.plugin.ExtensionPlugin; +import org.opensearch.dataprepper.model.plugin.ExtensionPoints; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@DataPrepperExtensionPlugin(modelType = KafkaClusterConfig.class, rootKeyJsonPath = "/kafka_cluster_config") +public class KafkaClusterConfigExtension implements ExtensionPlugin { + private static final Logger LOG = LoggerFactory.getLogger(KafkaClusterConfigExtension.class); + private DefaultKafkaClusterConfigSupplier defaultKafkaClusterConfigSupplier; + + @DataPrepperPluginConstructor + public KafkaClusterConfigExtension(final KafkaClusterConfig kafkaClusterConfig) { + this.defaultKafkaClusterConfigSupplier = new DefaultKafkaClusterConfigSupplier(kafkaClusterConfig); + } + @Override + public void apply(ExtensionPoints extensionPoints) { + LOG.info("Applying Kafka Cluster Config Extension."); + extensionPoints.addExtensionProvider(new KafkaClusterConfigProvider(this.defaultKafkaClusterConfigSupplier)); + } +} diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/extension/KafkaClusterConfigProvider.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/extension/KafkaClusterConfigProvider.java new file mode 100644 index 0000000000..1017b07139 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/extension/KafkaClusterConfigProvider.java @@ -0,0 +1,28 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.extension; + +import org.opensearch.dataprepper.model.plugin.ExtensionProvider; + +import java.util.Optional; + +public class KafkaClusterConfigProvider implements ExtensionProvider { + private final KafkaClusterConfigSupplier kafkaClusterConfigSupplier; + + public KafkaClusterConfigProvider(KafkaClusterConfigSupplier kafkaClusterConfigSupplier) { + this.kafkaClusterConfigSupplier = kafkaClusterConfigSupplier; + } + + @Override + public Optional provideInstance(Context context) { + return Optional.of(this.kafkaClusterConfigSupplier); + } + + @Override + public Class supportedClass() { + return KafkaClusterConfigSupplier.class; + } +} diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/extension/KafkaClusterConfigSupplier.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/extension/KafkaClusterConfigSupplier.java new file mode 100644 index 0000000000..2a73a500fa --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/extension/KafkaClusterConfigSupplier.java @@ -0,0 +1,26 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.extension; + +import org.opensearch.dataprepper.plugins.kafka.configuration.AuthConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.AwsConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.EncryptionConfig; + +import java.util.List; + +/** + * An interface available to plugins via the Kafka Cluster Config Plugin Extension which supplies + * bootstrap_servers, authentication,encryption and aws configs. + */ +public interface KafkaClusterConfigSupplier { + List getBootStrapServers(); + + AuthConfig getAuthConfig(); + + AwsConfig getAwsConfig(); + + EncryptionConfig getEncryptionConfig(); +} diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSource.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSource.java index b668a9a94d..39ef4ca2a5 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSource.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSource.java @@ -38,6 +38,7 @@ import org.opensearch.dataprepper.plugins.kafka.configuration.SchemaRegistryType; import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; import org.opensearch.dataprepper.plugins.kafka.consumer.KafkaCustomConsumer; +import org.opensearch.dataprepper.plugins.kafka.extension.KafkaClusterConfigSupplier; import org.opensearch.dataprepper.plugins.kafka.util.ClientDNSLookupType; import org.opensearch.dataprepper.plugins.kafka.util.KafkaSecurityConfigurer; import org.opensearch.dataprepper.plugins.kafka.util.KafkaTopicMetrics; @@ -90,7 +91,8 @@ public class KafkaSource implements Source> { public KafkaSource(final KafkaSourceConfig sourceConfig, final PluginMetrics pluginMetrics, final AcknowledgementSetManager acknowledgementSetManager, - final PipelineDescription pipelineDescription) { + final PipelineDescription pipelineDescription, + final KafkaClusterConfigSupplier kafkaClusterConfigSupplier) { this.sourceConfig = sourceConfig; this.pluginMetrics = pluginMetrics; this.acknowledgementSetManager = acknowledgementSetManager; @@ -99,6 +101,7 @@ public KafkaSource(final KafkaSourceConfig sourceConfig, this.shutdownInProgress = new AtomicBoolean(false); this.allTopicExecutorServices = new ArrayList<>(); this.allTopicConsumers = new ArrayList<>(); + this.updateConfig(kafkaClusterConfigSupplier); } @Override @@ -350,4 +353,21 @@ private void setPropertiesForSchemaRegistryConnectivity(Properties properties) { protected void sleep(final long millis) throws InterruptedException { Thread.sleep(millis); } + + private void updateConfig(final KafkaClusterConfigSupplier kafkaClusterConfigSupplier) { + if (kafkaClusterConfigSupplier != null) { + if (sourceConfig.getBootstrapServers() == null) { + sourceConfig.setBootStrapServers(kafkaClusterConfigSupplier.getBootStrapServers()); + } + if (sourceConfig.getAuthConfig() == null) { + sourceConfig.setAuthConfig(kafkaClusterConfigSupplier.getAuthConfig()); + } + if (sourceConfig.getAwsConfig() == null) { + sourceConfig.setAwsConfig(kafkaClusterConfigSupplier.getAwsConfig()); + } + if (sourceConfig.getEncryptionConfigRaw() == null) { + sourceConfig.setEncryptionConfig(kafkaClusterConfigSupplier.getEncryptionConfig()); + } + } + } } \ No newline at end of file diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaClusterAuthConfig.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaClusterAuthConfig.java new file mode 100644 index 0000000000..2965f34107 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaClusterAuthConfig.java @@ -0,0 +1,22 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.util; + +import org.opensearch.dataprepper.plugins.kafka.configuration.AuthConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.AwsConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.EncryptionConfig; + +import java.util.Collection; + +public interface KafkaClusterAuthConfig { + AwsConfig getAwsConfig(); + + AuthConfig getAuthConfig(); + + EncryptionConfig getEncryptionConfig(); + + Collection getBootstrapServers(); +} diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaSecurityConfigurer.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaSecurityConfigurer.java index 4f338367bd..db01c919cf 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaSecurityConfigurer.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaSecurityConfigurer.java @@ -8,8 +8,8 @@ import org.opensearch.dataprepper.plugins.kafka.configuration.AwsConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.AwsIamAuthConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.EncryptionConfig; -import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaConnectionConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaConsumerConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.SchemaConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.OAuthConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.EncryptionType; @@ -120,9 +120,9 @@ private static void setPlainTextAuthProperties(Properties properties, final Plai } } - public static void setOauthProperties(final KafkaConnectionConfig kafkaConsumerConfig, + public static void setOauthProperties(final KafkaClusterAuthConfig kafkaClusterAuthConfig, final Properties properties) { - final OAuthConfig oAuthConfig = kafkaConsumerConfig.getAuthConfig().getSaslAuthConfig().getOAuthConfig(); + final OAuthConfig oAuthConfig = kafkaClusterAuthConfig.getAuthConfig().getSaslAuthConfig().getOAuthConfig(); final String oauthClientId = oAuthConfig.getOauthClientId(); final String oauthClientSecret = oAuthConfig.getOauthClientSecret(); final String oauthLoginServer = oAuthConfig.getOauthLoginServer(); @@ -157,9 +157,11 @@ public static void setOauthProperties(final KafkaConnectionConfig kafkaConsumerC String jass_config = String.format(OAUTH_JAASCONFIG, oauthClientId, oauthClientSecret, oauthLoginScope, oauthLoginServer, oauthLoginEndpoint, oauthLoginGrantType, oauthLoginScope, oauthAuthorizationToken, instrospect_properties); - if ("USER_INFO".equalsIgnoreCase(kafkaConsumerConfig.getSchemaConfig().getBasicAuthCredentialsSource())) { - final String apiKey = kafkaConsumerConfig.getSchemaConfig().getSchemaRegistryApiKey(); - final String apiSecret = kafkaConsumerConfig.getSchemaConfig().getSchemaRegistryApiSecret(); + if (kafkaClusterAuthConfig instanceof KafkaSourceConfig && + "USER_INFO".equalsIgnoreCase(((KafkaSourceConfig) kafkaClusterAuthConfig).getSchemaConfig().getBasicAuthCredentialsSource())) { + final SchemaConfig schemaConfig = ((KafkaSourceConfig) kafkaClusterAuthConfig).getSchemaConfig(); + final String apiKey = schemaConfig.getSchemaRegistryApiKey(); + final String apiSecret = schemaConfig.getSchemaRegistryApiSecret(); final String extensionLogicalCluster = oAuthConfig.getExtensionLogicalCluster(); final String extensionIdentityPoolId = oAuthConfig.getExtensionIdentityPoolId(); properties.put(REGISTRY_BASIC_AUTH_USER_INFO, apiKey + ":" + apiSecret); @@ -256,17 +258,17 @@ public static String getBootStrapServersForMsk(final AwsIamAuthConfig awsIamAuth } } - public static void setAuthProperties(Properties properties, final KafkaConnectionConfig consumerConfig, final Logger LOG) { - final AwsConfig awsConfig = consumerConfig.getAwsConfig(); - final AuthConfig authConfig = consumerConfig.getAuthConfig(); - final EncryptionConfig encryptionConfig = consumerConfig.getEncryptionConfig(); + public static void setAuthProperties(Properties properties, final KafkaClusterAuthConfig kafkaClusterAuthConfig, final Logger LOG) { + final AwsConfig awsConfig = kafkaClusterAuthConfig.getAwsConfig(); + final AuthConfig authConfig = kafkaClusterAuthConfig.getAuthConfig(); + final EncryptionConfig encryptionConfig = kafkaClusterAuthConfig.getEncryptionConfig(); final EncryptionType encryptionType = encryptionConfig.getType(); credentialsProvider = DefaultCredentialsProvider.create(); String bootstrapServers = ""; - if (Objects.nonNull(consumerConfig.getBootstrapServers())) { - bootstrapServers = String.join(",", consumerConfig.getBootstrapServers()); + if (Objects.nonNull(kafkaClusterAuthConfig.getBootstrapServers())) { + bootstrapServers = String.join(",", kafkaClusterAuthConfig.getBootstrapServers()); } AwsIamAuthConfig awsIamAuthConfig = null; if (Objects.nonNull(authConfig)) { @@ -285,7 +287,7 @@ public static void setAuthProperties(Properties properties, final KafkaConnectio setAwsIamAuthProperties(properties, awsIamAuthConfig, awsConfig); bootstrapServers = getBootStrapServersForMsk(awsIamAuthConfig, awsConfig, LOG); } else if (Objects.nonNull(saslAuthConfig.getOAuthConfig())) { - setOauthProperties(consumerConfig, properties); + setOauthProperties(kafkaClusterAuthConfig, properties); } else if (Objects.nonNull(plainTextAuthConfig)) { setPlainTextAuthProperties(properties, plainTextAuthConfig, encryptionType); } else { diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/extension/DefaultKafkaClusterConfigSupplierTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/extension/DefaultKafkaClusterConfigSupplierTest.java new file mode 100644 index 0000000000..ba9ff9cbe4 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/extension/DefaultKafkaClusterConfigSupplierTest.java @@ -0,0 +1,48 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.extension; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.plugins.kafka.configuration.AuthConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.AwsConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.EncryptionConfig; + +import java.util.List; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +public class DefaultKafkaClusterConfigSupplierTest { + @Mock + private KafkaClusterConfig kafkaClusterConfig; + + private DefaultKafkaClusterConfigSupplier createObjectUnderTest() { + return new DefaultKafkaClusterConfigSupplier(kafkaClusterConfig); + } + + @Test + void test_getters() { + final List bootstrapServers = List.of("localhost:9092"); + final AuthConfig authConfig = mock(AuthConfig.class); + final AwsConfig awsConfig = mock(AwsConfig.class); + final EncryptionConfig encryptionConfig = mock(EncryptionConfig.class); + when(kafkaClusterConfig.getBootStrapServers()).thenReturn(bootstrapServers); + when(kafkaClusterConfig.getAuthConfig()).thenReturn(authConfig); + when(kafkaClusterConfig.getAwsConfig()).thenReturn(awsConfig); + when(kafkaClusterConfig.getEncryptionConfig()).thenReturn(encryptionConfig); + DefaultKafkaClusterConfigSupplier defaultKafkaClusterConfigSupplier = createObjectUnderTest(); + assertThat(defaultKafkaClusterConfigSupplier.getBootStrapServers(), equalTo(bootstrapServers)); + assertThat(defaultKafkaClusterConfigSupplier.getAuthConfig(), equalTo(authConfig)); + assertThat(defaultKafkaClusterConfigSupplier.getAwsConfig(), equalTo(awsConfig)); + assertThat(defaultKafkaClusterConfigSupplier.getEncryptionConfig(), equalTo(encryptionConfig)); + } +} diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/extension/KafkaClusterConfigExtensionTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/extension/KafkaClusterConfigExtensionTest.java new file mode 100644 index 0000000000..9b921c5d1c --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/extension/KafkaClusterConfigExtensionTest.java @@ -0,0 +1,44 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.extension; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.plugin.ExtensionPoints; +import org.opensearch.dataprepper.model.plugin.ExtensionProvider; + +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.Mockito.verify; + +@ExtendWith(MockitoExtension.class) +public class KafkaClusterConfigExtensionTest { + @Mock + private ExtensionPoints extensionPoints; + + @Mock + private KafkaClusterConfig kafkaClusterConfig; + + private KafkaClusterConfigExtension createObjectUnderTest() { + return new KafkaClusterConfigExtension(kafkaClusterConfig); + } + + @Test + void apply_should_addExtensionProvider() { + createObjectUnderTest().apply(extensionPoints); + final ArgumentCaptor extensionProviderArgumentCaptor = + ArgumentCaptor.forClass(ExtensionProvider.class); + + verify(extensionPoints).addExtensionProvider(extensionProviderArgumentCaptor.capture()); + + final ExtensionProvider actualExtensionProvider = extensionProviderArgumentCaptor.getValue(); + + assertThat(actualExtensionProvider, instanceOf(KafkaClusterConfigProvider.class)); + } +} diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/extension/KafkaClusterConfigProviderTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/extension/KafkaClusterConfigProviderTest.java new file mode 100644 index 0000000000..5d9c032bb8 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/extension/KafkaClusterConfigProviderTest.java @@ -0,0 +1,52 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.extension; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.plugin.ExtensionProvider; + +import java.util.Optional; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.CoreMatchers.sameInstance; +import static org.hamcrest.MatcherAssert.assertThat; + +@ExtendWith(MockitoExtension.class) +public class KafkaClusterConfigProviderTest { + @Mock + private KafkaClusterConfigSupplier kafkaClusterConfigSupplier; + + @Mock + private ExtensionProvider.Context context; + + private KafkaClusterConfigProvider createObjectUnderTest() { + return new KafkaClusterConfigProvider(kafkaClusterConfigSupplier); + } + + @Test + void supportedClass_returns_kafkaClusterConfigSupplier() { + assertThat(createObjectUnderTest().supportedClass(), equalTo(KafkaClusterConfigSupplier.class)); + } + + @Test + void provideInstance_returns_the_kafkaClusterConfigSupplier_from_the_constructor() { + final KafkaClusterConfigProvider objectUnderTest = createObjectUnderTest(); + + final Optional optionalKafkaClusterConfigSupplier = objectUnderTest.provideInstance(context); + assertThat(optionalKafkaClusterConfigSupplier, notNullValue()); + assertThat(optionalKafkaClusterConfigSupplier.isPresent(), equalTo(true)); + assertThat(optionalKafkaClusterConfigSupplier.get(), equalTo(kafkaClusterConfigSupplier)); + + final Optional anotherOptionalKafkaClusterConfigSupplier = objectUnderTest.provideInstance(context); + assertThat(anotherOptionalKafkaClusterConfigSupplier, notNullValue()); + assertThat(anotherOptionalKafkaClusterConfigSupplier.isPresent(), equalTo(true)); + assertThat(anotherOptionalKafkaClusterConfigSupplier.get(), sameInstance(optionalKafkaClusterConfigSupplier.get())); + } +} diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/extension/KafkaClusterConfigTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/extension/KafkaClusterConfigTest.java new file mode 100644 index 0000000000..58fc6a5d6f --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/extension/KafkaClusterConfigTest.java @@ -0,0 +1,60 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.extension; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.model.types.ByteCount; +import org.opensearch.dataprepper.parser.ByteCountDeserializer; +import org.opensearch.dataprepper.parser.DataPrepperDurationDeserializer; +import org.opensearch.dataprepper.parser.model.DataPrepperConfiguration; +import org.opensearch.dataprepper.plugins.kafka.configuration.AwsIamAuthConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.EncryptionType; +import org.opensearch.dataprepper.plugins.kafka.configuration.MskBrokerConnectionType; + +import java.io.File; +import java.io.IOException; +import java.io.Reader; +import java.io.StringReader; +import java.time.Duration; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.MatcherAssert.assertThat; + +public class KafkaClusterConfigTest { + private static SimpleModule simpleModule = new SimpleModule() + .addDeserializer(Duration.class, new DataPrepperDurationDeserializer()) + .addDeserializer(ByteCount.class, new ByteCountDeserializer()); + private static ObjectMapper OBJECT_MAPPER = new ObjectMapper(new YAMLFactory()).registerModule(simpleModule); + + private KafkaClusterConfig makeConfig(String filePath) throws IOException { + final File configurationFile = new File(filePath); + final DataPrepperConfiguration dataPrepperConfiguration = OBJECT_MAPPER.readValue(configurationFile, DataPrepperConfiguration.class); + assertThat(dataPrepperConfiguration, notNullValue()); + assertThat(dataPrepperConfiguration.getPipelineExtensions(), notNullValue()); + final Map kafkaClusterConfigMap = (Map) dataPrepperConfiguration.getPipelineExtensions().getExtensionMap().get("kafka_cluster_config"); + String json = OBJECT_MAPPER.writeValueAsString(kafkaClusterConfigMap); + Reader reader = new StringReader(json); + return OBJECT_MAPPER.readValue(reader, KafkaClusterConfig.class); + } + + @Test + void testConfigWithTestExtension() throws IOException { + final KafkaClusterConfig kafkaClusterConfig = makeConfig( + "src/test/resources/valid-data-prepper-config-with-kafka-cluster-extension.yaml"); + assertThat(kafkaClusterConfig.getBootStrapServers(), equalTo(List.of("localhost:9092"))); + assertThat(kafkaClusterConfig.getEncryptionConfig().getType(), equalTo(EncryptionType.NONE)); + assertThat(kafkaClusterConfig.getAwsConfig().getRegion(), equalTo("us-east-1")); + assertThat(kafkaClusterConfig.getAwsConfig().getAwsMskConfig().getArn(), equalTo("test-arn")); + assertThat(kafkaClusterConfig.getAwsConfig().getAwsMskConfig().getBrokerConnectionType(), equalTo(MskBrokerConnectionType.PUBLIC)); + assertThat(kafkaClusterConfig.getAuthConfig().getSaslAuthConfig().getAwsIamAuthConfig(), equalTo(AwsIamAuthConfig.DEFAULT)); + } +} diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceTest.java index bc79f330d3..d9c78ab0af 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceTest.java +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceTest.java @@ -12,10 +12,14 @@ import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.configuration.PipelineDescription; import org.opensearch.dataprepper.plugins.kafka.configuration.EncryptionConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.AuthConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.AwsConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.EncryptionConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.SchemaConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.EncryptionType; +import org.opensearch.dataprepper.plugins.kafka.extension.KafkaClusterConfigSupplier; import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; @@ -34,10 +38,13 @@ import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.Arrays; +import java.util.List; import java.util.Collections; import java.util.Objects; import java.time.Duration; @@ -71,10 +78,13 @@ class KafkaSourceTest { @Mock private Buffer> buffer; + @Mock + private KafkaClusterConfigSupplier kafkaClusterConfigSupplier; + private static final String TEST_GROUP_ID = "testGroupId"; public KafkaSource createObjectUnderTest() { - return new KafkaSource(sourceConfig, pluginMetrics, acknowledgementSetManager, pipelineDescription); + return new KafkaSource(sourceConfig, pluginMetrics, acknowledgementSetManager, pipelineDescription, kafkaClusterConfigSupplier); } @BeforeEach @@ -158,4 +168,50 @@ void test_kafkaSource_retry_consumer_create() throws InterruptedException { .createKafkaConsumer(any(), any()); kafkaSource.start(buffer); } + + @Test + void test_updateConfig_using_kafkaClusterConfigExtension() { + final List bootstrapServers = List.of("localhost:9092"); + final AuthConfig authConfig = mock(AuthConfig.class); + final AwsConfig awsConfig = mock(AwsConfig.class); + final EncryptionConfig encryptionConfig = mock(EncryptionConfig.class); + doNothing().when(sourceConfig).setBootStrapServers(bootstrapServers); + doNothing().when(sourceConfig).setAuthConfig(any()); + doNothing().when(sourceConfig).setAwsConfig(any()); + doNothing().when(sourceConfig).setEncryptionConfig(any()); + when(sourceConfig.getAuthConfig()).thenReturn(null); + when(sourceConfig.getAwsConfig()).thenReturn(null); + when(sourceConfig.getEncryptionConfigRaw()).thenReturn(null); + when(sourceConfig.getBootstrapServers()).thenReturn(null); + when(kafkaClusterConfigSupplier.getBootStrapServers()).thenReturn(bootstrapServers); + when(kafkaClusterConfigSupplier.getAuthConfig()).thenReturn(authConfig); + when(kafkaClusterConfigSupplier.getAwsConfig()).thenReturn(awsConfig); + when(kafkaClusterConfigSupplier.getEncryptionConfig()).thenReturn(encryptionConfig); + kafkaSource = createObjectUnderTest(); + verify(sourceConfig).setBootStrapServers(bootstrapServers); + verify(sourceConfig).setAuthConfig(authConfig); + verify(sourceConfig).setAwsConfig(awsConfig); + verify(sourceConfig).setEncryptionConfig(encryptionConfig); + } + + @Test + void test_updateConfig_not_using_kafkaClusterConfigExtension() { + final List bootstrapServers = List.of("localhost:9092"); + final AuthConfig authConfig = mock(AuthConfig.class); + final AwsConfig awsConfig = mock(AwsConfig.class); + final EncryptionConfig encryptionConfig = mock(EncryptionConfig.class); + doNothing().when(sourceConfig).setBootStrapServers(any()); + doNothing().when(sourceConfig).setAuthConfig(any()); + doNothing().when(sourceConfig).setAwsConfig(any()); + doNothing().when(sourceConfig).setEncryptionConfig(any()); + when(sourceConfig.getAuthConfig()).thenReturn(authConfig); + when(sourceConfig.getAwsConfig()).thenReturn(awsConfig); + when(sourceConfig.getEncryptionConfigRaw()).thenReturn(encryptionConfig); + when(sourceConfig.getBootstrapServers()).thenReturn(bootstrapServers); + kafkaSource = createObjectUnderTest(); + verify(sourceConfig, never()).setBootStrapServers(any()); + verify(sourceConfig, never()).setAuthConfig(any()); + verify(sourceConfig, never()).setAwsConfig(any()); + verify(sourceConfig, never()).setEncryptionConfig(any()); + } } diff --git a/data-prepper-plugins/kafka-plugins/src/test/resources/valid-data-prepper-config-with-kafka-cluster-extension.yaml b/data-prepper-plugins/kafka-plugins/src/test/resources/valid-data-prepper-config-with-kafka-cluster-extension.yaml new file mode 100644 index 0000000000..7cc137575d --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/test/resources/valid-data-prepper-config-with-kafka-cluster-extension.yaml @@ -0,0 +1,15 @@ +extensions: + kafka_cluster_config: + bootstrap_servers: + - localhost:9092 + encryption: + type: none + aws: + region: "us-east-1" + msk: + # Provide the MSK ARN. + broker_connection_type: public + arn: "test-arn" + authentication: + sasl: + aws_msk_iam: default \ No newline at end of file diff --git a/settings.gradle b/settings.gradle index 433c6827cb..5ce7405bbc 100644 --- a/settings.gradle +++ b/settings.gradle @@ -39,6 +39,8 @@ dependencyResolutionManagement { library('bouncycastle-bcpkix', 'org.bouncycastle', 'bcpkix-jdk18on').versionRef('bouncycastle') version('guava', '32.1.2-jre') library('guava-core', 'com.google.guava', 'guava').versionRef('guava') + version('reflections', '0.9.12') + library('reflections-core', 'org.reflections', 'reflections').versionRef('reflections') library('commons-lang3', 'org.apache.commons', 'commons-lang3').version('3.13.0') library('commons-io', 'commons-io', 'commons-io').version('2.13.0') library('commons-codec', 'commons-codec', 'commons-codec').version('1.16.0') @@ -129,6 +131,7 @@ include 'data-prepper-plugins:failures-common' include 'data-prepper-plugins:newline-codecs' include 'data-prepper-plugins:avro-codecs' include 'data-prepper-plugins:kafka-plugins' +include 'data-prepper-plugins:kafka-connect-plugins' include 'data-prepper-plugins:opensearch-source' include 'data-prepper-plugins:user-agent-processor' include 'data-prepper-plugins:in-memory-source-coordination-store'