From e252f1944829b785019bc20c38cfd8038b81e7ca Mon Sep 17 00:00:00 2001 From: rajeshLovesToCode Date: Wed, 26 Jul 2023 21:06:40 +0530 Subject: [PATCH 1/9] -Support for kafka-sink Signed-off-by: rajeshLovesToCode --- .../kafka-plugins/build.gradle | 7 + .../kafka/configuration/AuthConfig.java | 20 +- .../KafkaProducerProperties.java | 250 ++++++++++++++ .../kafka/configuration/KafkaSinkConfig.java | 140 ++++++++ .../kafka/configuration/PlainConfig.java | 26 ++ .../kafka/producer/KafkaSinkProducer.java | 171 ++++++++++ .../kafka/producer/ProducerWorker.java | 33 ++ .../plugins/kafka/sink/DLQSink.java | 94 ++++++ .../plugins/kafka/sink/KafkaSink.java | 160 +++++++++ .../AuthenticationPropertyConfigurer.java | 145 ++++++++ .../kafka/util/SinkPropertyConfigurer.java | 313 ++++++++++++++++++ .../configuration/KafkaSinkConfigTest.java | 106 ++++++ .../kafka/producer/KafkaSinkProducerTest.java | 148 +++++++++ .../kafka/producer/ProducerWorkerTest.java | 50 +++ .../plugins/kafka/sink/DLQSinkTest.java | 96 ++++++ .../plugins/kafka/sink/KafkasinkTest.java | 186 +++++++++++ .../AuthenticationPropertyConfigurerTest.java | 75 +++++ .../util/SinkPropertyConfigurerTest.java | 92 +++++ .../sample-pipelines-sink-oauth.yaml | 34 ++ .../resources/sample-pipelines-sink-ssl.yaml | 56 ++++ .../test/resources/sample-pipelines-sink.yaml | 57 ++++ 21 files changed, 2256 insertions(+), 3 deletions(-) create mode 100644 data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaProducerProperties.java create mode 100644 data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaSinkConfig.java create mode 100644 data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/PlainConfig.java create mode 100644 data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/producer/KafkaSinkProducer.java create mode 100644 data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/producer/ProducerWorker.java create mode 100644 data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/sink/DLQSink.java create mode 100644 data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkaSink.java create mode 100644 data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/AuthenticationPropertyConfigurer.java create mode 100644 data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/SinkPropertyConfigurer.java create mode 100644 data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaSinkConfigTest.java create mode 100644 data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/producer/KafkaSinkProducerTest.java create mode 100644 data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/producer/ProducerWorkerTest.java create mode 100644 data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/sink/DLQSinkTest.java create mode 100644 data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkasinkTest.java create mode 100644 data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/util/AuthenticationPropertyConfigurerTest.java create mode 100644 data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/util/SinkPropertyConfigurerTest.java create mode 100644 data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-sink-oauth.yaml create mode 100644 data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-sink-ssl.yaml create mode 100644 data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-sink.yaml diff --git a/data-prepper-plugins/kafka-plugins/build.gradle b/data-prepper-plugins/kafka-plugins/build.gradle index 899061b611..298f11fb86 100644 --- a/data-prepper-plugins/kafka-plugins/build.gradle +++ b/data-prepper-plugins/kafka-plugins/build.gradle @@ -43,6 +43,13 @@ dependencies { testImplementation 'org.apache.kafka:kafka-clients:3.4.0:test' testImplementation 'org.apache.kafka:connect-json:3.4.0' testImplementation('com.kjetland:mbknor-jackson-jsonschema_2.13:1.0.39') + implementation project(':data-prepper-plugins:failures-common') + testImplementation group: 'org.powermock', name: 'powermock-api-mockito2', version: '2.0.9' + implementation 'org.apache.kafka:connect-json:3.4.0' + implementation 'com.github.fge:json-schema-validator:2.2.14' + implementation 'commons-collections:commons-collections:3.2.2' + + } test { diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/AuthConfig.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/AuthConfig.java index 883e33f630..a3fe1163d8 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/AuthConfig.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/AuthConfig.java @@ -6,8 +6,8 @@ package org.opensearch.dataprepper.plugins.kafka.configuration; import com.fasterxml.jackson.annotation.JsonProperty; -import jakarta.validation.constraints.AssertTrue; import jakarta.validation.Valid; +import jakarta.validation.constraints.AssertTrue; import java.util.stream.Stream; @@ -27,6 +27,12 @@ public static class SaslAuthConfig { @JsonProperty("aws_iam") private AwsIamAuthConfig awsIamAuthConfig; + @JsonProperty("ssl_endpoint_identification_algorithm") + private String sslEndpointAlgorithm; + + @JsonProperty("plain_config") + private PlainConfig plain; + public AwsIamAuthConfig getAwsIamAuthConfig() { return awsIamAuthConfig; } @@ -39,14 +45,22 @@ public OAuthConfig getOAuthConfig() { return oAuthConfig; } + public PlainConfig getPlain() { + return plain; + } + public String getSslEndpointAlgorithm() { + return sslEndpointAlgorithm; + } + @AssertTrue(message = "Only one of AwsIam or oAuth or PlainText auth config must be specified") public boolean hasOnlyOneConfig() { - return Stream.of(awsIamAuthConfig, plainTextAuthConfig, oAuthConfig).filter(n -> n!=null).count() == 1; + return Stream.of(awsIamAuthConfig, plainTextAuthConfig, oAuthConfig, plain).filter(n -> n != null).count() == 1; } } - public static class SslAuthConfig { + + public static class SslAuthConfig { // TODO Add Support for SSL authentication types like // one-way or two-way authentication diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaProducerProperties.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaProducerProperties.java new file mode 100644 index 0000000000..6d624fcd84 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaProducerProperties.java @@ -0,0 +1,250 @@ +package org.opensearch.dataprepper.plugins.kafka.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.time.Duration; +import java.util.List; + +public class KafkaProducerProperties { + private static final String DEFAULT_BYTE_CAPACITY = "50mb"; + + @JsonProperty("buffer_memory") + private String bufferMemory = DEFAULT_BYTE_CAPACITY; + + @JsonProperty("compression_type") + private String compressionType; + + @JsonProperty("retries") + private int retries; + + @JsonProperty("batch_size") + private int batchSize; + + @JsonProperty("client_dns_lookup") + private String clientDnsLookup; + + @JsonProperty("client_id") + private String clientId; + + @JsonProperty("connections_max_idle") + private Duration connectionsMaxIdleMs; + + @JsonProperty("delivery_timeout") + private Duration deliveryTimeoutMs; + + @JsonProperty("linger_ms") + private Long lingerMs; + + @JsonProperty("max_block") + private Duration maxBlockMs; + + @JsonProperty("max_request_size") + private int maxRequestSize; + + @JsonProperty("partitioner_class") + private Class partitionerClass; + + @JsonProperty("partitioner_ignore_keys") + private Boolean partitionerIgnoreKeys; + + @JsonProperty("receive_buffer") + private String receiveBufferBytes=DEFAULT_BYTE_CAPACITY; + + @JsonProperty("request_timeout") + private Duration requestTimeoutMs; + + @JsonProperty("send_buffer") + private String sendBufferBytes=DEFAULT_BYTE_CAPACITY; + + @JsonProperty("socket_connection_setup_timeout_max") + private Duration socketConnectionSetupMaxTimeout; + + @JsonProperty("socket_connection_setup_timeout") + private Duration socketConnectionSetupTimeout; + + @JsonProperty("acks") + private String acks; + + @JsonProperty("enable_idempotence") + private Boolean enableIdempotence; + + @JsonProperty("interceptor_classes") + private List interceptorClasses; + + @JsonProperty("max_in_flight_requests_per_connection") + private int maxInFlightRequestsPerConnection; + + @JsonProperty("metadata_max_age") + private Duration metadataMaxAgeMs; + + @JsonProperty("metadata_max_idle") + private Duration metadataMaxIdleMs; + + @JsonProperty("metric_reporters") + private List metricReporters; + + @JsonProperty("metrics_num_samples") + private int metricsNumSamples; + + @JsonProperty("metrics_recording_level") + private String metricsRecordingLevel; + + @JsonProperty("metrics_sample_window") + private Duration metricsSampleWindowMs; + + @JsonProperty("partitioner_adaptive_partitioning_enable") + private boolean partitionerAdaptivePartitioningEnable; + + @JsonProperty("partitioner_availability_timeout") + private Duration partitionerAvailabilityTimeoutMs; + + @JsonProperty("reconnect_backoff_max") + private Duration reconnectBackoffMaxMs; + + @JsonProperty("reconnect_backoff") + private Duration reconnectBackoffMs; + + @JsonProperty("retry_backoff") + private Duration retryBackoffMs; + + + public String getCompressionType() { + return compressionType; + } + + public int getRetries() { + if (retries == 0) { + retries = 5; + } + return retries; + } + + public int getBatchSize() { + return batchSize; + } + + public String getClientDnsLookup() { + return clientDnsLookup; + } + + public String getClientId() { + return clientId; + } + + + public Long getLingerMs() { + return lingerMs; + } + + + public int getMaxRequestSize() { + return maxRequestSize; + } + + public Class getPartitionerClass() { + return partitionerClass; + } + + public Boolean getPartitionerIgnoreKeys() { + return partitionerIgnoreKeys; + } + + + public String getAcks() { + return acks; + } + + public Boolean getEnableIdempotence() { + return enableIdempotence; + } + + public List getInterceptorClasses() { + return interceptorClasses; + } + + public int getMaxInFlightRequestsPerConnection() { + return maxInFlightRequestsPerConnection; + } + + + public List getMetricReporters() { + return metricReporters; + } + + public int getMetricsNumSamples() { + return metricsNumSamples; + } + + public String getMetricsRecordingLevel() { + return metricsRecordingLevel; + } + + + public boolean isPartitionerAdaptivePartitioningEnable() { + return partitionerAdaptivePartitioningEnable; + } + + public String getBufferMemory() { + return bufferMemory; + } + + public Long getConnectionsMaxIdleMs() { + return connectionsMaxIdleMs.toMillis(); + } + + public Long getDeliveryTimeoutMs() { + return deliveryTimeoutMs.toMillis(); + } + + public Long getMaxBlockMs() { + return maxBlockMs.toMillis(); + } + + public String getReceiveBufferBytes() { + return receiveBufferBytes; + } + + public Long getRequestTimeoutMs() { + return requestTimeoutMs.toMillis(); + } + + public String getSendBufferBytes() { + return sendBufferBytes; + } + + public Long getSocketConnectionSetupMaxTimeout() { + return socketConnectionSetupMaxTimeout.toMillis(); + } + + public Long getSocketConnectionSetupTimeout() { + return socketConnectionSetupTimeout.toMillis(); + } + + public Long getMetadataMaxAgeMs() { + return metadataMaxAgeMs.toMillis(); + } + + public Long getMetadataMaxIdleMs() { + return metadataMaxIdleMs.toMillis(); + } + + public Long getMetricsSampleWindowMs() { + return metricsSampleWindowMs.toMillis(); + } + + public Long getPartitionerAvailabilityTimeoutMs() { + return partitionerAvailabilityTimeoutMs.toMillis(); + } + + public Long getReconnectBackoffMaxMs() { + return reconnectBackoffMaxMs.toMillis(); + } + + public Long getReconnectBackoffMs() { + return reconnectBackoffMs.toMillis(); + } + + public Long getRetryBackoffMs() { + return retryBackoffMs.toMillis(); + } +} diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaSinkConfig.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaSinkConfig.java new file mode 100644 index 0000000000..0e35750657 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaSinkConfig.java @@ -0,0 +1,140 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.Valid; +import jakarta.validation.constraints.NotEmpty; +import jakarta.validation.constraints.NotNull; +import jakarta.validation.constraints.Size; +import org.apache.commons.lang3.ObjectUtils; +import org.opensearch.dataprepper.model.configuration.PluginModel; +import org.opensearch.dataprepper.model.configuration.PluginSetting; + +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +/** + * * A helper class that helps to read user configuration values from + * pipelines.yaml + */ + +public class KafkaSinkConfig { + + public static final String DLQ = "dlq"; + + @JsonProperty("bootstrap_servers") + @NotNull + @Size(min = 1, message = "Bootstrap servers can't be empty") + private List bootStrapServers; + + private PluginModel dlq; + + public Optional getDlq() { + return Optional.ofNullable(dlq); + } + + public void setDlqConfig(final PluginSetting pluginSetting) { + final LinkedHashMap> dlq = (LinkedHashMap) pluginSetting.getAttributeFromSettings(DLQ); + if (dlq != null) { + if (dlq.size() != 1) { + throw new RuntimeException("dlq option must declare exactly one dlq configuration"); + } + final Map.Entry> entry = dlq.entrySet().stream() + .findFirst() + .get(); + + this.dlq = new PluginModel(entry.getKey(), entry.getValue()); + + } + } + + + @JsonProperty("thread_wait_time") + private Long threadWaitTime; + + + @JsonProperty("topic") + TopicConfig topic; + + @JsonProperty("authentication") + private AuthConfig authConfig; + + @JsonProperty("schema") + @Valid + private SchemaConfig schemaConfig; + + @JsonProperty("serde_format") + private String serdeFormat; + + + @JsonProperty("partition_key") + @NotNull + @NotEmpty + private String partitionKey; + + @JsonProperty("producer_properties") + private KafkaProducerProperties kafkaProducerProperties; + + public SchemaConfig getSchemaConfig() { + return schemaConfig; + } + + + public AuthConfig getAuthConfig() { + return authConfig; + } + + + public List getBootStrapServers() { + return bootStrapServers; + } + + public String getSerdeFormat() { + if (ObjectUtils.isEmpty(serdeFormat)) { + serdeFormat = "plaintext"; + } + return serdeFormat; + } + + public Long getThreadWaitTime() { + return threadWaitTime; + } + + public void setBootStrapServers(List bootStrapServers) { + this.bootStrapServers = bootStrapServers; + } + + public void setThreadWaitTime(Long threadWaitTime) { + this.threadWaitTime = threadWaitTime; + } + + public void setAuthConfig(AuthConfig authConfig) { + this.authConfig = authConfig; + } + + public void setSchemaConfig(SchemaConfig schemaConfig) { + this.schemaConfig = schemaConfig; + } + + public TopicConfig getTopic() { + return topic; + } + + public void setTopic(TopicConfig topic) { + this.topic = topic; + } + + public KafkaProducerProperties getKafkaProducerProperties() { + return kafkaProducerProperties; + } + + public String getPartitionKey() { + return partitionKey; + } +} diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/PlainConfig.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/PlainConfig.java new file mode 100644 index 0000000000..01a79c6447 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/PlainConfig.java @@ -0,0 +1,26 @@ +package org.opensearch.dataprepper.plugins.kafka.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.NotNull; + +public class PlainConfig { + @JsonProperty("username") + @NotNull + private String userName; + + @JsonProperty("password") + @NotNull + private String password; + + public String getUserName() { + return userName; + } + + public String getPassword() { + return password; + } + + public PlainConfig() { + + } +} diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/producer/KafkaSinkProducer.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/producer/KafkaSinkProducer.java new file mode 100644 index 0000000000..a416c86076 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/producer/KafkaSinkProducer.java @@ -0,0 +1,171 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.producer; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.github.fge.jsonschema.core.exceptions.ProcessingException; +import com.github.fge.jsonschema.core.report.ProcessingReport; +import com.github.fge.jsonschema.main.JsonSchema; +import com.github.fge.jsonschema.main.JsonSchemaFactory; +import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient; +import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.opensearch.dataprepper.expression.ExpressionEvaluator; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventHandle; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSinkConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; +import org.opensearch.dataprepper.plugins.kafka.sink.DLQSink; +import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Collection; +import java.util.LinkedList; + + +/** + * * A helper class which helps takes the buffer data + * and produce it to a given kafka topic + */ + +public class KafkaSinkProducer { + + private static final Logger LOG = LoggerFactory.getLogger(KafkaSinkProducer.class); + + private final Producer producer; + + private final KafkaSinkConfig kafkaSinkConfig; + + private final DLQSink dlqSink; + + private final CachedSchemaRegistryClient schemaRegistryClient; + + private final Collection bufferedEventHandles; + + final ExpressionEvaluator expressionEvaluator; + + public KafkaSinkProducer(final Producer producer, + final KafkaSinkConfig kafkaSinkConfig, + final DLQSink dlqSink, + final CachedSchemaRegistryClient schemaRegistryClient, + final ExpressionEvaluator expressionEvaluator) { + this.producer = producer; + this.kafkaSinkConfig = kafkaSinkConfig; + this.dlqSink = dlqSink; + this.schemaRegistryClient = schemaRegistryClient; + bufferedEventHandles = new LinkedList<>(); + this.expressionEvaluator = expressionEvaluator; + + } + + public void produceRecords(final Record record) { + if (record.getData().getEventHandle() != null) { + bufferedEventHandles.add(record.getData().getEventHandle()); + } + TopicConfig topic = kafkaSinkConfig.getTopic(); + final Event event = record.getData(); + final String key = event.formatString(kafkaSinkConfig.getPartitionKey(), expressionEvaluator); + Object dataForDlq = event.toJsonString(); + LOG.info("Producing record " + dataForDlq); + try { + final String serdeFormat = kafkaSinkConfig.getSerdeFormat(); + if (MessageFormat.JSON.toString().equalsIgnoreCase(serdeFormat)) { + publishJsonMessage(record, topic, key, dataForDlq); + } else if (MessageFormat.AVRO.toString().equalsIgnoreCase(serdeFormat)) { + publishAvroMessage(record, topic, key, dataForDlq); + } else { + publishPlaintextMessage(record, topic, key, dataForDlq); + } + } catch (Exception e) { + releaseEventHandles(false); + } + + + } + + private void publishPlaintextMessage(Record record, TopicConfig topic, String key, Object dataForDlq) { + producer.send(new ProducerRecord(topic.getName(), key, record.getData().toJsonString()), callBack(dataForDlq)); + } + + private void publishAvroMessage(Record record, TopicConfig topic, String key, Object dataForDlq) throws RestClientException, IOException { + final String valueToParse = schemaRegistryClient. + getLatestSchemaMetadata(topic.getName() + "-value").getSchema(); + final Schema schema = new Schema.Parser().parse(valueToParse); + final GenericRecord genericRecord = getGenericRecord(record.getData(), schema); + producer.send(new ProducerRecord(topic.getName(), key, genericRecord), callBack(dataForDlq)); + } + + private void publishJsonMessage(Record record, TopicConfig topic, String key, Object dataForDlq) throws IOException, RestClientException, ProcessingException { + final JsonNode dataNode = new ObjectMapper().convertValue(record.getData().toJsonString(), JsonNode.class); + if (validateJson(topic.getName(), dataForDlq)) { + producer.send(new ProducerRecord(topic.getName(), key, dataNode), callBack(dataForDlq)); + } else { + dlqSink.perform(dataForDlq, new RuntimeException("Invalid Json")); + } + } + + private Boolean validateJson(final String topicName, Object dataForDlq) throws IOException, RestClientException, ProcessingException { + if (schemaRegistryClient != null) { + final String schemaJson = schemaRegistryClient. + getLatestSchemaMetadata(topicName + "-value").getSchema(); + return validateSchema(dataForDlq.toString(), schemaJson); + } else { + return true; + } + } + + private boolean validateSchema(final String jsonData, final String schemaJson) throws IOException, ProcessingException { + ObjectMapper objectMapper = new ObjectMapper(); + JsonNode schemaNode = objectMapper.readTree(schemaJson); + JsonNode dataNode = objectMapper.readTree(jsonData); + JsonSchemaFactory schemaFactory = JsonSchemaFactory.byDefault(); + JsonSchema schema = schemaFactory.getJsonSchema(schemaNode); + ProcessingReport report = schema.validate(dataNode); + if (report.isSuccess()) { + return true; + } else { + return false; + } + } + + private Callback callBack(final Object dataForDlq) { + return (metadata, exception) -> { + if (null != exception) { + releaseEventHandles(false); + dlqSink.perform(dataForDlq, exception); + } else { + releaseEventHandles(true); + } + }; + } + + + private GenericRecord getGenericRecord(final Event event,final Schema schema) { + final GenericRecord record = new GenericData.Record(schema); + for (final String key : event.toMap().keySet()) { + record.put(key, event.toMap().get(key)); + } + return record; + } + + private void releaseEventHandles(final boolean result) { + for (final EventHandle eventHandle : bufferedEventHandles) { + eventHandle.release(result); + } + bufferedEventHandles.clear(); + } + + +} diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/producer/ProducerWorker.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/producer/ProducerWorker.java new file mode 100644 index 0000000000..dbb05e7401 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/producer/ProducerWorker.java @@ -0,0 +1,33 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.producer; + +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; + +/** + * * A Multithreaded helper class which helps to produce the records to multiple topics in an + * asynchronous way. + */ + +public class ProducerWorker implements Runnable { + + private final Record record; + private final KafkaSinkProducer producer; + + + public ProducerWorker(final KafkaSinkProducer producer, + final Record record) { + this.record = record; + this.producer = producer; + } + + @Override + public void run() { + producer.produceRecords(record); + } + +} diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/sink/DLQSink.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/sink/DLQSink.java new file mode 100644 index 0000000000..5b393d3f19 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/sink/DLQSink.java @@ -0,0 +1,94 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + + +package org.opensearch.dataprepper.plugins.kafka.sink; + +import org.opensearch.dataprepper.metrics.MetricNames; +import org.opensearch.dataprepper.model.configuration.PluginModel; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.failures.DlqObject; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.plugins.dlq.DlqProvider; +import org.opensearch.dataprepper.plugins.dlq.DlqWriter; +import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSinkConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.StringJoiner; + +import static java.util.UUID.randomUUID; +import static org.opensearch.dataprepper.logging.DataPrepperMarkers.SENSITIVE; + + +/** + * *This class which helps log failed data to AWS S3 bucket + */ + +public class DLQSink { + + private static final Logger LOG = LoggerFactory.getLogger(DLQSink.class); + + private final DlqProvider dlqProvider; + private final PluginSetting pluginSetting; + + public DLQSink(final PluginFactory pluginFactory, final KafkaSinkConfig kafkaSinkConfig, final PluginSetting pluginSetting) { + this.pluginSetting = pluginSetting; + this.dlqProvider = getDlqProvider(pluginFactory, kafkaSinkConfig); + } + + public void perform(final Object failedData, final Exception e) { + final DlqWriter dlqWriter = getDlqWriter(); + final DlqObject dlqObject = DlqObject.builder() + .withPluginId(randomUUID().toString()) + .withPluginName(pluginSetting.getName()) + .withPipelineName(pluginSetting.getPipelineName()) + .withFailedData(failedData) + .build(); + logFailureForDlqObjects(dlqWriter, List.of(dlqObject), e); + } + + private DlqWriter getDlqWriter() { + final Optional potentialDlq = dlqProvider.getDlqWriter(new StringJoiner(MetricNames.DELIMITER) + .add(pluginSetting.getPipelineName()) + .add(pluginSetting.getName()).toString()); + final DlqWriter dlqWriter = potentialDlq.isPresent() ? potentialDlq.get() : null; + return dlqWriter; + } + + private DlqProvider getDlqProvider(final PluginFactory pluginFactory, final KafkaSinkConfig kafkaSinkConfig) { + final Map props = new HashMap<>(); + kafkaSinkConfig.setDlqConfig(pluginSetting); + final Optional dlq = kafkaSinkConfig.getDlq(); + if (dlq.isPresent()) { + final PluginModel dlqPluginModel = dlq.get(); + final PluginSetting dlqPluginSetting = new PluginSetting(dlqPluginModel.getPluginName(), dlqPluginModel.getPluginSettings()); + dlqPluginSetting.setPipelineName(pluginSetting.getPipelineName()); + final DlqProvider dlqProvider = pluginFactory.loadPlugin(DlqProvider.class, dlqPluginSetting); + return dlqProvider; + } + return null; + } + + private void logFailureForDlqObjects(final DlqWriter dlqWriter, final List dlqObjects, final Throwable failure) { + try { + dlqWriter.write(dlqObjects, pluginSetting.getPipelineName(), pluginSetting.getName()); + dlqObjects.forEach((dlqObject) -> { + dlqObject.releaseEventHandle(true); + }); + } catch (final IOException e) { + dlqObjects.forEach(dlqObject -> { + LOG.error(SENSITIVE, "DLQ failure for Document[{}]", dlqObject.getFailedData(), e); + dlqObject.releaseEventHandle(false); + }); + } + } +} + diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkaSink.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkaSink.java new file mode 100644 index 0000000000..00365236ad --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkaSink.java @@ -0,0 +1,160 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.sink; + +import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.opensearch.dataprepper.expression.ExpressionEvaluator; +import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; +import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.sink.AbstractSink; +import org.opensearch.dataprepper.model.sink.Sink; +import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSinkConfig; +import org.opensearch.dataprepper.plugins.kafka.producer.KafkaSinkProducer; +import org.opensearch.dataprepper.plugins.kafka.producer.ProducerWorker; +import org.opensearch.dataprepper.plugins.kafka.util.SinkPropertyConfigurer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; +import java.util.Map; +import java.util.Objects; +import java.util.Properties; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + +/** + * Implementation class of kafka--sink plugin. It is responsible for receive the collection of + * {@link Event} and produce it to different kafka topics. + */ +@DataPrepperPlugin(name = "kafka", pluginType = Sink.class, pluginConfigurationType = KafkaSinkConfig.class) +public class KafkaSink extends AbstractSink> { + + private static final Logger LOG = LoggerFactory.getLogger(KafkaSink.class); + + private final KafkaSinkConfig kafkaSinkConfig; + + private volatile boolean sinkInitialized; + + private static final Integer totalWorkers = 1; + + private ProducerWorker producerWorker; + + private ExecutorService executorService; + + private final PluginFactory pluginFactory; + + private final PluginSetting pluginSetting; + + private final ExpressionEvaluator expressionEvaluator; + + + @DataPrepperPluginConstructor + public KafkaSink(final PluginSetting pluginSetting, final KafkaSinkConfig kafkaSinkConfig, final PluginFactory pluginFactory, + final ExpressionEvaluator expressionEvaluator) { + super(pluginSetting); + this.pluginSetting = pluginSetting; + this.kafkaSinkConfig = kafkaSinkConfig; + this.pluginFactory = pluginFactory; + this.expressionEvaluator = expressionEvaluator; + + } + + + @Override + public void doInitialize() { + try { + doInitializeInternal(); + } catch (InvalidPluginConfigurationException e) { + LOG.error("Invalid plugin configuration, Hence failed to initialize kafka-sink plugin."); + this.shutdown(); + throw e; + } catch (Exception e) { + LOG.error("Failed to initialize kafka-sink plugin."); + this.shutdown(); + throw e; + } + } + + private void doInitializeInternal() { + executorService = Executors.newFixedThreadPool(totalWorkers); + sinkInitialized = Boolean.TRUE; + } + + @Override + public void doOutput(Collection> records) { + if (records.isEmpty()) { + return; + } + try { + final KafkaSinkProducer producer = createProducer(); + records.forEach(record -> { + producerWorker = new ProducerWorker(producer, record); + executorService.submit(producerWorker); + }); + + } catch (Exception e) { + LOG.error("Failed to setup the Kafka sink Plugin.", e); + throw new RuntimeException(e.getMessage()); + } + } + + public KafkaSinkProducer createProducer() { + Properties properties = SinkPropertyConfigurer.getProducerProperties(kafkaSinkConfig); + properties = Objects.requireNonNull(properties); + return new KafkaSinkProducer(new KafkaProducer<>(properties), + kafkaSinkConfig, new DLQSink(pluginFactory, kafkaSinkConfig, pluginSetting), getSchemaRegistryClient(), expressionEvaluator); + } + + private CachedSchemaRegistryClient getSchemaRegistryClient() { + if (kafkaSinkConfig.getSchemaConfig() != null && kafkaSinkConfig.getSchemaConfig().getRegistryURL() != null) { + Properties schemaProps = new Properties(); + SinkPropertyConfigurer.setSchemaProps(kafkaSinkConfig, schemaProps); + Map propertiesMap = schemaProps; + return new CachedSchemaRegistryClient( + kafkaSinkConfig.getSchemaConfig().getRegistryURL(), + 100, propertiesMap); + } + return null; + } + + @Override + public void shutdown() { + try { + if (!executorService.awaitTermination( + calculateLongestThreadWaitingTime(), TimeUnit.MILLISECONDS)) { + executorService.shutdownNow(); + } + LOG.info("Sink threads are waiting for shutting down..."); + } catch (InterruptedException e) { + if (e.getCause() instanceof InterruptedException) { + LOG.error("Interrupted during sink shutdown, exiting uncleanly..."); + executorService.shutdownNow(); + Thread.currentThread().interrupt(); + } + } + super.shutdown(); + LOG.info("Producer shutdown successfully..."); + } + + private long calculateLongestThreadWaitingTime() { + return kafkaSinkConfig.getThreadWaitTime(); + } + + + @Override + public boolean isReady() { + return sinkInitialized; + } + +} + diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/AuthenticationPropertyConfigurer.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/AuthenticationPropertyConfigurer.java new file mode 100644 index 0000000000..53ae709c81 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/AuthenticationPropertyConfigurer.java @@ -0,0 +1,145 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.kafka.util; + +import org.opensearch.dataprepper.plugins.kafka.configuration.OAuthConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.PlainConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.PlainTextAuthConfig; + +import java.util.Base64; +import java.util.Properties; + +/** + * * This is static property configurer dedicated to authencation related information given in pipeline.yml + */ + +public class AuthenticationPropertyConfigurer { + + private static final String SASL_MECHANISM = "sasl.mechanism"; + + private static final String SASL_SECURITY_PROTOCOL = "security.protocol"; + + private static final String SASL_JAS_CONFIG = "sasl.jaas.config"; + + private static final String SASL_CALLBACK_HANDLER_CLASS = "sasl.login.callback.handler.class"; + + private static final String SASL_JWKS_ENDPOINT_URL = "sasl.oauthbearer.jwks.endpoint.url"; + + private static final String SASL_TOKEN_ENDPOINT_URL = "sasl.oauthbearer.token.endpoint.url"; + + private static final String PLAINTEXT_JAASCONFIG = "org.apache.kafka.common.security.plain.PlainLoginModule required username= \"%s\" password= " + + " \"%s\";"; + private static final String OAUTH_JAASCONFIG = "org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule required clientId='" + + "%s" + "' clientSecret='" + "%s" + "' scope='" + "%s" + "' OAUTH_LOGIN_SERVER='" + "%s" + + "' OAUTH_LOGIN_ENDPOINT='" + "%s" + "' OAUT_LOGIN_GRANT_TYPE=" + "%s" + + " OAUTH_LOGIN_SCOPE=%s OAUTH_AUTHORIZATION='Basic " + "%s" + "';"; + + private static final String INSTROSPECT_SERVER_PROPERTIES = " OAUTH_INTROSPECT_SERVER='" + + "%s" + "' OAUTH_INTROSPECT_ENDPOINT='" + "%s" + "' " + + "OAUTH_INTROSPECT_AUTHORIZATION='Basic " + "%s"; + + private static final String PLAIN_MECHANISM = "PLAIN"; + + private static final String SASL_PLAINTEXT_PROTOCOL = "SASL_PLAINTEXT"; + + private static final String SASL_SSL_PROTOCOL = "SASL_SSL"; + + private static final String SSL_ENDPOINT_IDENTIFICATION = "ssl.endpoint.identification.algorithm"; + + private static final String USER_INFO = "USER_INFO"; + + + public static void setSaslPlainTextProperties(final PlainTextAuthConfig plainTextAuthConfig, + final Properties properties) { + + String username = plainTextAuthConfig.getUsername(); + String password = plainTextAuthConfig.getPassword(); + properties.put(SASL_MECHANISM, PLAIN_MECHANISM); + properties.put(SASL_JAS_CONFIG, String.format(PLAINTEXT_JAASCONFIG, username, password)); + properties.put(SASL_SECURITY_PROTOCOL, SASL_PLAINTEXT_PROTOCOL); + } + + public static void setOauthProperties(final OAuthConfig oAuthConfig, + final Properties properties) { + final String oauthClientId = oAuthConfig.getOauthClientId(); + final String oauthClientSecret = oAuthConfig.getOauthClientSecret(); + final String oauthLoginServer = oAuthConfig.getOauthLoginServer(); + final String oauthLoginEndpoint = oAuthConfig.getOauthLoginEndpoint(); + final String oauthLoginGrantType = oAuthConfig.getOauthLoginGrantType(); + final String oauthLoginScope = oAuthConfig.getOauthLoginScope(); + final String oauthAuthorizationToken = Base64.getEncoder().encodeToString((oauthClientId + ":" + oauthClientSecret).getBytes()); + final String oauthIntrospectEndpoint = oAuthConfig.getOauthIntrospectEndpoint(); + final String tokenEndPointURL = oAuthConfig.getOauthTokenEndpointURL(); + final String saslMechanism = oAuthConfig.getOauthSaslMechanism(); + final String securityProtocol = oAuthConfig.getOauthSecurityProtocol(); + final String loginCallBackHandler = oAuthConfig.getOauthSaslLoginCallbackHandlerClass(); + final String oauthJwksEndpointURL = oAuthConfig.getOauthJwksEndpointURL(); + final String introspectServer = oAuthConfig.getOauthIntrospectServer(); + final String extensionLogicalCluster = oAuthConfig.getExtensionLogicalCluster(); + final String extensionIdentityPoolId = oAuthConfig.getExtensionIdentityPoolId(); + + properties.put(SASL_MECHANISM, saslMechanism); + properties.put(SASL_SECURITY_PROTOCOL, securityProtocol); + properties.put(SASL_TOKEN_ENDPOINT_URL, tokenEndPointURL); + properties.put(SASL_CALLBACK_HANDLER_CLASS, loginCallBackHandler); + + populateJwksEndpoint(properties, oauthJwksEndpointURL); + String instrospect_properties = getInstrospectProperties(oauthAuthorizationToken, oauthIntrospectEndpoint, oauthJwksEndpointURL, introspectServer); + String jass_config = createJassConfig(oauthClientId, oauthClientSecret, oauthLoginServer, + oauthLoginEndpoint, oauthLoginGrantType, oauthLoginScope, oauthAuthorizationToken, + extensionLogicalCluster, extensionIdentityPoolId, instrospect_properties); + + properties.put(SASL_JAS_CONFIG, jass_config); + } + + private static void populateJwksEndpoint(final Properties properties, final String oauthJwksEndpointURL) { + if (oauthJwksEndpointURL != null && !oauthJwksEndpointURL.isEmpty() && !oauthJwksEndpointURL.isBlank()) { + properties.put(SASL_JWKS_ENDPOINT_URL, oauthJwksEndpointURL); + } + } + + private static String getInstrospectProperties(final String oauthAuthorizationToken, final String oauthIntrospectEndpoint, + final String oauthJwksEndpointURL, final String introspectServer) { + String instrospect_properties = ""; + if (oauthJwksEndpointURL != null && !oauthIntrospectEndpoint.isBlank() && !oauthIntrospectEndpoint.isEmpty()) { + instrospect_properties = String.format(INSTROSPECT_SERVER_PROPERTIES, introspectServer, oauthIntrospectEndpoint, oauthAuthorizationToken); + } + return instrospect_properties; + } + + private static String createJassConfig(final String oauthClientId, final String oauthClientSecret, final String oauthLoginServer, + final String oauthLoginEndpoint, final String oauthLoginGrantType, final String oauthLoginScope, + final String oauthAuthorizationToken, final String extensionLogicalCluster, + final String extensionIdentityPoolId, final String instrospect_properties) { + String jass_config = String.format(OAUTH_JAASCONFIG, oauthClientId, oauthClientSecret, oauthLoginScope, oauthLoginServer, + oauthLoginEndpoint, oauthLoginGrantType, oauthLoginScope, oauthAuthorizationToken, instrospect_properties); + + jass_config = getJassConfigWithClusterInforation(extensionLogicalCluster, extensionIdentityPoolId, jass_config); + return jass_config; + } + + private static String getJassConfigWithClusterInforation(final String extensionLogicalCluster, final String extensionIdentityPoolId, + String jass_config) { + if (extensionLogicalCluster != null && extensionIdentityPoolId != null) { + String extensionValue = "extension_logicalCluster= \"%s\" extension_identityPoolId=" + " \"%s\";"; + jass_config = jass_config.replace(";", " "); + jass_config = jass_config + String.format(extensionValue, extensionLogicalCluster, extensionIdentityPoolId); + } + return jass_config; + } + + public static void setSaslPlainProperties(final PlainConfig sslAuthConfig, + final Properties properties) { + + String username = sslAuthConfig.getUserName(); + String password = sslAuthConfig.getPassword(); + properties.put(SASL_MECHANISM, PLAIN_MECHANISM); + properties.put(SASL_JAS_CONFIG, String.format(PLAINTEXT_JAASCONFIG, username, password)); + properties.put(SASL_SECURITY_PROTOCOL, SASL_SSL_PROTOCOL); + + } + +} + diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/SinkPropertyConfigurer.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/SinkPropertyConfigurer.java new file mode 100644 index 0000000000..9b97f01039 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/SinkPropertyConfigurer.java @@ -0,0 +1,313 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.util; + + +import io.confluent.kafka.serializers.KafkaAvroSerializer; +import org.apache.commons.lang3.ObjectUtils; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.connect.json.JsonSerializer; +import org.opensearch.dataprepper.model.types.ByteCount; +import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaProducerProperties; +import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSinkConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.SchemaConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.Properties; + +/** + * * This is static property configurer for related information given in pipeline.yml + */ +public class SinkPropertyConfigurer { + + private static final Logger LOG = LoggerFactory.getLogger(SinkPropertyConfigurer.class); + + private static final String VALUE_SERIALIZER = "value.serializer"; + + private static final String KEY_SERIALIZER = "key.serializer"; + + private static final String SESSION_TIMEOUT_MS_CONFIG = "30000"; + + private static final String REGISTRY_URL = "schema.registry.url"; + + private static final String REGISTRY_BASIC_AUTH_USER_INFO = "schema.registry.basic.auth.user.info"; + + private static final String CREDENTIALS_SOURCE = "basic.auth.credentials.source"; + + public static final String BUFFER_MEMORY = "buffer.memory"; + + public static final String COMPRESSION_TYPE = "compression.type"; + + public static final String RETRIES = "retries"; + + public static final String BATCH_SIZE = "batch.size"; + + public static final String CLIENT_DNS_LOOKUP = "client.dns.lookup"; + + public static final String CLIENT_ID = "client.id"; + + public static final String CONNECTIONS_MAX_IDLE_MS = "connections.max.idle.ms"; + + public static final String DELIVERY_TIMEOUT_MS = "delivery.timeout.ms"; + + public static final String LINGER_MS = "linger.ms"; + + public static final String MAX_BLOCK_MS = "max.block.ms"; + + public static final String MAX_REQUEST_SIZE = "max.request.size"; + + public static final String PARTITIONER_CLASS = "partitioner.class"; + + public static final String PARTITIONER_IGNORE_KEYS = "partitioner.ignore.keys"; + + public static final String RECEIVE_BUFFER_BYTES = "receive.buffer.bytes"; + + public static final String REQUEST_TIMEOUT_MS = "request.timeout.ms"; + + public static final String SEND_BUFFER_BYTES = "send.buffer.bytes"; + + public static final String SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS = "socket.connection.setup.timeout.max.ms"; + + public static final String SOCKET_CONNECTION_SETUP_TIMEOUT_MS = "socket.connection.setup.timeout.ms"; + + public static final String ACKS = "acks"; + + public static final String ENABLE_IDEMPOTENCE = "enable.idempotence"; + + public static final String INTERCEPTOR_CLASSES = "interceptor.classes"; + + public static final String MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION = "max.in.flight.requests.per.connection"; + + public static final String METADATA_MAX_AGE_MS = "metadata.max.age.ms"; + + public static final String METADATA_MAX_IDLE_MS = "metadata.max.idle.ms"; + + public static final String METRIC_REPORTERS = "metric.reporters"; + + public static final String METRICS_NUM_SAMPLES = "metrics.num.samples"; + + public static final String METRICS_RECORDING_LEVEL = "metrics.recording.level"; + + public static final String METRICS_SAMPLE_WINDOW_MS = "metrics.sample.window.ms"; + + public static final String PARTITIONER_ADAPTIVE_PARTITIONING_ENABLE = "partitioner.adaptive.partitioning.enable"; + + public static final String PARTITIONER_AVAILABILITY_TIMEOUT_MS = "partitioner.availability.timeout.ms"; + + public static final String RECONNECT_BACKOFF_MAX_MS = "reconnect.backoff.max.ms"; + + public static final String RECONNECT_BACKOFF_MS = "reconnect.backoff.ms"; + + public static final String RETRY_BACKOFF_MS = "retry.backoff.ms"; + + public static Properties getProducerProperties(final KafkaSinkConfig kafkaSinkConfig) { + final Properties properties = new Properties(); + + setCommonServerProperties(properties, kafkaSinkConfig); + + setPropertiesForSerializer(properties, kafkaSinkConfig.getSerdeFormat()); + + if (kafkaSinkConfig.getSchemaConfig() != null) { + setSchemaProps(kafkaSinkConfig, properties); + } + if (kafkaSinkConfig.getKafkaProducerProperties() != null) { + setPropertiesProviderByKafkaProducer(kafkaSinkConfig.getKafkaProducerProperties(), properties); + } + + if (kafkaSinkConfig.getAuthConfig().getSaslAuthConfig().getPlainTextAuthConfig() != null) { + AuthenticationPropertyConfigurer.setSaslPlainTextProperties(kafkaSinkConfig.getAuthConfig() + .getSaslAuthConfig().getPlainTextAuthConfig(), properties); + } else if (kafkaSinkConfig.getAuthConfig().getSaslAuthConfig().getOAuthConfig() != null) { + AuthenticationPropertyConfigurer.setOauthProperties(kafkaSinkConfig.getAuthConfig().getSaslAuthConfig() + .getOAuthConfig(), properties); + } else if (kafkaSinkConfig.getAuthConfig().getSaslAuthConfig().getPlain() != null) { + AuthenticationPropertyConfigurer.setSaslPlainProperties(kafkaSinkConfig.getAuthConfig().getSaslAuthConfig().getPlain(), properties); + } + + return properties; + } + + private static void setCommonServerProperties(final Properties properties, final KafkaSinkConfig kafkaSinkConfig) { + properties.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, kafkaSinkConfig.getBootStrapServers()); + properties.put(CommonClientConfigs.SESSION_TIMEOUT_MS_CONFIG, SESSION_TIMEOUT_MS_CONFIG); + } + + private static void setPropertiesForSerializer(final Properties properties, final String serdeFormat) { + properties.put(KEY_SERIALIZER, StringSerializer.class.getName()); + if (serdeFormat.equalsIgnoreCase(MessageFormat.JSON.toString())) { + properties.put(VALUE_SERIALIZER, JsonSerializer.class.getName()); + } else if (serdeFormat.equalsIgnoreCase(MessageFormat.AVRO.toString())) { + properties.put(VALUE_SERIALIZER, KafkaAvroSerializer.class.getName()); + } else { + properties.put(VALUE_SERIALIZER, StringSerializer.class.getName()); + } + } + + private static void validateForRegistryURL(final KafkaSinkConfig kafkaSinkConfig) { + String serdeFormat = kafkaSinkConfig.getSerdeFormat(); + if (serdeFormat.equalsIgnoreCase(MessageFormat.AVRO.toString())) { + if (kafkaSinkConfig.getSchemaConfig() == null || kafkaSinkConfig.getSchemaConfig().getRegistryURL() == null || + kafkaSinkConfig.getSchemaConfig().getRegistryURL().isBlank() || kafkaSinkConfig.getSchemaConfig().getRegistryURL().isEmpty()) { + throw new RuntimeException("Schema registry is mandatory when serde type is avro"); + } + } + if (serdeFormat.equalsIgnoreCase(MessageFormat.PLAINTEXT.toString())) { + if (kafkaSinkConfig.getSchemaConfig() != null && + kafkaSinkConfig.getSchemaConfig().getRegistryURL() != null) { + throw new RuntimeException("Schema registry is not required for type plaintext"); + } + } + } + + public static void setSchemaProps(final KafkaSinkConfig kafkaSinkConfig, final Properties properties) { + validateForRegistryURL(kafkaSinkConfig); + SchemaConfig schemaConfig = kafkaSinkConfig.getSchemaConfig(); + final String registryURL = schemaConfig != null ? schemaConfig.getRegistryURL() : null; + if (registryURL != null && !registryURL.isEmpty()) { + properties.put(REGISTRY_URL, registryURL); + } + if (!ObjectUtils.isEmpty(schemaConfig.getBasicAuthCredentialsSource())) { + properties.put(CREDENTIALS_SOURCE, schemaConfig.getBasicAuthCredentialsSource()); + } + if (!ObjectUtils.isEmpty(schemaConfig.getSchemaRegistryApiKey()) && !(ObjectUtils.isEmpty(schemaConfig.getSchemaRegistryApiSecret()))) { + final String apiKey = schemaConfig.getSchemaRegistryApiKey(); + final String apiSecret = schemaConfig.getSchemaRegistryApiSecret(); + properties.put(REGISTRY_BASIC_AUTH_USER_INFO, apiKey + ":" + apiSecret); + } + } + + private static void setPropertiesProviderByKafkaProducer(final KafkaProducerProperties producerProperties, final + Properties properties) { + + if (producerProperties.getBufferMemory() != null) { + properties.put(BUFFER_MEMORY, ByteCount.parse(producerProperties.getBufferMemory()).getBytes()); + } + if (producerProperties.getCompressionType() != null) { + properties.put(COMPRESSION_TYPE, producerProperties.getCompressionType()); + } + properties.put(RETRIES, producerProperties.getRetries()); + + if (producerProperties.getBatchSize() > 0) { + properties.put(BATCH_SIZE, producerProperties.getBatchSize()); + } + if (producerProperties.getClientDnsLookup() != null) { + properties.put(CLIENT_DNS_LOOKUP, producerProperties.getClientDnsLookup()); + } + if (producerProperties.getClientId() != null) { + properties.put(CLIENT_ID, producerProperties.getClientId()); + } + if (producerProperties.getConnectionsMaxIdleMs() > 0) { + properties.put(CONNECTIONS_MAX_IDLE_MS, producerProperties.getConnectionsMaxIdleMs()); + } + if (producerProperties.getDeliveryTimeoutMs() > 0) { + properties.put(DELIVERY_TIMEOUT_MS, producerProperties.getDeliveryTimeoutMs().intValue()); + } + if (producerProperties.getLingerMs() > 0) { + properties.put(LINGER_MS, (producerProperties.getLingerMs())); + } + if (producerProperties.getMaxBlockMs() > 0) { + properties.put(MAX_BLOCK_MS, producerProperties.getMaxBlockMs()); + } + if (producerProperties.getMaxRequestSize() > 0) { + properties.put(MAX_REQUEST_SIZE, producerProperties.getMaxRequestSize()); + } + if (producerProperties.getPartitionerClass() != null) { + properties.put(PARTITIONER_CLASS, producerProperties.getPartitionerClass().getName()); + } + if (producerProperties.getPartitionerIgnoreKeys() != null) { + properties.put(PARTITIONER_IGNORE_KEYS, producerProperties.getPartitionerIgnoreKeys()); + } + if (producerProperties.getReceiveBufferBytes() != null) { + final Long receiveBufferBytes = ByteCount.parse(producerProperties.getReceiveBufferBytes()).getBytes(); + properties.put(RECEIVE_BUFFER_BYTES, receiveBufferBytes.intValue()); + } + if (producerProperties.getRequestTimeoutMs() > 0) { + properties.put(REQUEST_TIMEOUT_MS, producerProperties.getRequestTimeoutMs().intValue()); + } + if (producerProperties.getSendBufferBytes() != null) { + final Long sendBufferBytes = ByteCount.parse(producerProperties.getSendBufferBytes()).getBytes(); + properties.put(SEND_BUFFER_BYTES, sendBufferBytes.intValue()); + } + if (producerProperties.getSocketConnectionSetupMaxTimeout() > 0) { + properties.put(SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS, producerProperties.getSocketConnectionSetupMaxTimeout()); + } + if (producerProperties.getSocketConnectionSetupTimeout() > 0) { + properties.put(SOCKET_CONNECTION_SETUP_TIMEOUT_MS, producerProperties.getSocketConnectionSetupTimeout()); + } + if (producerProperties.getAcks() != null) { + properties.put(ACKS, producerProperties.getAcks()); + } + if (producerProperties.getEnableIdempotence() != null) { + properties.put(ENABLE_IDEMPOTENCE, producerProperties.getEnableIdempotence()); + } + + + List interceptorClasses = producerProperties.getInterceptorClasses(); + if (interceptorClasses != null && !interceptorClasses.isEmpty()) { + properties.put(INTERCEPTOR_CLASSES, String.join(",", interceptorClasses)); + } + + if (producerProperties.getMaxInFlightRequestsPerConnection() > 0) { + properties.put(MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, producerProperties.getMaxInFlightRequestsPerConnection()); + } + + if (producerProperties.getMetadataMaxAgeMs() > 0) { + properties.put(METADATA_MAX_AGE_MS, producerProperties.getMetadataMaxAgeMs()); + } + + if (producerProperties.getMetadataMaxIdleMs() > 0) { + properties.put(METADATA_MAX_IDLE_MS, producerProperties.getMetadataMaxIdleMs()); + } + + + List metricReporters = producerProperties.getMetricReporters(); + if (metricReporters != null && !metricReporters.isEmpty()) { + properties.put(METRIC_REPORTERS, String.join(",", metricReporters)); + } + + if (producerProperties.getMetricsNumSamples() > 0) { + properties.put(METRICS_NUM_SAMPLES, producerProperties.getMetricsNumSamples()); + } + + if (producerProperties.getMetricsRecordingLevel() != null) { + properties.put(METRICS_RECORDING_LEVEL, producerProperties.getMetricsRecordingLevel()); + } + + if (producerProperties.getMetricsSampleWindowMs() > 0) { + properties.put(METRICS_SAMPLE_WINDOW_MS, producerProperties.getMetricsSampleWindowMs()); + } + + properties.put(PARTITIONER_ADAPTIVE_PARTITIONING_ENABLE, producerProperties.isPartitionerAdaptivePartitioningEnable()); + + if (producerProperties.getPartitionerAvailabilityTimeoutMs() > 0) { + properties.put(PARTITIONER_AVAILABILITY_TIMEOUT_MS, producerProperties.getPartitionerAvailabilityTimeoutMs()); + } + + if (producerProperties.getReconnectBackoffMaxMs() > 0) { + properties.put(RECONNECT_BACKOFF_MAX_MS, producerProperties.getReconnectBackoffMaxMs()); + } + + if (producerProperties.getReconnectBackoffMs() > 0) { + properties.put(RECONNECT_BACKOFF_MS, producerProperties.getReconnectBackoffMs()); + } + + if (producerProperties.getRetryBackoffMs() > 0) { + properties.put(RETRY_BACKOFF_MS, producerProperties.getRetryBackoffMs()); + } + + + LOG.info("Producer properties"); + properties.entrySet().forEach(prop -> { + LOG.info("property " + prop.getKey() + " value" + prop.getValue()); + }); + + LOG.info("Producer properties ends"); + } + +} diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaSinkConfigTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaSinkConfigTest.java new file mode 100644 index 0000000000..de3308a1f0 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaSinkConfigTest.java @@ -0,0 +1,106 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.configuration; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.model.configuration.PluginSetting; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasItem; +import static org.hamcrest.Matchers.notNullValue; +import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.mock; + + +class KafkaSinkConfigTest { + + KafkaSinkConfig kafkaSinkConfig; + + List bootstrapServers; + + @BeforeEach + void setUp() { + kafkaSinkConfig = new KafkaSinkConfig(); + kafkaSinkConfig.setBootStrapServers(Arrays.asList("127.0.0.1:9093")); + kafkaSinkConfig.setAuthConfig(mock(AuthConfig.class)); + kafkaSinkConfig.setTopic(mock(TopicConfig.class)); + kafkaSinkConfig.setSchemaConfig((mock(SchemaConfig.class))); + kafkaSinkConfig.setThreadWaitTime(10L); + // kafkaSinkConfig.setSerdeFormat("JSON"); + + } + + @Test + void test_kafka_config_not_null() { + assertThat(kafkaSinkConfig, notNullValue()); + } + + @Test + void test_bootStrapServers_not_null() { + assertThat(kafkaSinkConfig.getBootStrapServers(), notNullValue()); + List servers = kafkaSinkConfig.getBootStrapServers(); + bootstrapServers = servers.stream(). + flatMap(str -> Arrays.stream(str.split(","))). + map(String::trim). + collect(Collectors.toList()); + assertThat(bootstrapServers.size(), equalTo(1)); + assertThat(bootstrapServers, hasItem("127.0.0.1:9093")); + } + + @Test + void test_topics_not_null() { + assertThat(kafkaSinkConfig.getTopic(), notNullValue()); + } + + @Test + void test_schema_not_null() { + assertThat(kafkaSinkConfig.getSchemaConfig(), notNullValue()); + } + + @Test + void test_authentication_not_null() { + assertThat(kafkaSinkConfig.getAuthConfig(), notNullValue()); + } + + @Test + void test_serde_format_not_null() { + assertThat(kafkaSinkConfig.getSerdeFormat(), notNullValue()); + } + + @Test + void test_thread_wait_time_null() { + assertThat(kafkaSinkConfig.getThreadWaitTime(), notNullValue()); + } + + @Test + public void testDLQConfiguration() { + final Map fakePlugin = new LinkedHashMap<>(); + final Map lowLevelPluginSettings = new HashMap<>(); + lowLevelPluginSettings.put("field1", "value1"); + lowLevelPluginSettings.put("field2", "value2"); + fakePlugin.put("another_dlq", lowLevelPluginSettings); + kafkaSinkConfig.setDlqConfig(generatePluginSetting(fakePlugin)); + assertEquals("another_dlq", kafkaSinkConfig.getDlq().get().getPluginName()); + } + + private PluginSetting generatePluginSetting(final Map pluginSettings) { + final Map metadata = new HashMap<>(); + if (pluginSettings != null) { + metadata.put(KafkaSinkConfig.DLQ, pluginSettings); + } + + return new PluginSetting("S3", metadata); + } +} diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/producer/KafkaSinkProducerTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/producer/KafkaSinkProducerTest.java new file mode 100644 index 0000000000..fe06847b61 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/producer/KafkaSinkProducerTest.java @@ -0,0 +1,148 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.producer; + +import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient; +import io.confluent.kafka.schemaregistry.client.SchemaMetadata; +import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.kafka.clients.producer.MockProducer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.connect.json.JsonSerializer; +import org.junit.jupiter.api.Assertions; +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.junit.jupiter.MockitoExtension; +import org.mockito.junit.jupiter.MockitoSettings; +import org.mockito.quality.Strictness; +import org.opensearch.dataprepper.expression.ExpressionEvaluator; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSinkConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.SchemaConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; +import org.opensearch.dataprepper.plugins.kafka.sink.DLQSink; + +import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.UUID; +import java.util.concurrent.ExecutionException; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + + +@ExtendWith(MockitoExtension.class) +@MockitoSettings(strictness = Strictness.LENIENT) + +public class KafkaSinkProducerTest { + + private KafkaSinkProducer producer; + + @Mock + private KafkaSinkConfig kafkaSinkConfig; + + private Record record; + + KafkaSinkProducer sinkProducer; + + @Mock + private DLQSink dlqSink; + + private Event event; + + @Mock + private CachedSchemaRegistryClient cachedSchemaRegistryClient; + + @BeforeEach + public void setUp() { + event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + record = new Record<>(event); + final TopicConfig topicConfig = new TopicConfig(); + topicConfig.setName("test-topic"); + + when(kafkaSinkConfig.getTopic()).thenReturn(topicConfig); + when(kafkaSinkConfig.getSchemaConfig()).thenReturn(mock(SchemaConfig.class)); + when(kafkaSinkConfig.getSchemaConfig().getRegistryURL()).thenReturn("http://localhost:8085/"); + when(kafkaSinkConfig.getPartitionKey()).thenReturn("testkey"); + + } + + @Test + public void producePlainTextRecordsTest() throws ExecutionException, InterruptedException { + when(kafkaSinkConfig.getSerdeFormat()).thenReturn("plaintext"); + MockProducer mockProducer = new MockProducer<>(true, new StringSerializer(), new StringSerializer()); + producer = new KafkaSinkProducer(mockProducer, kafkaSinkConfig, dlqSink, cachedSchemaRegistryClient, mock(ExpressionEvaluator.class)); + sinkProducer = spy(producer); + sinkProducer.produceRecords(record); + verify(sinkProducer).produceRecords(record); + assertEquals(1, mockProducer.history().size()); + + } + + @Test + public void produceJsonRecordsTest() throws RestClientException, IOException { + when(kafkaSinkConfig.getSerdeFormat()).thenReturn("json"); + MockProducer mockProducer = new MockProducer<>(true, new StringSerializer(), new JsonSerializer()); + producer = new KafkaSinkProducer(mockProducer, kafkaSinkConfig, dlqSink, cachedSchemaRegistryClient, mock(ExpressionEvaluator.class)); + SchemaMetadata schemaMetadata = mock(SchemaMetadata.class); + String jsonSchema = "{\n" + + " \"$schema\": \"http://json-schema.org/draft-07/schema#\",\n" + + " \"type\": \"object\",\n" + + " \"properties\": {\n" + + " \"message\": {\n" + + " \"type\": \"string\"\n" + + " }\n" + + "}\n" + + "}\n"; + + when(schemaMetadata.getSchema()).thenReturn(jsonSchema); + when(cachedSchemaRegistryClient.getLatestSchemaMetadata(kafkaSinkConfig.getTopic().getName() + "-value")).thenReturn(schemaMetadata); + sinkProducer = spy(producer); + sinkProducer.produceRecords(record); + verify(sinkProducer).produceRecords(record); + assertEquals(1, mockProducer.history().size()); + } + + @Test + public void produceAvroRecordsTest() throws Exception { + when(kafkaSinkConfig.getSerdeFormat()).thenReturn("avro"); + MockProducer mockProducer = new MockProducer<>(true, new StringSerializer(), new StringSerializer()); + producer = new KafkaSinkProducer(mockProducer, kafkaSinkConfig, dlqSink, cachedSchemaRegistryClient, mock(ExpressionEvaluator.class)); + SchemaMetadata schemaMetadata = mock(SchemaMetadata.class); + String avroSchema = "{\"type\":\"record\",\"name\":\"MyMessage\",\"fields\":[{\"name\":\"message\",\"type\":\"string\"}]}"; + when(schemaMetadata.getSchema()).thenReturn(avroSchema); + when(cachedSchemaRegistryClient.getLatestSchemaMetadata(kafkaSinkConfig.getTopic().getName() + "-value")).thenReturn(schemaMetadata); + sinkProducer = spy(producer); + sinkProducer.produceRecords(record); + verify(sinkProducer).produceRecords(record); + + } + + @Test + public void testGetGenericRecord() throws NoSuchMethodException, InvocationTargetException, IllegalAccessException { + producer = new KafkaSinkProducer(new MockProducer(), kafkaSinkConfig, dlqSink, cachedSchemaRegistryClient, mock(ExpressionEvaluator.class)); + final Schema schema = createMockSchema(); + Method privateMethod = KafkaSinkProducer.class.getDeclaredMethod("getGenericRecord", Event.class, Schema.class); + privateMethod.setAccessible(true); + GenericRecord result = (GenericRecord) privateMethod.invoke(producer, event, schema); + Assertions.assertNotNull(result); + } + + private Schema createMockSchema() { + String schemaDefinition = "{\"type\":\"record\",\"name\":\"MyRecord\",\"fields\":[{\"name\":\"message\",\"type\":\"string\"}]}"; + return new Schema.Parser().parse(schemaDefinition); + } +} + diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/producer/ProducerWorkerTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/producer/ProducerWorkerTest.java new file mode 100644 index 0000000000..65492af774 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/producer/ProducerWorkerTest.java @@ -0,0 +1,50 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + + +package org.opensearch.dataprepper.plugins.kafka.producer; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.record.Record; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; + + +class ProducerWorkerTest { + + @Mock + ProducerWorker producerWorker; + + + private Record record; + + + @BeforeEach + public void setUp() { + Event event = JacksonEvent.fromMessage("Testing multithreaded producer"); + record = new Record<>(event); + } + + + private ProducerWorker createObjectUnderTest() { + return new ProducerWorker(mock(KafkaSinkProducer.class), record); + } + + @Test + void testWritingToTopic() { + producerWorker = createObjectUnderTest(); + Thread spySink = spy(new Thread(producerWorker)); + spySink.start(); + verify(spySink).start(); + } + + +} \ No newline at end of file diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/sink/DLQSinkTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/sink/DLQSinkTest.java new file mode 100644 index 0000000000..09f697cfda --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/sink/DLQSinkTest.java @@ -0,0 +1,96 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.sink; + + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.plugins.dlq.DlqProvider; +import org.opensearch.dataprepper.plugins.dlq.DlqWriter; +import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSinkConfig; +import org.springframework.test.util.ReflectionTestUtils; +import org.yaml.snakeyaml.Yaml; + +import java.io.FileReader; +import java.io.IOException; +import java.io.Reader; +import java.io.StringReader; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyList; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + + +public class DLQSinkTest { + + @Mock + private PluginFactory pluginFactory; + + + private KafkaSinkConfig kafkaSinkConfig; + + @Mock + private DlqProvider dlqProvider; + + @Mock + private DlqWriter dlqWriter; + + + private PluginSetting pluginSetting; + + private DLQSink dlqSink; + + @BeforeEach + public void setUp() throws IOException { + MockitoAnnotations.initMocks(this); + when(pluginFactory.loadPlugin(any(Class.class), any(PluginSetting.class))).thenReturn(dlqProvider); + when(dlqProvider.getDlqWriter(anyString())).thenReturn(Optional.of(dlqWriter)); + pluginSetting = new PluginSetting("kafka-sink", new HashMap<>()); + + + Yaml yaml = new Yaml(); + FileReader fileReader = new FileReader(getClass().getClassLoader().getResource("sample-pipelines-sink.yaml").getFile()); + Object data = yaml.load(fileReader); + if (data instanceof Map) { + Map propertyMap = (Map) data; + Map logPipelineMap = (Map) propertyMap.get("log-pipeline"); + Map sinkeMap = (Map) logPipelineMap.get("sink"); + Map kafkaConfigMap = (Map) sinkeMap.get("kafka"); + ObjectMapper mapper = new ObjectMapper(); + mapper.registerModule(new JavaTimeModule()); + String json = mapper.writeValueAsString(kafkaConfigMap); + Reader reader = new StringReader(json); + kafkaSinkConfig = mapper.readValue(reader, KafkaSinkConfig.class); + + + } + dlqSink = new DLQSink(pluginFactory, kafkaSinkConfig, pluginSetting); + } + + @Test + public void testPerform() throws IOException { + Object failedData = new Object(); + ReflectionTestUtils.setField(pluginSetting, "pipelineName", "test"); + doNothing().when(dlqWriter).write(anyList(), anyString(), anyString()); + dlqSink.perform(failedData, mock(Exception.class)); + verify(dlqWriter).write(anyList(), anyString(), anyString()); + } + + +} diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkasinkTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkasinkTest.java new file mode 100644 index 0000000000..9ccdc03046 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkasinkTest.java @@ -0,0 +1,186 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + + +package org.opensearch.dataprepper.plugins.kafka.sink; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; +import org.apache.kafka.common.serialization.StringSerializer; +import org.junit.jupiter.api.AfterEach; +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.junit.jupiter.MockitoSettings; +import org.mockito.quality.Strictness; +import org.opensearch.dataprepper.expression.ExpressionEvaluator; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSinkConfig; +import org.opensearch.dataprepper.plugins.kafka.producer.ProducerWorker; +import org.springframework.test.util.ReflectionTestUtils; +import org.yaml.snakeyaml.Yaml; + +import java.io.FileReader; +import java.io.Reader; +import java.io.StringReader; +import java.util.Arrays; +import java.util.Collection; +import java.util.Map; +import java.util.Properties; +import java.util.UUID; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.FutureTask; +import java.util.concurrent.TimeUnit; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + + +@ExtendWith(MockitoExtension.class) +@MockitoSettings(strictness = Strictness.LENIENT) +public class KafkasinkTest { + + + KafkaSink kafkaSink; + + + KafkaSinkConfig kafkaSinkConfig; + + + ExecutorService executorService; + + @Mock + PluginSetting pluginSetting; + + @Mock + FutureTask futureTask; + + + Event event; + + KafkaSink spySink; + + private static final Integer totalWorkers = 1; + + MockedStatic executorsMockedStatic; + + @Mock + private PluginFactory pluginFactoryMock; + + Properties props; + + + @BeforeEach + void setUp() throws Exception { + Yaml yaml = new Yaml(); + FileReader fileReader = new FileReader(getClass().getClassLoader().getResource("sample-pipelines-sink.yaml").getFile()); + Object data = yaml.load(fileReader); + if (data instanceof Map) { + Map propertyMap = (Map) data; + Map logPipelineMap = (Map) propertyMap.get("log-pipeline"); + Map sinkeMap = (Map) logPipelineMap.get("sink"); + Map kafkaConfigMap = (Map) sinkeMap.get("kafka"); + ObjectMapper mapper = new ObjectMapper(); + mapper.registerModule(new JavaTimeModule()); + String json = mapper.writeValueAsString(kafkaConfigMap); + Reader reader = new StringReader(json); + kafkaSinkConfig = mapper.readValue(reader, KafkaSinkConfig.class); + } + executorService = mock(ExecutorService.class); + when(pluginSetting.getPipelineName()).thenReturn("Kafka-sink"); + event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + kafkaSink = new KafkaSink(pluginSetting, kafkaSinkConfig, pluginFactoryMock, mock(ExpressionEvaluator.class)); + spySink = spy(kafkaSink); + executorsMockedStatic = mockStatic(Executors.class); + props = new Properties(); + props.put("bootstrap.servers", "127.0.0.1:9093"); + props.put("key.serializer", StringSerializer.class.getName()); + props.put("value.serializer", StringSerializer.class.getName()); + ReflectionTestUtils.setField(spySink, "executorService", executorService); + + + } + + @AfterEach + public void after() { + executorsMockedStatic.close(); + } + + @Test + public void doOutputTest() { + ReflectionTestUtils.setField(kafkaSinkConfig, "schemaConfig", null); + when(executorService.submit(any(ProducerWorker.class))).thenReturn(futureTask); + final Collection records = Arrays.asList(new Record(event)); + spySink.doOutput(records); + verify(spySink).doOutput(records); + } + + + @Test + public void doOutputExceptionTest() { + final Collection records = Arrays.asList(new Record(event)); + when(executorService.submit(any(ProducerWorker.class))).thenThrow(new RuntimeException()); + assertThrows(RuntimeException.class, () -> spySink.doOutput(records)); + } + + @Test + public void doOutputEmptyRecordsTest() { + final Collection records = Arrays.asList(); + spySink.doOutput(records); + verify(spySink).doOutput(records); + + } + + @Test + public void shutdownTest() { + spySink.shutdown(); + verify(spySink).shutdown(); + } + + @Test + public void shutdownExceptionTest() throws InterruptedException { + final InterruptedException interruptedException = new InterruptedException(); + interruptedException.initCause(new InterruptedException()); + + when(executorService.awaitTermination( + 1000L, TimeUnit.MILLISECONDS)).thenThrow(interruptedException); + spySink.shutdown(); + + } + + + @Test + public void doInitializeTest() { + spySink.doInitialize(); + verify(spySink).doInitialize(); + } + + @Test + public void doInitializeNullPointerExceptionTest() { + when(Executors.newFixedThreadPool(totalWorkers)).thenThrow(NullPointerException.class); + assertThrows(NullPointerException.class, () -> spySink.doInitialize()); + } + + + @Test + public void isReadyTest() { + ReflectionTestUtils.setField(kafkaSink, "sinkInitialized", true); + assertEquals(true, kafkaSink.isReady()); + } +} diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/util/AuthenticationPropertyConfigurerTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/util/AuthenticationPropertyConfigurerTest.java new file mode 100644 index 0000000000..666a992007 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/util/AuthenticationPropertyConfigurerTest.java @@ -0,0 +1,75 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.util; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSinkConfig; +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; + +@ExtendWith(MockitoExtension.class) +public class AuthenticationPropertyConfigurerTest { + + + KafkaSinkConfig kafkaSinkConfig; + + + private KafkaSinkConfig createKafkaSinkConfig(String fileName) throws IOException { + Yaml yaml = new Yaml(); + FileReader fileReader = new FileReader(getClass().getClassLoader().getResource(fileName).getFile()); + Object data = yaml.load(fileReader); + if (data instanceof Map) { + Map propertyMap = (Map) data; + Map logPipelineMap = (Map) propertyMap.get("log-pipeline"); + Map sinkeMap = (Map) logPipelineMap.get("sink"); + Map kafkaConfigMap = (Map) sinkeMap.get("kafka"); + ObjectMapper mapper = new ObjectMapper(); + mapper.registerModule(new JavaTimeModule()); + String json = mapper.writeValueAsString(kafkaConfigMap); + Reader reader = new StringReader(json); + return mapper.readValue(reader, KafkaSinkConfig.class); + } + return null; + + } + + @Test + public void testSetSaslPlainTextProperties() throws IOException { + Properties props = new Properties(); + kafkaSinkConfig = createKafkaSinkConfig("sample-pipelines-sink.yaml"); + AuthenticationPropertyConfigurer.setSaslPlainTextProperties(kafkaSinkConfig.getAuthConfig().getSaslAuthConfig(). + getPlainTextAuthConfig(), props); + Assertions.assertEquals("PLAIN", props.getProperty("sasl.mechanism")); + } + + @Test + public void testSetSaslOauthProperties() throws IOException { + Properties props = new Properties(); + kafkaSinkConfig = createKafkaSinkConfig("sample-pipelines-sink-oauth.yaml"); + AuthenticationPropertyConfigurer.setOauthProperties(kafkaSinkConfig.getAuthConfig().getSaslAuthConfig() + .getOAuthConfig(), props); + Assertions.assertEquals("OAUTHBEARER", props.getProperty("sasl.mechanism")); + } + + @Test + public void testSetSaslSslProperties() throws IOException { + Properties props = new Properties(); + kafkaSinkConfig = createKafkaSinkConfig("sample-pipelines-sink-ssl.yaml"); + AuthenticationPropertyConfigurer.setSaslPlainProperties(kafkaSinkConfig.getAuthConfig().getSaslAuthConfig().getPlain(), props); + Assertions.assertEquals("PLAIN", props.getProperty("sasl.mechanism")); + } +} diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/util/SinkPropertyConfigurerTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/util/SinkPropertyConfigurerTest.java new file mode 100644 index 0000000000..4319085f97 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/util/SinkPropertyConfigurerTest.java @@ -0,0 +1,92 @@ +package org.opensearch.dataprepper.plugins.kafka.util; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.MockedStatic; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSinkConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.SchemaConfig; +import org.springframework.test.util.ReflectionTestUtils; +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.mockito.Mockito.mockStatic; + +@ExtendWith(MockitoExtension.class) +public class SinkPropertyConfigurerTest { + + + KafkaSinkConfig kafkaSinkConfig; + + MockedStatic authenticationPropertyConfigurerMockedStatic; + + + @BeforeEach + public void setUp() throws IOException { + + Yaml yaml = new Yaml(); + FileReader fileReader = new FileReader(getClass().getClassLoader().getResource("sample-pipelines-sink.yaml").getFile()); + Object data = yaml.load(fileReader); + if (data instanceof Map) { + Map propertyMap = (Map) data; + Map logPipelineMap = (Map) propertyMap.get("log-pipeline"); + Map sinkeMap = (Map) logPipelineMap.get("sink"); + Map kafkaConfigMap = (Map) sinkeMap.get("kafka"); + ObjectMapper mapper = new ObjectMapper(); + mapper.registerModule(new JavaTimeModule()); + String json = mapper.writeValueAsString(kafkaConfigMap); + Reader reader = new StringReader(json); + kafkaSinkConfig = mapper.readValue(reader, KafkaSinkConfig.class); + } + + authenticationPropertyConfigurerMockedStatic = mockStatic(AuthenticationPropertyConfigurer.class); + } + + @AfterEach + public void postProcess() { + authenticationPropertyConfigurerMockedStatic.close(); + } + + @Test + public void testGetProducerPropertiesForJson() { + ReflectionTestUtils.setField(kafkaSinkConfig, "schemaConfig", null); + Properties props = SinkPropertyConfigurer.getProducerProperties(kafkaSinkConfig); + Assertions.assertEquals("30000", props.getProperty("session.timeout.ms")); + } + + @Test + public void testGetProducerPropertiesForPlainText() { + ReflectionTestUtils.setField(kafkaSinkConfig, "serdeFormat", "plaintext"); + Assertions.assertThrows(RuntimeException.class, () -> SinkPropertyConfigurer.getProducerProperties(kafkaSinkConfig)); + + } + + @Test + public void testGetProducerPropertiesForAvro() { + ReflectionTestUtils.setField(kafkaSinkConfig, "serdeFormat", "avro"); + SchemaConfig schemaConfig = kafkaSinkConfig.getSchemaConfig(); + ReflectionTestUtils.setField(kafkaSinkConfig, "schemaConfig", null); + // Assertions.assertThrows(RuntimeException.class, () -> SinkPropertyConfigurer.getProducerProperties(kafkaSinkConfig)); + ReflectionTestUtils.setField(kafkaSinkConfig, "schemaConfig", schemaConfig); + Assertions.assertEquals("30000", SinkPropertyConfigurer.getProducerProperties(kafkaSinkConfig).getProperty("session.timeout.ms")); + + } + + @Test + public void testGetProducerPropertiesForNoSerde() { + ReflectionTestUtils.setField(kafkaSinkConfig, "serdeFormat", null); + Assertions.assertThrows(RuntimeException.class, () -> SinkPropertyConfigurer.getProducerProperties(kafkaSinkConfig)); + } + +} diff --git a/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-sink-oauth.yaml b/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-sink-oauth.yaml new file mode 100644 index 0000000000..c436697e96 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-sink-oauth.yaml @@ -0,0 +1,34 @@ +log-pipeline : + source : + random: + sink : + kafka: + bootstrap_servers: + - "http://localhost:9092" + thread_wait_time: 1000 + dlq: + s3: + bucket: "mydlqtestbucket" + key_path_prefix: "dlq-files/" + sts_role_arn: "arn:aws:iam::045129910014:role/dataprepper" + region: "ap-south-1" + topic: + name: plaintext + authentication: + sasl: + oauth: + oauth_client_id: XXXXXXXXXXXX + oauth_client_secret: XXXXXXXXXXXX + oauth_login_server: https://dev-XXXXXXXXXXXX.okta.com + oauth_login_endpoint: /oauth2/default/v1/token + oauth_login_grant_type: refresh_token + oauth_login_scope: kafka + oauth_introspect_server: https://dev-XXXXXXXXXXXX.okta.com + oauth_introspect_endpoint: /oauth2/default/v1/introspect + oauth_token_endpoint_url: https://dev-XXXXXXXXXXXX.okta.com/oauth2/default/v1/token + oauth_sasl_mechanism: OAUTHBEARER + oauth_security_protocol: SASL_SSL + oauth_sasl_login_callback_handler_class: org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler + oauth_jwks_endpoint_url: https://dev-XXXXXXXXXXXX.okta.com/oauth2/default/v1/keys + extension_logicalCluster: lkc-XXXXXXXXXXXX + extension_identityPoolId: pool-XXXXXXXXXXXX \ No newline at end of file diff --git a/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-sink-ssl.yaml b/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-sink-ssl.yaml new file mode 100644 index 0000000000..1574472afc --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-sink-ssl.yaml @@ -0,0 +1,56 @@ +log-pipeline : + source : + random: + sink : + kafka: + bootstrap_servers: + - "http://localhost:9092" + thread_wait_time: 1000 + dlq: + s3: + bucket: "mydlqtestbucket" + key_path_prefix: "dlq-files/" + sts_role_arn: "arn:aws:iam::xxxxx:role/dataprepper" + region: "ap-south-1" + + serde_format: plaintext + topic: + name: ssl_test_topic + producer_properties: + buffer_memory: 10mb + compression_type: gzip + retries: 3 + batch_size: 16384 + client_dns_lookup: use_all_dns_ips + connections_max_idle: PT2M + delivery_timeout: PT2M + linger_ms: 0 + max_block: PT2M + max_request_size: 1048576 + partitioner_class: org.apache.kafka.clients.producer.internals.DefaultPartitioner + partitioner_ignore_keys: false + receive_buffer: 3mb + request_timeout: PT2M + send_buffer: 2mb + socket_connection_setup_timeout_max: PT2M + socket_connection_setup_timeout: PT2M + acks: all + enable_idempotence: true + max_in_flight_requests_per_connection: 5 + metadata_max_age: PT2M + metadata_max_idle: PT2M + metrics_num_samples: 2 + metrics_recording_level: INFO + metrics_sample_window: PT2M + partitioner_adaptive_partitioning_enable: true + partitioner_availability_timeout: PT2M + reconnect_backoff_max: PT2M + reconnect_backoff: PT2M + retry_backoff: PT2M + authentication: + sasl: + ssl_endpoint_identification_algorithm: https + plain_config: + username: 2BRMRAPEMGRJ25AY + password: C6vLQcD0bmTcDBkzvkUMxPFEsJB2XocSg8aJaLOZQT3PN28WrbR8a0+bRRjYyyei + diff --git a/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-sink.yaml b/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-sink.yaml new file mode 100644 index 0000000000..7e8b540061 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-sink.yaml @@ -0,0 +1,57 @@ +log-pipeline : + source : + random: + sink : + kafka: + bootstrap_servers: + - "localhost:9092" + thread_wait_time: 1000 + dlq: + s3: + bucket: "mydlqtestbucket" + key_path_prefix: "dlq-files/" + sts_role_arn: "arn:aws:iam::XXXXXXXXX:role/dataprepper" + region: "ap-south-1" + serde_format: plaintext + topic: + name: plaintext_test_topic + producer_properties: + buffer_memory: 10mb + compression_type: gzip + retries: 3 + batch_size: 16384 + client_dns_lookup: use_all_dns_ips + connections_max_idle: PT2M + delivery_timeout: PT2M + linger_ms: 0 + max_block: PT2M + max_request_size: 1048576 + partitioner_class: org.apache.kafka.clients.producer.internals.DefaultPartitioner + partitioner_ignore_keys: false + receive_buffer: 3mb + request_timeout: PT2M + send_buffer: 2mb + socket_connection_setup_timeout_max: PT2M + socket_connection_setup_timeout: PT2M + acks: all + enable_idempotence: true + max_in_flight_requests_per_connection: 5 + metadata_max_age: PT2M + metadata_max_idle: PT2M + metrics_num_samples: 2 + metrics_recording_level: INFO + metrics_sample_window: PT2M + partitioner_adaptive_partitioning_enable: true + partitioner_availability_timeout: PT2M + reconnect_backoff_max: PT2M + reconnect_backoff: PT2M + retry_backoff: PT2M + schema: + registry_url: https://localhost:8085 + authentication: + sasl: + plaintext: + username: username + password: password + + From 5e3302552577520bbbd5981efc7f4fe17bd77dcd Mon Sep 17 00:00:00 2001 From: rajeshLovesToCode Date: Fri, 28 Jul 2023 16:49:41 +0530 Subject: [PATCH 2/9] -Support for kafka-sink Signed-off-by: rajeshLovesToCode --- .../kafka/configuration/AuthConfig.java | 8 +---- .../kafka/configuration/PlainConfig.java | 26 --------------- .../kafka/producer/KafkaSinkProducer.java | 33 ++++++++++++++++--- .../plugins/kafka/sink/KafkaSink.java | 10 ++++-- .../AuthenticationPropertyConfigurer.java | 7 ++-- .../kafka/util/SinkPropertyConfigurer.java | 20 ++++++++--- .../kafka/producer/KafkaSinkProducerTest.java | 10 +++--- .../plugins/kafka/sink/KafkasinkTest.java | 7 +++- .../AuthenticationPropertyConfigurerTest.java | 2 +- .../resources/sample-pipelines-sink-ssl.yaml | 2 +- 10 files changed, 70 insertions(+), 55 deletions(-) delete mode 100644 data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/PlainConfig.java diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/AuthConfig.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/AuthConfig.java index a3fe1163d8..0820fe46f7 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/AuthConfig.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/AuthConfig.java @@ -30,9 +30,6 @@ public static class SaslAuthConfig { @JsonProperty("ssl_endpoint_identification_algorithm") private String sslEndpointAlgorithm; - @JsonProperty("plain_config") - private PlainConfig plain; - public AwsIamAuthConfig getAwsIamAuthConfig() { return awsIamAuthConfig; } @@ -45,16 +42,13 @@ public OAuthConfig getOAuthConfig() { return oAuthConfig; } - public PlainConfig getPlain() { - return plain; - } public String getSslEndpointAlgorithm() { return sslEndpointAlgorithm; } @AssertTrue(message = "Only one of AwsIam or oAuth or PlainText auth config must be specified") public boolean hasOnlyOneConfig() { - return Stream.of(awsIamAuthConfig, plainTextAuthConfig, oAuthConfig, plain).filter(n -> n != null).count() == 1; + return Stream.of(awsIamAuthConfig, plainTextAuthConfig, oAuthConfig).filter(n -> n != null).count() == 1; } } diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/PlainConfig.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/PlainConfig.java deleted file mode 100644 index 01a79c6447..0000000000 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/PlainConfig.java +++ /dev/null @@ -1,26 +0,0 @@ -package org.opensearch.dataprepper.plugins.kafka.configuration; - -import com.fasterxml.jackson.annotation.JsonProperty; -import jakarta.validation.constraints.NotNull; - -public class PlainConfig { - @JsonProperty("username") - @NotNull - private String userName; - - @JsonProperty("password") - @NotNull - private String password; - - public String getUserName() { - return userName; - } - - public String getPassword() { - return password; - } - - public PlainConfig() { - - } -} diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/producer/KafkaSinkProducer.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/producer/KafkaSinkProducer.java index a416c86076..2bd87b9200 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/producer/KafkaSinkProducer.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/producer/KafkaSinkProducer.java @@ -5,6 +5,8 @@ package org.opensearch.dataprepper.plugins.kafka.producer; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.github.fge.jsonschema.core.exceptions.ProcessingException; @@ -22,6 +24,7 @@ import org.opensearch.dataprepper.expression.ExpressionEvaluator; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventHandle; +import org.opensearch.dataprepper.model.log.JacksonLog; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSinkConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; @@ -33,6 +36,7 @@ import java.io.IOException; import java.util.Collection; import java.util.LinkedList; +import java.util.Map; /** @@ -54,19 +58,25 @@ public class KafkaSinkProducer { private final Collection bufferedEventHandles; - final ExpressionEvaluator expressionEvaluator; + private final ExpressionEvaluator expressionEvaluator; + + private final String tagTargetKey; + + private final ObjectMapper objectMapper = new ObjectMapper(); public KafkaSinkProducer(final Producer producer, final KafkaSinkConfig kafkaSinkConfig, final DLQSink dlqSink, final CachedSchemaRegistryClient schemaRegistryClient, - final ExpressionEvaluator expressionEvaluator) { + final ExpressionEvaluator expressionEvaluator, + final String tagTargetKey) { this.producer = producer; this.kafkaSinkConfig = kafkaSinkConfig; this.dlqSink = dlqSink; this.schemaRegistryClient = schemaRegistryClient; bufferedEventHandles = new LinkedList<>(); this.expressionEvaluator = expressionEvaluator; + this.tagTargetKey = tagTargetKey; } @@ -75,7 +85,7 @@ public void produceRecords(final Record record) { bufferedEventHandles.add(record.getData().getEventHandle()); } TopicConfig topic = kafkaSinkConfig.getTopic(); - final Event event = record.getData(); + Event event = getEvent(record); final String key = event.formatString(kafkaSinkConfig.getPartitionKey(), expressionEvaluator); Object dataForDlq = event.toJsonString(); LOG.info("Producing record " + dataForDlq); @@ -92,7 +102,16 @@ public void produceRecords(final Record record) { releaseEventHandles(false); } + } + private Event getEvent(Record record) { + Event event = record.getData(); + try { + event = addTagsToEvent(event, tagTargetKey); + } catch (JsonProcessingException e) { + LOG.error("error occured while processing tag target key"); + } + return event; } private void publishPlaintextMessage(Record record, TopicConfig topic, String key, Object dataForDlq) { @@ -152,7 +171,7 @@ private Callback callBack(final Object dataForDlq) { } - private GenericRecord getGenericRecord(final Event event,final Schema schema) { + private GenericRecord getGenericRecord(final Event event, final Schema schema) { final GenericRecord record = new GenericData.Record(schema); for (final String key : event.toMap().keySet()) { record.put(key, event.toMap().get(key)); @@ -167,5 +186,11 @@ private void releaseEventHandles(final boolean result) { bufferedEventHandles.clear(); } + private Event addTagsToEvent(Event event, String tagsTargetKey) throws JsonProcessingException { + String eventJsonString = event.jsonBuilder().includeTags(tagsTargetKey).toJsonString(); + Map eventData = objectMapper.readValue(eventJsonString, new TypeReference<>() { + }); + return JacksonLog.builder().withData(eventData).build(); + } } diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkaSink.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkaSink.java index 00365236ad..4a530e2794 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkaSink.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkaSink.java @@ -17,6 +17,7 @@ import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.sink.AbstractSink; import org.opensearch.dataprepper.model.sink.Sink; +import org.opensearch.dataprepper.model.sink.SinkContext; import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSinkConfig; import org.opensearch.dataprepper.plugins.kafka.producer.KafkaSinkProducer; import org.opensearch.dataprepper.plugins.kafka.producer.ProducerWorker; @@ -57,15 +58,19 @@ public class KafkaSink extends AbstractSink> { private final ExpressionEvaluator expressionEvaluator; + private final SinkContext sinkContext; + @DataPrepperPluginConstructor public KafkaSink(final PluginSetting pluginSetting, final KafkaSinkConfig kafkaSinkConfig, final PluginFactory pluginFactory, - final ExpressionEvaluator expressionEvaluator) { + final ExpressionEvaluator expressionEvaluator, final SinkContext sinkContext) { super(pluginSetting); this.pluginSetting = pluginSetting; this.kafkaSinkConfig = kafkaSinkConfig; this.pluginFactory = pluginFactory; this.expressionEvaluator = expressionEvaluator; + this.sinkContext = sinkContext; + } @@ -112,7 +117,8 @@ public KafkaSinkProducer createProducer() { Properties properties = SinkPropertyConfigurer.getProducerProperties(kafkaSinkConfig); properties = Objects.requireNonNull(properties); return new KafkaSinkProducer(new KafkaProducer<>(properties), - kafkaSinkConfig, new DLQSink(pluginFactory, kafkaSinkConfig, pluginSetting), getSchemaRegistryClient(), expressionEvaluator); + kafkaSinkConfig, new DLQSink(pluginFactory, kafkaSinkConfig, pluginSetting), getSchemaRegistryClient(), expressionEvaluator + , Objects.nonNull(sinkContext) ? sinkContext.getTagsTargetKey() : null); } private CachedSchemaRegistryClient getSchemaRegistryClient() { diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/AuthenticationPropertyConfigurer.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/AuthenticationPropertyConfigurer.java index 53ae709c81..97b9951b97 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/AuthenticationPropertyConfigurer.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/AuthenticationPropertyConfigurer.java @@ -5,7 +5,6 @@ package org.opensearch.dataprepper.plugins.kafka.util; import org.opensearch.dataprepper.plugins.kafka.configuration.OAuthConfig; -import org.opensearch.dataprepper.plugins.kafka.configuration.PlainConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.PlainTextAuthConfig; import java.util.Base64; @@ -130,11 +129,11 @@ private static String getJassConfigWithClusterInforation(final String extensionL return jass_config; } - public static void setSaslPlainProperties(final PlainConfig sslAuthConfig, + public static void setSaslPlainProperties(final PlainTextAuthConfig plainTextAuthConfig, final Properties properties) { - String username = sslAuthConfig.getUserName(); - String password = sslAuthConfig.getPassword(); + String username = plainTextAuthConfig.getUsername(); + String password = plainTextAuthConfig.getPassword(); properties.put(SASL_MECHANISM, PLAIN_MECHANISM); properties.put(SASL_JAS_CONFIG, String.format(PLAINTEXT_JAASCONFIG, username, password)); properties.put(SASL_SECURITY_PROTOCOL, SASL_SSL_PROTOCOL); diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/SinkPropertyConfigurer.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/SinkPropertyConfigurer.java index 9b97f01039..8484335fce 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/SinkPropertyConfigurer.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/SinkPropertyConfigurer.java @@ -120,17 +120,27 @@ public static Properties getProducerProperties(final KafkaSinkConfig kafkaSinkCo setPropertiesProviderByKafkaProducer(kafkaSinkConfig.getKafkaProducerProperties(), properties); } + setAuthProperties(kafkaSinkConfig, properties); + + return properties; + } + + private static void setAuthProperties(final KafkaSinkConfig kafkaSinkConfig, final Properties properties) { if (kafkaSinkConfig.getAuthConfig().getSaslAuthConfig().getPlainTextAuthConfig() != null) { - AuthenticationPropertyConfigurer.setSaslPlainTextProperties(kafkaSinkConfig.getAuthConfig() - .getSaslAuthConfig().getPlainTextAuthConfig(), properties); + final String sslEndpointAlgorithm = kafkaSinkConfig.getAuthConfig().getSaslAuthConfig().getSslEndpointAlgorithm(); + if (null != sslEndpointAlgorithm && !sslEndpointAlgorithm.isEmpty() && sslEndpointAlgorithm.equalsIgnoreCase("https")) { + AuthenticationPropertyConfigurer.setSaslPlainProperties(kafkaSinkConfig.getAuthConfig().getSaslAuthConfig().getPlainTextAuthConfig(), properties); + } else { + AuthenticationPropertyConfigurer.setSaslPlainTextProperties(kafkaSinkConfig.getAuthConfig() + .getSaslAuthConfig().getPlainTextAuthConfig(), properties); + } + } else if (kafkaSinkConfig.getAuthConfig().getSaslAuthConfig().getOAuthConfig() != null) { AuthenticationPropertyConfigurer.setOauthProperties(kafkaSinkConfig.getAuthConfig().getSaslAuthConfig() .getOAuthConfig(), properties); - } else if (kafkaSinkConfig.getAuthConfig().getSaslAuthConfig().getPlain() != null) { - AuthenticationPropertyConfigurer.setSaslPlainProperties(kafkaSinkConfig.getAuthConfig().getSaslAuthConfig().getPlain(), properties); } - return properties; + } private static void setCommonServerProperties(final Properties properties, final KafkaSinkConfig kafkaSinkConfig) { diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/producer/KafkaSinkProducerTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/producer/KafkaSinkProducerTest.java index fe06847b61..1d28b5d50e 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/producer/KafkaSinkProducerTest.java +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/producer/KafkaSinkProducerTest.java @@ -65,6 +65,8 @@ public class KafkaSinkProducerTest { @Mock private CachedSchemaRegistryClient cachedSchemaRegistryClient; + final String tag = "tag"; + @BeforeEach public void setUp() { event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); @@ -83,7 +85,7 @@ record = new Record<>(event); public void producePlainTextRecordsTest() throws ExecutionException, InterruptedException { when(kafkaSinkConfig.getSerdeFormat()).thenReturn("plaintext"); MockProducer mockProducer = new MockProducer<>(true, new StringSerializer(), new StringSerializer()); - producer = new KafkaSinkProducer(mockProducer, kafkaSinkConfig, dlqSink, cachedSchemaRegistryClient, mock(ExpressionEvaluator.class)); + producer = new KafkaSinkProducer(mockProducer, kafkaSinkConfig, dlqSink, cachedSchemaRegistryClient, mock(ExpressionEvaluator.class), tag); sinkProducer = spy(producer); sinkProducer.produceRecords(record); verify(sinkProducer).produceRecords(record); @@ -95,7 +97,7 @@ public void producePlainTextRecordsTest() throws ExecutionException, Interrupted public void produceJsonRecordsTest() throws RestClientException, IOException { when(kafkaSinkConfig.getSerdeFormat()).thenReturn("json"); MockProducer mockProducer = new MockProducer<>(true, new StringSerializer(), new JsonSerializer()); - producer = new KafkaSinkProducer(mockProducer, kafkaSinkConfig, dlqSink, cachedSchemaRegistryClient, mock(ExpressionEvaluator.class)); + producer = new KafkaSinkProducer(mockProducer, kafkaSinkConfig, dlqSink, cachedSchemaRegistryClient, mock(ExpressionEvaluator.class), tag); SchemaMetadata schemaMetadata = mock(SchemaMetadata.class); String jsonSchema = "{\n" + " \"$schema\": \"http://json-schema.org/draft-07/schema#\",\n" + @@ -119,7 +121,7 @@ public void produceJsonRecordsTest() throws RestClientException, IOException { public void produceAvroRecordsTest() throws Exception { when(kafkaSinkConfig.getSerdeFormat()).thenReturn("avro"); MockProducer mockProducer = new MockProducer<>(true, new StringSerializer(), new StringSerializer()); - producer = new KafkaSinkProducer(mockProducer, kafkaSinkConfig, dlqSink, cachedSchemaRegistryClient, mock(ExpressionEvaluator.class)); + producer = new KafkaSinkProducer(mockProducer, kafkaSinkConfig, dlqSink, cachedSchemaRegistryClient, mock(ExpressionEvaluator.class), tag); SchemaMetadata schemaMetadata = mock(SchemaMetadata.class); String avroSchema = "{\"type\":\"record\",\"name\":\"MyMessage\",\"fields\":[{\"name\":\"message\",\"type\":\"string\"}]}"; when(schemaMetadata.getSchema()).thenReturn(avroSchema); @@ -132,7 +134,7 @@ public void produceAvroRecordsTest() throws Exception { @Test public void testGetGenericRecord() throws NoSuchMethodException, InvocationTargetException, IllegalAccessException { - producer = new KafkaSinkProducer(new MockProducer(), kafkaSinkConfig, dlqSink, cachedSchemaRegistryClient, mock(ExpressionEvaluator.class)); + producer = new KafkaSinkProducer(new MockProducer(), kafkaSinkConfig, dlqSink, cachedSchemaRegistryClient, mock(ExpressionEvaluator.class), tag); final Schema schema = createMockSchema(); Method privateMethod = KafkaSinkProducer.class.getDeclaredMethod("getGenericRecord", Event.class, Schema.class); privateMethod.setAccessible(true); diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkasinkTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkasinkTest.java index 9ccdc03046..4f12f11c87 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkasinkTest.java +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkasinkTest.java @@ -24,6 +24,7 @@ import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.model.plugin.PluginFactory; import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.sink.SinkContext; import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSinkConfig; import org.opensearch.dataprepper.plugins.kafka.producer.ProducerWorker; import org.springframework.test.util.ReflectionTestUtils; @@ -85,6 +86,9 @@ public class KafkasinkTest { Properties props; + @Mock + SinkContext sinkContext; + @BeforeEach void setUp() throws Exception { @@ -105,7 +109,8 @@ void setUp() throws Exception { executorService = mock(ExecutorService.class); when(pluginSetting.getPipelineName()).thenReturn("Kafka-sink"); event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); - kafkaSink = new KafkaSink(pluginSetting, kafkaSinkConfig, pluginFactoryMock, mock(ExpressionEvaluator.class)); + when(sinkContext.getTagsTargetKey()).thenReturn("tag"); + kafkaSink = new KafkaSink(pluginSetting, kafkaSinkConfig, pluginFactoryMock, mock(ExpressionEvaluator.class), sinkContext); spySink = spy(kafkaSink); executorsMockedStatic = mockStatic(Executors.class); props = new Properties(); diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/util/AuthenticationPropertyConfigurerTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/util/AuthenticationPropertyConfigurerTest.java index 666a992007..0656fe96ee 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/util/AuthenticationPropertyConfigurerTest.java +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/util/AuthenticationPropertyConfigurerTest.java @@ -69,7 +69,7 @@ public void testSetSaslOauthProperties() throws IOException { public void testSetSaslSslProperties() throws IOException { Properties props = new Properties(); kafkaSinkConfig = createKafkaSinkConfig("sample-pipelines-sink-ssl.yaml"); - AuthenticationPropertyConfigurer.setSaslPlainProperties(kafkaSinkConfig.getAuthConfig().getSaslAuthConfig().getPlain(), props); + AuthenticationPropertyConfigurer.setSaslPlainProperties(kafkaSinkConfig.getAuthConfig().getSaslAuthConfig().getPlainTextAuthConfig(), props); Assertions.assertEquals("PLAIN", props.getProperty("sasl.mechanism")); } } diff --git a/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-sink-ssl.yaml b/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-sink-ssl.yaml index 1574472afc..7e4be88737 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-sink-ssl.yaml +++ b/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-sink-ssl.yaml @@ -50,7 +50,7 @@ log-pipeline : authentication: sasl: ssl_endpoint_identification_algorithm: https - plain_config: + plaintext: username: 2BRMRAPEMGRJ25AY password: C6vLQcD0bmTcDBkzvkUMxPFEsJB2XocSg8aJaLOZQT3PN28WrbR8a0+bRRjYyyei From 2708c67dac9797264f5ec3eb05cb8fb45df9003e Mon Sep 17 00:00:00 2001 From: rajeshLovesToCode Date: Wed, 9 Aug 2023 21:39:01 +0530 Subject: [PATCH 3/9] -Support for kafka-sink Signed-off-by: rajeshLovesToCode --- .../kafka-plugins/build.gradle | 2 + .../kafka/configuration/SchemaConfig.java | 76 +++++ .../kafka/configuration/TopicConfig.java | 34 ++- .../kafka/producer/KafkaSinkProducer.java | 89 +++--- .../plugins/kafka/service/SchemaService.java | 278 ++++++++++++++++++ .../plugins/kafka/service/TopicService.java | 39 +++ .../plugins/kafka/sink/DLQSink.java | 7 +- .../plugins/kafka/sink/KafkaSink.java | 58 +++- .../AuthenticationPropertyConfigurer.java | 5 - .../plugins/kafka/util/JsonUtils.java | 46 +++ .../plugins/kafka/util/RestUtils.java | 65 ++++ .../kafka/util/SinkPropertyConfigurer.java | 32 +- .../kafka/configuration/TopicConfigTest.java | 119 ++++---- .../kafka/producer/KafkaSinkProducerTest.java | 36 ++- .../plugins/kafka/sink/KafkasinkTest.java | 21 ++ .../plugins/kafka/util/JsonUtilsTest.java | 58 ++++ .../test/resources/sample-pipelines-sink.yaml | 1 + 17 files changed, 814 insertions(+), 152 deletions(-) create mode 100644 data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/service/SchemaService.java create mode 100644 data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/service/TopicService.java create mode 100644 data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/JsonUtils.java create mode 100644 data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/RestUtils.java create mode 100644 data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/util/JsonUtilsTest.java diff --git a/data-prepper-plugins/kafka-plugins/build.gradle b/data-prepper-plugins/kafka-plugins/build.gradle index 298f11fb86..ec2a44f906 100644 --- a/data-prepper-plugins/kafka-plugins/build.gradle +++ b/data-prepper-plugins/kafka-plugins/build.gradle @@ -48,6 +48,8 @@ dependencies { implementation 'org.apache.kafka:connect-json:3.4.0' implementation 'com.github.fge:json-schema-validator:2.2.14' implementation 'commons-collections:commons-collections:3.2.2' + implementation 'software.amazon.awssdk:s3' + implementation 'software.amazon.awssdk:apache-client' } diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/SchemaConfig.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/SchemaConfig.java index ee4e8c316e..049a261e95 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/SchemaConfig.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/SchemaConfig.java @@ -6,6 +6,12 @@ package org.opensearch.dataprepper.plugins.kafka.configuration; import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.Valid; +import jakarta.validation.constraints.AssertTrue; +import jakarta.validation.constraints.NotNull; +import jakarta.validation.constraints.Size; + +import java.util.stream.Stream; /** * * A helper class that helps to read user configuration values from @@ -37,6 +43,51 @@ public class SchemaConfig { @JsonProperty("basic_auth_credentials_source") private String basicAuthCredentialsSource; + @JsonProperty("inline_schema") + private String inlineSchema; + + @JsonProperty("schema_file_location") + private String schemaFileLocation; + + @JsonProperty("s3_file_config") + private S3FileConfig s3FileConfig; + + @JsonProperty("is_create") + @NotNull + private Boolean isCreate; + + public static class S3FileConfig { + @Valid + @Size(max = 0, message = "Schema from file is not supported.") + @JsonProperty("bucket_name") + private String bucketName; + + @Valid + @Size(max = 0, message = "Schema from file is not supported.") + @JsonProperty("file_key") + private String fileKey; + + @Valid + @Size(max = 0, message = "Schema from file is not supported.") + @JsonProperty("region") + private String region; + + public String getRegion() { + return region; + } + + public S3FileConfig() { + } + + public String getBucketName() { + return bucketName; + } + + public String getFileKey() { + return fileKey; + } + } + public int getSessionTimeoutms() { return sessionTimeoutms; } @@ -64,4 +115,29 @@ public String getSchemaRegistryApiKey() { public String getSchemaRegistryApiSecret() { return schemaRegistryApiSecret; } + + + public String getInlineSchema() { + return inlineSchema; + } + + public String getSchemaFileLocation() { + return schemaFileLocation; + } + + public S3FileConfig getS3FileConfig() { + return s3FileConfig; + } + + @AssertTrue(message = "Only one of Inline schema or Schema file location or S3 file config config must be specified") + public boolean hasOnlyOneConfig() { + if(isCreate) { + return Stream.of(inlineSchema, schemaFileLocation, s3FileConfig).filter(n -> n != null).count() == 1; + } + return true; + } + + public Boolean isCreate() { + return isCreate; + } } diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfig.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfig.java index d613db3cf5..0c1ff2f892 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfig.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfig.java @@ -9,10 +9,10 @@ import jakarta.validation.Valid; import jakarta.validation.constraints.NotNull; import jakarta.validation.constraints.Size; - import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat; import java.time.Duration; + /** * * A helper class that helps to read consumer configuration values from * pipelines.yaml @@ -23,6 +23,7 @@ public class TopicConfig { static final Duration DEFAULT_SESSION_TIMEOUT = Duration.ofSeconds(45); static final int DEFAULT_MAX_RETRY_ATTEMPT = Integer.MAX_VALUE; static final String DEFAULT_AUTO_OFFSET_RESET = "latest"; + static final Duration DEFAULT_THREAD_WAITING_TIME = Duration.ofSeconds(5); static final Duration DEFAULT_MAX_RECORD_FETCH_TIME = Duration.ofSeconds(4); static final Duration DEFAULT_BUFFER_TIMEOUT = Duration.ofSeconds(5); @@ -38,6 +39,11 @@ public class TopicConfig { static final Integer DEFAULT_NUM_OF_WORKERS = 2; static final Duration DEFAULT_HEART_BEAT_INTERVAL_DURATION = Duration.ofSeconds(5); + + private static final Integer NUM_OF_PARTITIONS = 3; + private static final Short REPLICATION_FACTOR = 1; + + @JsonProperty("name") @NotNull @Valid @@ -64,7 +70,7 @@ public class TopicConfig { private Duration maxRetryDelay = DEFAULT_MAX_RETRY_DELAY; @JsonProperty("serde_format") - private MessageFormat serdeFormat= MessageFormat.PLAINTEXT; + private MessageFormat serdeFormat = MessageFormat.PLAINTEXT; @JsonProperty("auto_commit") private Boolean autoCommit = DEFAULT_AUTO_COMMIT; @@ -134,7 +140,16 @@ public class TopicConfig { @JsonProperty("heart_beat_interval") @Valid @Size(min = 1) - private Duration heartBeatInterval= DEFAULT_HEART_BEAT_INTERVAL_DURATION; + private Duration heartBeatInterval = DEFAULT_HEART_BEAT_INTERVAL_DURATION; + + @JsonProperty("is_create") + private Boolean isCreate=Boolean.FALSE; + + @JsonProperty("number_of_partitions") + private Integer numberOfPartions = NUM_OF_PARTITIONS; + + @JsonProperty("replication_factor") + private Short replicationFactor = REPLICATION_FACTOR; public String getGroupId() { return groupId; @@ -288,8 +303,21 @@ public void setName(String name) { this.name = name; } + public KafkaKeyMode getKafkaKeyMode() { return kafkaKeyMode; } + public Boolean isCreate() { + return isCreate; + } + + public Integer getNumberOfPartions() { + return numberOfPartions; + } + + public Short getReplicationFactor() { + return replicationFactor; + } + } diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/producer/KafkaSinkProducer.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/producer/KafkaSinkProducer.java index 2bd87b9200..17a177fd0f 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/producer/KafkaSinkProducer.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/producer/KafkaSinkProducer.java @@ -13,11 +13,10 @@ import com.github.fge.jsonschema.core.report.ProcessingReport; import com.github.fge.jsonschema.main.JsonSchema; import com.github.fge.jsonschema.main.JsonSchemaFactory; -import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient; -import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException; import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; +import org.apache.commons.lang3.ObjectUtils; import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; @@ -27,7 +26,7 @@ import org.opensearch.dataprepper.model.log.JacksonLog; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSinkConfig; -import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; +import org.opensearch.dataprepper.plugins.kafka.service.SchemaService; import org.opensearch.dataprepper.plugins.kafka.sink.DLQSink; import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat; import org.slf4j.Logger; @@ -54,29 +53,37 @@ public class KafkaSinkProducer { private final DLQSink dlqSink; - private final CachedSchemaRegistryClient schemaRegistryClient; - private final Collection bufferedEventHandles; private final ExpressionEvaluator expressionEvaluator; + private final ObjectMapper objectMapper = new ObjectMapper(); + private final String tagTargetKey; - private final ObjectMapper objectMapper = new ObjectMapper(); + private final String topicName; + + private final String serdeFormat; + + private final SchemaService schemaService; + public KafkaSinkProducer(final Producer producer, final KafkaSinkConfig kafkaSinkConfig, final DLQSink dlqSink, - final CachedSchemaRegistryClient schemaRegistryClient, final ExpressionEvaluator expressionEvaluator, - final String tagTargetKey) { + final String tagTargetKey + ) { this.producer = producer; this.kafkaSinkConfig = kafkaSinkConfig; this.dlqSink = dlqSink; - this.schemaRegistryClient = schemaRegistryClient; bufferedEventHandles = new LinkedList<>(); this.expressionEvaluator = expressionEvaluator; this.tagTargetKey = tagTargetKey; + this.topicName = ObjectUtils.isEmpty(kafkaSinkConfig.getTopic()) ? null : kafkaSinkConfig.getTopic().getName(); + this.serdeFormat = ObjectUtils.isEmpty(kafkaSinkConfig.getSerdeFormat()) ? null : kafkaSinkConfig.getSerdeFormat(); + schemaService = new SchemaService.SchemaServiceBuilder().getFetchSchemaService(topicName, kafkaSinkConfig.getSchemaConfig()).build(); + } @@ -84,27 +91,24 @@ public void produceRecords(final Record record) { if (record.getData().getEventHandle() != null) { bufferedEventHandles.add(record.getData().getEventHandle()); } - TopicConfig topic = kafkaSinkConfig.getTopic(); Event event = getEvent(record); final String key = event.formatString(kafkaSinkConfig.getPartitionKey(), expressionEvaluator); - Object dataForDlq = event.toJsonString(); - LOG.info("Producing record " + dataForDlq); try { - final String serdeFormat = kafkaSinkConfig.getSerdeFormat(); if (MessageFormat.JSON.toString().equalsIgnoreCase(serdeFormat)) { - publishJsonMessage(record, topic, key, dataForDlq); + publishJsonMessage(record, key); } else if (MessageFormat.AVRO.toString().equalsIgnoreCase(serdeFormat)) { - publishAvroMessage(record, topic, key, dataForDlq); + publishAvroMessage(record, key); } else { - publishPlaintextMessage(record, topic, key, dataForDlq); + publishPlaintextMessage(record, key); } } catch (Exception e) { + LOG.error("Error occured while publishing " + e.getMessage()); releaseEventHandles(false); } } - private Event getEvent(Record record) { + private Event getEvent(final Record record) { Event event = record.getData(); try { event = addTagsToEvent(event, tagTargetKey); @@ -114,54 +118,57 @@ private Event getEvent(Record record) { return event; } - private void publishPlaintextMessage(Record record, TopicConfig topic, String key, Object dataForDlq) { - producer.send(new ProducerRecord(topic.getName(), key, record.getData().toJsonString()), callBack(dataForDlq)); + + private void publishPlaintextMessage(final Record record, final String key) { + send(topicName, key, record.getData().toJsonString()); + } + + private void publishAvroMessage(final Record record, final String key) { + final Schema avroSchema = schemaService.getSchema(topicName); + if (avroSchema == null) { + throw new RuntimeException("Schema definition is mandatory in case of type avro"); + } + final GenericRecord genericRecord = getGenericRecord(record.getData(), avroSchema); + send(topicName, key, genericRecord); } - private void publishAvroMessage(Record record, TopicConfig topic, String key, Object dataForDlq) throws RestClientException, IOException { - final String valueToParse = schemaRegistryClient. - getLatestSchemaMetadata(topic.getName() + "-value").getSchema(); - final Schema schema = new Schema.Parser().parse(valueToParse); - final GenericRecord genericRecord = getGenericRecord(record.getData(), schema); - producer.send(new ProducerRecord(topic.getName(), key, genericRecord), callBack(dataForDlq)); + private void send(final String topicName, final String key, final Object record) { + producer.send(new ProducerRecord(topicName, key, record), callBack(record)); } - private void publishJsonMessage(Record record, TopicConfig topic, String key, Object dataForDlq) throws IOException, RestClientException, ProcessingException { + private void publishJsonMessage(final Record record, final String key) throws IOException, ProcessingException { final JsonNode dataNode = new ObjectMapper().convertValue(record.getData().toJsonString(), JsonNode.class); - if (validateJson(topic.getName(), dataForDlq)) { - producer.send(new ProducerRecord(topic.getName(), key, dataNode), callBack(dataForDlq)); + if (validateJson(topicName, record.getData().toJsonString())) { + send(topicName, key, dataNode); } else { - dlqSink.perform(dataForDlq, new RuntimeException("Invalid Json")); + dlqSink.perform(dataNode, new RuntimeException("Invalid Json")); } } - private Boolean validateJson(final String topicName, Object dataForDlq) throws IOException, RestClientException, ProcessingException { - if (schemaRegistryClient != null) { - final String schemaJson = schemaRegistryClient. - getLatestSchemaMetadata(topicName + "-value").getSchema(); - return validateSchema(dataForDlq.toString(), schemaJson); + private Boolean validateJson(final String topicName, final String jsonData) throws IOException, ProcessingException { + final String schemaJson = schemaService.getValueToParse(topicName); + if (schemaJson != null) { + return validateSchema(jsonData, schemaJson); + } else { return true; } } - private boolean validateSchema(final String jsonData, final String schemaJson) throws IOException, ProcessingException { + public boolean validateSchema(final String jsonData, final String schemaJson) throws IOException, ProcessingException { ObjectMapper objectMapper = new ObjectMapper(); JsonNode schemaNode = objectMapper.readTree(schemaJson); JsonNode dataNode = objectMapper.readTree(jsonData); JsonSchemaFactory schemaFactory = JsonSchemaFactory.byDefault(); JsonSchema schema = schemaFactory.getJsonSchema(schemaNode); ProcessingReport report = schema.validate(dataNode); - if (report.isSuccess()) { - return true; - } else { - return false; - } + return report != null ? report.isSuccess() : false; } private Callback callBack(final Object dataForDlq) { return (metadata, exception) -> { if (null != exception) { + LOG.error("Error occured while publishing " + exception.getMessage()); releaseEventHandles(false); dlqSink.perform(dataForDlq, exception); } else { @@ -186,7 +193,7 @@ private void releaseEventHandles(final boolean result) { bufferedEventHandles.clear(); } - private Event addTagsToEvent(Event event, String tagsTargetKey) throws JsonProcessingException { + private Event addTagsToEvent(final Event event, final String tagsTargetKey) throws JsonProcessingException { String eventJsonString = event.jsonBuilder().includeTags(tagsTargetKey).toJsonString(); Map eventData = objectMapper.readValue(eventJsonString, new TypeReference<>() { }); diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/service/SchemaService.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/service/SchemaService.java new file mode 100644 index 0000000000..a5d7883d7a --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/service/SchemaService.java @@ -0,0 +1,278 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.kafka.service; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient; +import io.confluent.kafka.schemaregistry.client.rest.entities.requests.CompatibilityCheckResponse; +import io.confluent.kafka.schemaregistry.client.rest.entities.requests.RegisterSchemaRequest; +import io.confluent.kafka.schemaregistry.client.rest.entities.requests.RegisterSchemaResponse; +import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException; +import org.apache.avro.Schema; +import org.apache.commons.lang3.ObjectUtils; +import org.jetbrains.annotations.NotNull; +import org.opensearch.dataprepper.plugins.kafka.configuration.SchemaConfig; +import org.opensearch.dataprepper.plugins.kafka.util.JsonUtils; +import org.opensearch.dataprepper.plugins.kafka.util.RestUtils; +import org.opensearch.dataprepper.plugins.kafka.util.SinkPropertyConfigurer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProviderChain; +import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; +import software.amazon.awssdk.core.ResponseInputStream; +import software.amazon.awssdk.http.apache.ApacheHttpClient; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.nio.file.Paths; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +public class SchemaService { + private static final Logger LOG = LoggerFactory.getLogger(SchemaService.class); + + private CachedSchemaRegistryClient schemaRegistryClient; + + private static final int cacheCapacity = 100; + + private static final String REGISTER_API_PATH = "/subjects/" + "%s" + "/versions?normalize=false"; + + private static final String COMPATIBILITY_API_PATH = "compatibility/subjects/" + "%s" + "/versions/" + "%s"; + + private final String schemaString; + + private final String serdeFormat; + + private final String topic; + + private final SchemaConfig schemaConfig; + + private RestUtils restUtils; + + private final JsonUtils jsonUtils = new JsonUtils(); + + + private SchemaService(SchemaServiceBuilder builder) { + this.serdeFormat = builder.serdeFormat; + this.schemaConfig = builder.schemaConfig; + this.topic = builder.topic; + this.restUtils = builder.restUtils; + this.schemaString = getSchemaString(); + this.schemaRegistryClient = builder.cachedSchemaRegistryClient; + } + + public static class SchemaServiceBuilder { + private SchemaConfig schemaConfig; + private String topic; + private String serdeFormat; + private RestUtils restUtils; + private CachedSchemaRegistryClient cachedSchemaRegistryClient; + + + public SchemaServiceBuilder() { + + } + + public SchemaServiceBuilder getRegisterationAndCompatibilityService(final String topic,final String serdeFormat,final RestUtils restUtils, + final SchemaConfig schemaConfig) { + this.topic = topic; + this.serdeFormat = serdeFormat; + this.restUtils = restUtils; + this.schemaConfig = schemaConfig; + this.cachedSchemaRegistryClient = getSchemaRegistryClient(schemaConfig); + return this; + } + + public SchemaServiceBuilder getFetchSchemaService(final String topic, final SchemaConfig schemaConfig) { + this.topic = topic; + this.schemaConfig = schemaConfig; + this.cachedSchemaRegistryClient = getSchemaRegistryClient(schemaConfig); + return this; + } + + public SchemaService build() { + return new SchemaService(this); + } + } + + public Schema getSchema(final String topic) { + final String valueToParse = getValueToParse(topic); + if (ObjectUtils.isEmpty(valueToParse)) { + return null; + } + return new Schema.Parser().parse(valueToParse); + + } + + public String getValueToParse(final String topic) { + try { + if (schemaRegistryClient != null) { + return schemaRegistryClient. + getLatestSchemaMetadata(topic).getSchema(); + } + } catch (IOException | RestClientException e) { + LOG.warn(e.getMessage()); + } + return null; + } + + + public void registerSchema(final String topic) { + try { + final String oldSchema = getValueToParse(topic); + if (ObjectUtils.isEmpty(oldSchema)) { + registerSchema(topic, schemaString); + } else if (isSchemaDifferent(oldSchema, schemaString) && isSchemasCompatible(schemaString, topic)) { + registerSchema(topic, schemaString); + } + } catch (Exception e) { + throw new RuntimeException("error occured while schema registeration " + e.getMessage()); + } + } + + private void registerSchema(final String topic, final String schemaString) throws IOException, RestClientException { + final String path = String.format(REGISTER_API_PATH, topic); + final RegisterSchemaRequest schemaRequest = new RegisterSchemaRequest(); + schemaRequest.setSchema(schemaString); + schemaRequest.setSchemaType(serdeFormat != null ? serdeFormat.toUpperCase() : null); + final RegisterSchemaResponse registerSchemaResponse = restUtils.getHttpResponse(schemaRequest.toJson(), path, new TypeReference<>() { + }); + if (registerSchemaResponse == null) { + throw new RuntimeException("Schema Registeration failed"); + } + } + + + @NotNull + private String getSchemaString() { + if (schemaConfig != null) { + if (schemaConfig.isCreate()) { + final String schemaString = getSchemaDefinition(); + if (schemaString == null) { + throw new RuntimeException("Invalid schema definition"); + } + return schemaString; + } + } + return null; + } + + + private String getSchemaDefinition() { + try { + if (schemaConfig.getInlineSchema() != null) { + return schemaConfig.getInlineSchema(); + } else if (schemaConfig.getSchemaFileLocation() != null) { + return parseSchemaFromJsonFile(schemaConfig.getSchemaFileLocation()); + } else if (checkS3SchemaValidity(schemaConfig.getS3FileConfig())) { + return getS3SchemaObject(schemaConfig.getS3FileConfig()); + } + } catch (IOException io) { + LOG.error(io.getMessage()); + } + return null; + } + + + private boolean checkS3SchemaValidity(final SchemaConfig.S3FileConfig s3FileConfig) throws IOException { + if (s3FileConfig.getBucketName() != null && s3FileConfig.getFileKey() != null && s3FileConfig.getRegion() != null) { + return true; + } else { + return false; + } + } + + private static S3Client buildS3Client(final String region) { + final AwsCredentialsProvider credentialsProvider = AwsCredentialsProviderChain.builder() + .addCredentialsProvider(DefaultCredentialsProvider.create()).build(); + return S3Client.builder() + .region(Region.of(region)) + .credentialsProvider(credentialsProvider) + .httpClientBuilder(ApacheHttpClient.builder()) + .build(); + } + + private String getS3SchemaObject(final SchemaConfig.S3FileConfig s3FileConfig) throws IOException { + final S3Client s3Client = buildS3Client(s3FileConfig.getRegion()); + final GetObjectRequest getObjectRequest = GetObjectRequest.builder() + .bucket(s3FileConfig.getBucketName()) + .key(s3FileConfig.getFileKey()) + .build(); + final ResponseInputStream s3Object = s3Client.getObject(getObjectRequest); + final Map stringObjectMap = jsonUtils.getReadValue(s3Object, new TypeReference<>() { + }); + return jsonUtils.getJsonValue(stringObjectMap); + } + + private String parseSchemaFromJsonFile(final String location) throws IOException { + final Map jsonMap; + try { + jsonMap = jsonUtils.getReadValue(Paths.get(location).toFile(), Map.class); + } catch (FileNotFoundException e) { + LOG.error("Schema file not found, Error: {}", e.getMessage()); + throw new IOException("Can't proceed without schema."); + } + final Map schemaMap = new HashMap<>(); + for (final Map.Entry entry : jsonMap.entrySet()) { + schemaMap.put(entry.getKey(), entry.getValue()); + } + try { + return jsonUtils.getJsonValue(schemaMap); + } catch (Exception e) { + LOG.error("Unable to parse schema from the provided schema file, Error: {}", e.getMessage()); + throw new IOException("Can't proceed without schema."); + } + } + + private static CachedSchemaRegistryClient getSchemaRegistryClient(final SchemaConfig schemaConfig) { + if (schemaConfig != null && schemaConfig.getRegistryURL() != null) { + return new CachedSchemaRegistryClient( + schemaConfig.getRegistryURL(), + cacheCapacity, getSchemaProperties(schemaConfig)); + } + return null; + } + + @NotNull + private static Map getSchemaProperties(final SchemaConfig schemaConfig) { + final Properties schemaProps = new Properties(); + SinkPropertyConfigurer.setSchemaCredentialsConfig(schemaConfig, schemaProps); + final Map propertiesMap = schemaProps; + return propertiesMap; + } + + private Boolean isSchemaDifferent(final String oldSchema, final String newSchema) throws JsonProcessingException { + return jsonUtils.isJsonNodeDifferent(jsonUtils.getJsonNode(oldSchema), jsonUtils.getJsonNode(newSchema)); + + } + + + private Boolean isSchemasCompatible(final String schemaString, final String topic) { + final String path = String.format(COMPATIBILITY_API_PATH, topic, schemaConfig.getVersion()); + try { + final RegisterSchemaRequest request = new RegisterSchemaRequest(); + request.setSchema(schemaString); + request.setSchemaType(serdeFormat != null ? serdeFormat.toUpperCase() : null); + final CompatibilityCheckResponse compatibilityCheckResponse = restUtils.getHttpResponse(request.toJson(), path, new TypeReference<>() { + }); + if (ObjectUtils.isEmpty(compatibilityCheckResponse)) { + return false; + } + return compatibilityCheckResponse.getIsCompatible(); + } catch (Exception ex) { + LOG.error("Error occured in testing compatiblity " + ex.getMessage()); + return false; + } + } + + +} diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/service/TopicService.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/service/TopicService.java new file mode 100644 index 0000000000..e8cb5d4e06 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/service/TopicService.java @@ -0,0 +1,39 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.kafka.service; + +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.NewTopic; +import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSinkConfig; +import org.opensearch.dataprepper.plugins.kafka.util.SinkPropertyConfigurer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; + +public class TopicService { + private static final Logger LOG = LoggerFactory.getLogger(TopicService.class); + private final AdminClient adminClient; + + + public TopicService(final KafkaSinkConfig sinkConfig) { + this.adminClient = AdminClient.create(SinkPropertyConfigurer.getPropertiesForAdmintClient(sinkConfig)); + } + + public void createTopic(final String topicName, final Integer numberOfPartitions, final Short replicationFactor) { + try { + final NewTopic newTopic = new NewTopic(topicName, numberOfPartitions, replicationFactor); + adminClient.createTopics(Collections.singletonList(newTopic)).all().get(); + LOG.info(topicName + " created successfully"); + + } catch (Exception e) { + LOG.info(topicName + " Topic already created so using the existing one"); + } + } + + public void closeAdminClient() { + adminClient.close(); + } +} diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/sink/DLQSink.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/sink/DLQSink.java index 5b393d3f19..7a82a50253 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/sink/DLQSink.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/sink/DLQSink.java @@ -18,9 +18,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.Optional; import java.util.StringJoiner; @@ -52,7 +50,7 @@ public void perform(final Object failedData, final Exception e) { .withPipelineName(pluginSetting.getPipelineName()) .withFailedData(failedData) .build(); - logFailureForDlqObjects(dlqWriter, List.of(dlqObject), e); + logFailureForDlqObjects(dlqWriter, List.of(dlqObject)); } private DlqWriter getDlqWriter() { @@ -64,7 +62,6 @@ private DlqWriter getDlqWriter() { } private DlqProvider getDlqProvider(final PluginFactory pluginFactory, final KafkaSinkConfig kafkaSinkConfig) { - final Map props = new HashMap<>(); kafkaSinkConfig.setDlqConfig(pluginSetting); final Optional dlq = kafkaSinkConfig.getDlq(); if (dlq.isPresent()) { @@ -77,7 +74,7 @@ private DlqProvider getDlqProvider(final PluginFactory pluginFactory, final Kafk return null; } - private void logFailureForDlqObjects(final DlqWriter dlqWriter, final List dlqObjects, final Throwable failure) { + private void logFailureForDlqObjects(final DlqWriter dlqWriter, final List dlqObjects) { try { dlqWriter.write(dlqObjects, pluginSetting.getPipelineName(), pluginSetting.getName()); dlqObjects.forEach((dlqObject) -> { diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkaSink.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkaSink.java index 4a530e2794..ccac810171 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkaSink.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkaSink.java @@ -5,7 +5,6 @@ package org.opensearch.dataprepper.plugins.kafka.sink; -import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient; import org.apache.kafka.clients.producer.KafkaProducer; import org.opensearch.dataprepper.expression.ExpressionEvaluator; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; @@ -19,19 +18,25 @@ import org.opensearch.dataprepper.model.sink.Sink; import org.opensearch.dataprepper.model.sink.SinkContext; import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSinkConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.SchemaConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; import org.opensearch.dataprepper.plugins.kafka.producer.KafkaSinkProducer; import org.opensearch.dataprepper.plugins.kafka.producer.ProducerWorker; +import org.opensearch.dataprepper.plugins.kafka.service.SchemaService; +import org.opensearch.dataprepper.plugins.kafka.service.TopicService; +import org.opensearch.dataprepper.plugins.kafka.util.RestUtils; import org.opensearch.dataprepper.plugins.kafka.util.SinkPropertyConfigurer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.Collection; -import java.util.Map; import java.util.Objects; import java.util.Properties; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; /** * Implementation class of kafka--sink plugin. It is responsible for receive the collection of @@ -58,6 +63,8 @@ public class KafkaSink extends AbstractSink> { private final ExpressionEvaluator expressionEvaluator; + private final Lock reentrantLock; + private final SinkContext sinkContext; @@ -69,6 +76,7 @@ public KafkaSink(final PluginSetting pluginSetting, final KafkaSinkConfig kafkaS this.kafkaSinkConfig = kafkaSinkConfig; this.pluginFactory = pluginFactory; this.expressionEvaluator = expressionEvaluator; + reentrantLock = new ReentrantLock(); this.sinkContext = sinkContext; @@ -97,10 +105,12 @@ private void doInitializeInternal() { @Override public void doOutput(Collection> records) { + reentrantLock.lock(); if (records.isEmpty()) { return; } try { + prepareTopicAndSchema(); final KafkaSinkProducer producer = createProducer(); records.forEach(record -> { producerWorker = new ProducerWorker(producer, record); @@ -111,27 +121,45 @@ public void doOutput(Collection> records) { LOG.error("Failed to setup the Kafka sink Plugin.", e); throw new RuntimeException(e.getMessage()); } + reentrantLock.unlock(); + } + + private void prepareTopicAndSchema() { + checkTopicCreationCriteriaAndCreateTopic(); + final SchemaConfig schemaConfig = kafkaSinkConfig.getSchemaConfig(); + if (schemaConfig != null) { + if (schemaConfig.isCreate()) { + final RestUtils restUtils = new RestUtils(schemaConfig); + final String topic = kafkaSinkConfig.getTopic().getName(); + final SchemaService schemaService = new SchemaService.SchemaServiceBuilder() + .getRegisterationAndCompatibilityService(topic, kafkaSinkConfig.getSerdeFormat(), + restUtils, schemaConfig).build(); + schemaService.registerSchema(topic); + } + + } + + } + + private void checkTopicCreationCriteriaAndCreateTopic() { + final TopicConfig topic = kafkaSinkConfig.getTopic(); + if (topic.isCreate()) { + final TopicService topicService = new TopicService(kafkaSinkConfig); + topicService.createTopic(kafkaSinkConfig.getTopic().getName(), topic.getNumberOfPartions(), topic.getReplicationFactor()); + topicService.closeAdminClient(); + } + + } public KafkaSinkProducer createProducer() { Properties properties = SinkPropertyConfigurer.getProducerProperties(kafkaSinkConfig); properties = Objects.requireNonNull(properties); return new KafkaSinkProducer(new KafkaProducer<>(properties), - kafkaSinkConfig, new DLQSink(pluginFactory, kafkaSinkConfig, pluginSetting), getSchemaRegistryClient(), expressionEvaluator - , Objects.nonNull(sinkContext) ? sinkContext.getTagsTargetKey() : null); + kafkaSinkConfig, new DLQSink(pluginFactory, kafkaSinkConfig, pluginSetting), + expressionEvaluator, Objects.nonNull(sinkContext) ? sinkContext.getTagsTargetKey() : null); } - private CachedSchemaRegistryClient getSchemaRegistryClient() { - if (kafkaSinkConfig.getSchemaConfig() != null && kafkaSinkConfig.getSchemaConfig().getRegistryURL() != null) { - Properties schemaProps = new Properties(); - SinkPropertyConfigurer.setSchemaProps(kafkaSinkConfig, schemaProps); - Map propertiesMap = schemaProps; - return new CachedSchemaRegistryClient( - kafkaSinkConfig.getSchemaConfig().getRegistryURL(), - 100, propertiesMap); - } - return null; - } @Override public void shutdown() { diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/AuthenticationPropertyConfigurer.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/AuthenticationPropertyConfigurer.java index 97b9951b97..b824634ac9 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/AuthenticationPropertyConfigurer.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/AuthenticationPropertyConfigurer.java @@ -45,11 +45,6 @@ public class AuthenticationPropertyConfigurer { private static final String SASL_SSL_PROTOCOL = "SASL_SSL"; - private static final String SSL_ENDPOINT_IDENTIFICATION = "ssl.endpoint.identification.algorithm"; - - private static final String USER_INFO = "USER_INFO"; - - public static void setSaslPlainTextProperties(final PlainTextAuthConfig plainTextAuthConfig, final Properties properties) { diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/JsonUtils.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/JsonUtils.java new file mode 100644 index 0000000000..7f44e26845 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/JsonUtils.java @@ -0,0 +1,46 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.kafka.util; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; + + +public class JsonUtils { + + private static final ObjectMapper objectMapper = new ObjectMapper(); + + public boolean isJsonNodeDifferent(final JsonNode oldNode,final JsonNode newNode) { + if (oldNode.isObject() && newNode.isObject()) { + final ObjectNode objNode1 = (ObjectNode) oldNode; + final ObjectNode objNode2 = (ObjectNode) newNode; + return !objNode1.equals(objNode2); + } + return !oldNode.equals(newNode); + } + + public JsonNode getJsonNode(final String value) throws JsonProcessingException { + return objectMapper.readTree(value); + } + + public T getReadValue(final InputStream value, final TypeReference type) throws IOException { + return objectMapper.readValue(value, type); + } + + public T getReadValue(final File value, final Class type) throws IOException { + return objectMapper.readValue(value, type); + } + + public String getJsonValue(final Object value) throws JsonProcessingException { + return objectMapper.writeValueAsString(value); + } +} diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/RestUtils.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/RestUtils.java new file mode 100644 index 0000000000..1afaa531ea --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/RestUtils.java @@ -0,0 +1,65 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.util; + +import com.fasterxml.jackson.core.type.TypeReference; +import io.confluent.kafka.schemaregistry.client.rest.RestService; +import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException; +import org.jetbrains.annotations.NotNull; +import org.opensearch.dataprepper.plugins.kafka.configuration.SchemaConfig; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +public class RestUtils { + + private final RestService restService; + + private static String method = "POST"; + private static String CONTENT_TYPE_KEY = "Content-Type"; + private static String CONTENT_TYPE_JSON_VALUE = "application/vnd.schemaregistry.v1+json"; + + private final SchemaConfig schemaConfig; + + public RestUtils(final SchemaConfig schemaConfig) { + this.schemaConfig = schemaConfig; + this.restService = getRestService(schemaConfig.getRegistryURL()); + } + + @NotNull + private RestService getRestService(final String url) { + final RestService restService = new RestService(url); + restService.configure(getSchemaProperties()); + return restService; + } + + + @NotNull + private Map getRequestProperties() { + final Map requestProperties = new HashMap(); + requestProperties.put(CONTENT_TYPE_KEY, CONTENT_TYPE_JSON_VALUE); + return requestProperties; + } + + @NotNull + private Map getSchemaProperties() { + final Properties schemaProps = new Properties(); + SinkPropertyConfigurer.setSchemaCredentialsConfig(schemaConfig, schemaProps); + final Map propertiesMap = schemaProps; + return propertiesMap; + } + + + public T getHttpResponse(final String schemaString, final String path, final TypeReference responseFormat) throws IOException, RestClientException { + return restService. + httpRequest(path, method, + schemaString.getBytes(StandardCharsets.UTF_8), + getRequestProperties(), responseFormat); + } +} diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/SinkPropertyConfigurer.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/SinkPropertyConfigurer.java index 8484335fce..19889e91bb 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/SinkPropertyConfigurer.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/SinkPropertyConfigurer.java @@ -114,7 +114,7 @@ public static Properties getProducerProperties(final KafkaSinkConfig kafkaSinkCo setPropertiesForSerializer(properties, kafkaSinkConfig.getSerdeFormat()); if (kafkaSinkConfig.getSchemaConfig() != null) { - setSchemaProps(kafkaSinkConfig, properties); + setSchemaProps(kafkaSinkConfig.getSerdeFormat(), kafkaSinkConfig.getSchemaConfig(), properties); } if (kafkaSinkConfig.getKafkaProducerProperties() != null) { setPropertiesProviderByKafkaProducer(kafkaSinkConfig.getKafkaProducerProperties(), properties); @@ -159,29 +159,32 @@ private static void setPropertiesForSerializer(final Properties properties, fina } } - private static void validateForRegistryURL(final KafkaSinkConfig kafkaSinkConfig) { - String serdeFormat = kafkaSinkConfig.getSerdeFormat(); + private static void validateForRegistryURL(final String serdeFormat, final SchemaConfig schemaConfig) { + if (serdeFormat.equalsIgnoreCase(MessageFormat.AVRO.toString())) { - if (kafkaSinkConfig.getSchemaConfig() == null || kafkaSinkConfig.getSchemaConfig().getRegistryURL() == null || - kafkaSinkConfig.getSchemaConfig().getRegistryURL().isBlank() || kafkaSinkConfig.getSchemaConfig().getRegistryURL().isEmpty()) { + if (schemaConfig == null || schemaConfig.getRegistryURL() == null || + schemaConfig.getRegistryURL().isBlank() || schemaConfig.getRegistryURL().isEmpty()) { throw new RuntimeException("Schema registry is mandatory when serde type is avro"); } } if (serdeFormat.equalsIgnoreCase(MessageFormat.PLAINTEXT.toString())) { - if (kafkaSinkConfig.getSchemaConfig() != null && - kafkaSinkConfig.getSchemaConfig().getRegistryURL() != null) { + if (schemaConfig != null && + schemaConfig.getRegistryURL() != null) { throw new RuntimeException("Schema registry is not required for type plaintext"); } } } - public static void setSchemaProps(final KafkaSinkConfig kafkaSinkConfig, final Properties properties) { - validateForRegistryURL(kafkaSinkConfig); - SchemaConfig schemaConfig = kafkaSinkConfig.getSchemaConfig(); + public static void setSchemaProps(final String serdeFormat, final SchemaConfig schemaConfig, final Properties properties) { + validateForRegistryURL(serdeFormat, schemaConfig); final String registryURL = schemaConfig != null ? schemaConfig.getRegistryURL() : null; if (registryURL != null && !registryURL.isEmpty()) { properties.put(REGISTRY_URL, registryURL); } + setSchemaCredentialsConfig(schemaConfig, properties); + } + + public static void setSchemaCredentialsConfig(final SchemaConfig schemaConfig,final Properties properties) { if (!ObjectUtils.isEmpty(schemaConfig.getBasicAuthCredentialsSource())) { properties.put(CREDENTIALS_SOURCE, schemaConfig.getBasicAuthCredentialsSource()); } @@ -311,7 +314,6 @@ private static void setPropertiesProviderByKafkaProducer(final KafkaProducerProp properties.put(RETRY_BACKOFF_MS, producerProperties.getRetryBackoffMs()); } - LOG.info("Producer properties"); properties.entrySet().forEach(prop -> { LOG.info("property " + prop.getKey() + " value" + prop.getValue()); @@ -320,4 +322,12 @@ private static void setPropertiesProviderByKafkaProducer(final KafkaProducerProp LOG.info("Producer properties ends"); } + public static Properties getPropertiesForAdmintClient(final KafkaSinkConfig kafkaSinkConfig) { + Properties properties = new Properties(); + setCommonServerProperties(properties, kafkaSinkConfig); + setAuthProperties(kafkaSinkConfig, properties); + return properties; + } + + } diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfigTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfigTest.java index e244ed15b1..0c4db63f8e 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfigTest.java +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfigTest.java @@ -5,13 +5,13 @@ package org.opensearch.dataprepper.plugins.kafka.configuration; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInfo; import org.mockito.Mock; import org.yaml.snakeyaml.Yaml; -import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; import java.io.FileReader; import java.io.IOException; @@ -29,7 +29,7 @@ class TopicConfigTest { @Mock - TopicConfig topicConfig; + TopicConfig topicsConfig; private static final String YAML_FILE_WITH_CONSUMER_CONFIG = "sample-pipelines.yaml"; @@ -38,7 +38,7 @@ class TopicConfigTest { @BeforeEach void setUp(TestInfo testInfo) throws IOException { String fileName = testInfo.getTags().stream().findFirst().orElse(""); - topicConfig = new TopicConfig(); + topicsConfig = new TopicConfig(); Yaml yaml = new Yaml(); FileReader fileReader = new FileReader(getClass().getClassLoader().getResource(fileName).getFile()); Object data = yaml.load(fileReader); @@ -52,82 +52,85 @@ void setUp(TestInfo testInfo) throws IOException { String json = mapper.writeValueAsString(kafkaConfigMap); Reader reader = new StringReader(json); KafkaSourceConfig kafkaSourceConfig = mapper.readValue(reader, KafkaSourceConfig.class); - List topicConfigList = kafkaSourceConfig.getTopics(); - topicConfig = topicConfigList.get(0); + List topicsConfigList = kafkaSourceConfig.getTopics(); + topicsConfig = topicsConfigList.get(0); } } @Test @Tag(YAML_FILE_WITH_CONSUMER_CONFIG) void test_topicsConfig_not_null() { - assertThat(topicConfig, notNullValue()); + assertThat(topicsConfig, notNullValue()); } @Test @Tag(YAML_FILE_WITH_MISSING_CONSUMER_CONFIG) void testConfigValues_default() { - assertEquals("my-topic-2", topicConfig.getName()); - assertEquals("my-test-group", topicConfig.getGroupId()); - assertEquals(TopicConfig.DEFAULT_AUTO_COMMIT, topicConfig.getAutoCommit()); - assertEquals(TopicConfig.DEFAULT_COMMIT_INTERVAL, topicConfig.getCommitInterval()); - assertEquals(TopicConfig.DEFAULT_SESSION_TIMEOUT, topicConfig.getSessionTimeOut()); - assertEquals(TopicConfig.DEFAULT_AUTO_OFFSET_RESET, topicConfig.getAutoOffsetReset()); - assertEquals(TopicConfig.DEFAULT_THREAD_WAITING_TIME, topicConfig.getThreadWaitingTime()); - assertEquals(TopicConfig.DEFAULT_MAX_RECORD_FETCH_TIME, topicConfig.getMaxRecordFetchTime()); - assertEquals(TopicConfig.DEFAULT_BUFFER_TIMEOUT, topicConfig.getBufferDefaultTimeout()); - assertEquals(TopicConfig.DEFAULT_FETCH_MAX_BYTES, topicConfig.getFetchMaxBytes()); - assertEquals(TopicConfig.DEFAULT_FETCH_MAX_WAIT, topicConfig.getFetchMaxWait()); - assertEquals(TopicConfig.DEFAULT_FETCH_MIN_BYTES, topicConfig.getFetchMinBytes()); - assertEquals(TopicConfig.DEFAULT_RETRY_BACKOFF, topicConfig.getRetryBackoff()); - assertEquals(TopicConfig.DEFAULT_RECONNECT_BACKOFF, topicConfig.getReconnectBackoff()); - assertEquals(TopicConfig.DEFAULT_MAX_POLL_INTERVAL, topicConfig.getMaxPollInterval()); - assertEquals(TopicConfig.DEFAULT_CONSUMER_MAX_POLL_RECORDS, topicConfig.getConsumerMaxPollRecords()); - assertEquals(TopicConfig.DEFAULT_NUM_OF_WORKERS, topicConfig.getWorkers()); - assertEquals(TopicConfig.DEFAULT_HEART_BEAT_INTERVAL_DURATION, topicConfig.getHeartBeatInterval()); - assertEquals(TopicConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES, topicConfig.getMaxPartitionFetchBytes()); + assertEquals("my-topic-2", topicsConfig.getName()); + assertEquals("my-test-group", topicsConfig.getGroupId()); + assertEquals("kafka-consumer-group-2", topicsConfig.getGroupName()); + assertEquals(false, topicsConfig.getAutoCommit()); + assertEquals(Duration.ofSeconds(5), topicsConfig.getCommitInterval()); + assertEquals(Duration.ofSeconds(45), topicsConfig.getSessionTimeOut()); + assertEquals("latest", topicsConfig.getAutoOffsetReset()); + assertEquals(Duration.ofSeconds(5), topicsConfig.getThreadWaitingTime()); + assertEquals(Duration.ofSeconds(4), topicsConfig.getMaxRecordFetchTime()); + assertEquals(Duration.ofSeconds(5), topicsConfig.getBufferDefaultTimeout()); + assertEquals(52428800L, topicsConfig.getFetchMaxBytes().longValue()); + assertEquals(500L, topicsConfig.getFetchMaxWait().longValue()); + assertEquals(1L, topicsConfig.getFetchMinBytes().longValue()); + assertEquals(Duration.ofSeconds(10), topicsConfig.getRetryBackoff()); + assertEquals(Duration.ofSeconds(300000), topicsConfig.getMaxPollInterval()); + assertEquals(500L, topicsConfig.getConsumerMaxPollRecords().longValue()); + assertEquals(2, topicsConfig.getWorkers().intValue()); + assertEquals(Duration.ofSeconds(5), topicsConfig.getHeartBeatInterval()); } @Test @Tag(YAML_FILE_WITH_CONSUMER_CONFIG) void testConfigValues_from_yaml() { - assertEquals("my-topic-1", topicConfig.getName()); - assertEquals(false, topicConfig.getAutoCommit()); - assertEquals(Duration.ofSeconds(5), topicConfig.getCommitInterval()); - assertEquals(45000, topicConfig.getSessionTimeOut().toMillis()); - assertEquals("earliest", topicConfig.getAutoOffsetReset()); - assertEquals(Duration.ofSeconds(1), topicConfig.getThreadWaitingTime()); - assertEquals(Duration.ofSeconds(4), topicConfig.getMaxRecordFetchTime()); - assertEquals(Duration.ofSeconds(5), topicConfig.getBufferDefaultTimeout()); - assertEquals(52428800, topicConfig.getFetchMaxBytes().longValue()); - assertEquals(500L, topicConfig.getFetchMaxWait().longValue()); - assertEquals(1L, topicConfig.getFetchMinBytes().longValue()); - assertEquals(Duration.ofSeconds(100), topicConfig.getRetryBackoff()); - assertEquals(Duration.ofSeconds(300000), topicConfig.getMaxPollInterval()); - assertEquals(500L, topicConfig.getConsumerMaxPollRecords().longValue()); - assertEquals(5, topicConfig.getWorkers().intValue()); - assertEquals(Duration.ofSeconds(3), topicConfig.getHeartBeatInterval()); - assertEquals(10*TopicConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES, topicConfig.getMaxPartitionFetchBytes()); + + assertEquals("my-topic-1", topicsConfig.getName()); + assertEquals("my-test-group", topicsConfig.getGroupId()); + assertEquals(null, topicsConfig.getGroupName()); + assertEquals(false, topicsConfig.getAutoCommit()); + assertEquals(Duration.ofSeconds(5), topicsConfig.getCommitInterval()); + assertEquals(Duration.ofSeconds(45), topicsConfig.getSessionTimeOut()); + assertEquals("earliest", topicsConfig.getAutoOffsetReset()); + assertEquals(Duration.ofSeconds(1), topicsConfig.getThreadWaitingTime()); + assertEquals(Duration.ofSeconds(4), topicsConfig.getMaxRecordFetchTime()); + assertEquals(Duration.ofSeconds(5), topicsConfig.getBufferDefaultTimeout()); + assertEquals(52428800L, topicsConfig.getFetchMaxBytes().longValue()); + assertEquals(500L, topicsConfig.getFetchMaxWait().longValue()); + assertEquals(1L, topicsConfig.getFetchMinBytes().longValue()); + assertEquals(Duration.ofSeconds(100), topicsConfig.getRetryBackoff()); + assertEquals(Duration.ofSeconds(300000), topicsConfig.getMaxPollInterval()); + assertEquals(500L, topicsConfig.getConsumerMaxPollRecords().longValue()); + assertEquals(5, topicsConfig.getWorkers().intValue()); + assertEquals(Duration.ofSeconds(3), topicsConfig.getHeartBeatInterval()); } @Test @Tag(YAML_FILE_WITH_CONSUMER_CONFIG) void testConfigValues_from_yaml_not_null() { - assertNotNull(topicConfig.getName()); - assertNotNull(topicConfig.getAutoCommit()); - assertNotNull(topicConfig.getCommitInterval()); - assertNotNull(topicConfig.getSessionTimeOut()); - assertNotNull(topicConfig.getAutoOffsetReset()); - assertNotNull(topicConfig.getThreadWaitingTime()); - assertNotNull(topicConfig.getMaxRecordFetchTime()); - assertNotNull(topicConfig.getBufferDefaultTimeout()); - assertNotNull(topicConfig.getFetchMaxBytes()); - assertNotNull(topicConfig.getFetchMaxWait()); - assertNotNull(topicConfig.getFetchMinBytes()); - assertNotNull(topicConfig.getRetryBackoff()); - assertNotNull(topicConfig.getMaxPollInterval()); - assertNotNull(topicConfig.getConsumerMaxPollRecords()); - assertNotNull(topicConfig.getWorkers()); - assertNotNull(topicConfig.getHeartBeatInterval()); + + assertNotNull(topicsConfig.getName()); + assertNotNull(topicsConfig.getGroupId()); + assertNotNull(topicsConfig.getAutoCommit()); + assertNotNull(topicsConfig.getCommitInterval()); + assertNotNull(topicsConfig.getSessionTimeOut()); + assertNotNull(topicsConfig.getAutoOffsetReset()); + assertNotNull(topicsConfig.getThreadWaitingTime()); + assertNotNull(topicsConfig.getMaxRecordFetchTime()); + assertNotNull(topicsConfig.getBufferDefaultTimeout()); + assertNotNull(topicsConfig.getFetchMaxBytes()); + assertNotNull(topicsConfig.getFetchMaxWait()); + assertNotNull(topicsConfig.getFetchMinBytes()); + assertNotNull(topicsConfig.getRetryBackoff()); + assertNotNull(topicsConfig.getMaxPollInterval()); + assertNotNull(topicsConfig.getConsumerMaxPollRecords()); + assertNotNull(topicsConfig.getWorkers()); + assertNotNull(topicsConfig.getHeartBeatInterval()); } } diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/producer/KafkaSinkProducerTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/producer/KafkaSinkProducerTest.java index 1d28b5d50e..c3d9625dcd 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/producer/KafkaSinkProducerTest.java +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/producer/KafkaSinkProducerTest.java @@ -5,7 +5,7 @@ package org.opensearch.dataprepper.plugins.kafka.producer; -import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient; +import com.github.fge.jsonschema.core.exceptions.ProcessingException; import io.confluent.kafka.schemaregistry.client.SchemaMetadata; import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException; import org.apache.avro.Schema; @@ -21,6 +21,7 @@ import org.mockito.junit.jupiter.MockitoExtension; import org.mockito.junit.jupiter.MockitoSettings; import org.mockito.quality.Strictness; +import org.opensearch.dataprepper.event.DefaultEventHandle; import org.opensearch.dataprepper.expression.ExpressionEvaluator; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.JacksonEvent; @@ -37,6 +38,7 @@ import java.util.concurrent.ExecutionException; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; @@ -60,16 +62,13 @@ public class KafkaSinkProducerTest { @Mock private DLQSink dlqSink; - private Event event; - - @Mock - private CachedSchemaRegistryClient cachedSchemaRegistryClient; - - final String tag = "tag"; + private JacksonEvent event; @BeforeEach public void setUp() { - event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + event = (JacksonEvent) JacksonEvent.fromMessage(UUID.randomUUID().toString()); + DefaultEventHandle defaultEventHandle = mock(DefaultEventHandle.class); + event.setEventHandle(defaultEventHandle); record = new Record<>(event); final TopicConfig topicConfig = new TopicConfig(); topicConfig.setName("test-topic"); @@ -85,7 +84,7 @@ record = new Record<>(event); public void producePlainTextRecordsTest() throws ExecutionException, InterruptedException { when(kafkaSinkConfig.getSerdeFormat()).thenReturn("plaintext"); MockProducer mockProducer = new MockProducer<>(true, new StringSerializer(), new StringSerializer()); - producer = new KafkaSinkProducer(mockProducer, kafkaSinkConfig, dlqSink, cachedSchemaRegistryClient, mock(ExpressionEvaluator.class), tag); + producer = new KafkaSinkProducer(mockProducer, kafkaSinkConfig, dlqSink, mock(ExpressionEvaluator.class), null); sinkProducer = spy(producer); sinkProducer.produceRecords(record); verify(sinkProducer).produceRecords(record); @@ -97,7 +96,7 @@ public void producePlainTextRecordsTest() throws ExecutionException, Interrupted public void produceJsonRecordsTest() throws RestClientException, IOException { when(kafkaSinkConfig.getSerdeFormat()).thenReturn("json"); MockProducer mockProducer = new MockProducer<>(true, new StringSerializer(), new JsonSerializer()); - producer = new KafkaSinkProducer(mockProducer, kafkaSinkConfig, dlqSink, cachedSchemaRegistryClient, mock(ExpressionEvaluator.class), tag); + producer = new KafkaSinkProducer(mockProducer, kafkaSinkConfig, dlqSink, mock(ExpressionEvaluator.class), null); SchemaMetadata schemaMetadata = mock(SchemaMetadata.class); String jsonSchema = "{\n" + " \"$schema\": \"http://json-schema.org/draft-07/schema#\",\n" + @@ -110,7 +109,6 @@ public void produceJsonRecordsTest() throws RestClientException, IOException { "}\n"; when(schemaMetadata.getSchema()).thenReturn(jsonSchema); - when(cachedSchemaRegistryClient.getLatestSchemaMetadata(kafkaSinkConfig.getTopic().getName() + "-value")).thenReturn(schemaMetadata); sinkProducer = spy(producer); sinkProducer.produceRecords(record); verify(sinkProducer).produceRecords(record); @@ -121,11 +119,10 @@ public void produceJsonRecordsTest() throws RestClientException, IOException { public void produceAvroRecordsTest() throws Exception { when(kafkaSinkConfig.getSerdeFormat()).thenReturn("avro"); MockProducer mockProducer = new MockProducer<>(true, new StringSerializer(), new StringSerializer()); - producer = new KafkaSinkProducer(mockProducer, kafkaSinkConfig, dlqSink, cachedSchemaRegistryClient, mock(ExpressionEvaluator.class), tag); + producer = new KafkaSinkProducer(mockProducer, kafkaSinkConfig, dlqSink, mock(ExpressionEvaluator.class), null); SchemaMetadata schemaMetadata = mock(SchemaMetadata.class); String avroSchema = "{\"type\":\"record\",\"name\":\"MyMessage\",\"fields\":[{\"name\":\"message\",\"type\":\"string\"}]}"; when(schemaMetadata.getSchema()).thenReturn(avroSchema); - when(cachedSchemaRegistryClient.getLatestSchemaMetadata(kafkaSinkConfig.getTopic().getName() + "-value")).thenReturn(schemaMetadata); sinkProducer = spy(producer); sinkProducer.produceRecords(record); verify(sinkProducer).produceRecords(record); @@ -134,7 +131,7 @@ public void produceAvroRecordsTest() throws Exception { @Test public void testGetGenericRecord() throws NoSuchMethodException, InvocationTargetException, IllegalAccessException { - producer = new KafkaSinkProducer(new MockProducer(), kafkaSinkConfig, dlqSink, cachedSchemaRegistryClient, mock(ExpressionEvaluator.class), tag); + producer = new KafkaSinkProducer(new MockProducer(), kafkaSinkConfig, dlqSink, mock(ExpressionEvaluator.class), null); final Schema schema = createMockSchema(); Method privateMethod = KafkaSinkProducer.class.getDeclaredMethod("getGenericRecord", Event.class, Schema.class); privateMethod.setAccessible(true); @@ -146,5 +143,16 @@ private Schema createMockSchema() { String schemaDefinition = "{\"type\":\"record\",\"name\":\"MyRecord\",\"fields\":[{\"name\":\"message\",\"type\":\"string\"}]}"; return new Schema.Parser().parse(schemaDefinition); } + + + @Test + public void validateSchema() throws IOException, ProcessingException { + when(kafkaSinkConfig.getSerdeFormat()).thenReturn("avro"); + MockProducer mockProducer = new MockProducer<>(true, new StringSerializer(), new StringSerializer()); + producer = new KafkaSinkProducer(mockProducer, kafkaSinkConfig, dlqSink, mock(ExpressionEvaluator.class), null); + String jsonSchema = "{\"type\": \"object\",\"properties\": {\"Year\": {\"type\": \"string\"},\"Age\": {\"type\": \"string\"},\"Ethnic\": {\"type\":\"string\",\"default\": null}}}"; + String jsonSchema2 = "{\"type\": \"object\",\"properties\": {\"Year\": {\"type\": \"string\"},\"Age\": {\"type\": \"string\"},\"Ethnic\": {\"type\":\"string\",\"default\": null}}}"; + assertTrue(producer.validateSchema(jsonSchema, jsonSchema2)); + } } diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkasinkTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkasinkTest.java index 4f12f11c87..f03a89acf4 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkasinkTest.java +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkasinkTest.java @@ -26,6 +26,8 @@ import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.sink.SinkContext; import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSinkConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.SchemaConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; import org.opensearch.dataprepper.plugins.kafka.producer.ProducerWorker; import org.springframework.test.util.ReflectionTestUtils; import org.yaml.snakeyaml.Yaml; @@ -188,4 +190,23 @@ public void isReadyTest() { ReflectionTestUtils.setField(kafkaSink, "sinkInitialized", true); assertEquals(true, kafkaSink.isReady()); } + + @Test + public void doOutputTestForAutoTopicCreate() { + + TopicConfig topicConfig = mock(TopicConfig.class); + when(topicConfig.isCreate()).thenReturn(true); + + SchemaConfig schemaConfig = mock(SchemaConfig.class); + when(schemaConfig.isCreate()).thenReturn(true); + when(schemaConfig.getRegistryURL()).thenReturn("http://localhost:8085"); + when(schemaConfig.getInlineSchema()).thenReturn("{ \"type\": \"record\", \"name\": \"Person\", \"fields\": [{ \"name\": \"Year\", \"type\": \"string\" } ] }"); + + ReflectionTestUtils.setField(kafkaSinkConfig, "schemaConfig", schemaConfig); + ReflectionTestUtils.setField(kafkaSinkConfig, "topic", topicConfig); + + when(executorService.submit(any(ProducerWorker.class))).thenReturn(futureTask); + final Collection records = Arrays.asList(new Record(event)); + assertThrows(RuntimeException.class, () -> spySink.doOutput(records)); + } } diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/util/JsonUtilsTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/util/JsonUtilsTest.java new file mode 100644 index 0000000000..0fa525cba2 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/util/JsonUtilsTest.java @@ -0,0 +1,58 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.util; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.junit.jupiter.MockitoExtension; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +@ExtendWith(MockitoExtension.class) +public class JsonUtilsTest { + + private JsonUtils jsonUtils = new JsonUtils(); + + private static final ObjectMapper objectMapper = new ObjectMapper(); + + private String json = "{\"Year\":\"2011\",\"Age\":\"25\",\"Ethnic\":\"4\"}"; + + @Test + public void isJsonNodeDifferentTest() throws JsonProcessingException { + JsonNode oldNode = objectMapper.readTree(json); + final String newJson = "{\"Year\":\"2011\",\"Age\":\"25\"}"; + JsonNode newNode = objectMapper.readTree(newJson); + Assertions.assertTrue(jsonUtils.isJsonNodeDifferent(oldNode, newNode)); + + } + + @Test + public void getJsonNodeTest() throws JsonProcessingException { + Assertions.assertNotNull(jsonUtils.getJsonNode(json)); + } + + @Test + public void getJsonValueTest() throws JsonProcessingException { + Map inputMap = new HashMap<>(); + inputMap.put("type", "string"); + Assertions.assertNotNull(jsonUtils.getJsonValue(inputMap)); + } + + @Test + public void getReadValueInputStreamTest() throws IOException { + Assertions.assertNotNull(jsonUtils.getReadValue(new ByteArrayInputStream(json.getBytes()), new TypeReference() { + })); + } + +} diff --git a/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-sink.yaml b/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-sink.yaml index 7e8b540061..f7dfd082bb 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-sink.yaml +++ b/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-sink.yaml @@ -15,6 +15,7 @@ log-pipeline : serde_format: plaintext topic: name: plaintext_test_topic + is_create: false producer_properties: buffer_memory: 10mb compression_type: gzip From a92caa148e0cdd70052c7921eaa35d25470dd738 Mon Sep 17 00:00:00 2001 From: rajeshLovesToCode Date: Thu, 10 Aug 2023 18:52:07 +0530 Subject: [PATCH 4/9] -Support for kafka-sink Signed-off-by: rajeshLovesToCode --- .../kafka-plugins/README-sink.md | 311 ++++++++++++++++++ 1 file changed, 311 insertions(+) create mode 100644 data-prepper-plugins/kafka-plugins/README-sink.md diff --git a/data-prepper-plugins/kafka-plugins/README-sink.md b/data-prepper-plugins/kafka-plugins/README-sink.md new file mode 100644 index 0000000000..9a415d1155 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/README-sink.md @@ -0,0 +1,311 @@ +# Kafka Sink + +This is the Data Prepper Kafka sink plugin that reads records from buffer and publish it to Kafka topic. It uses the producer API provided by Kafka to write messages to the kafka broker. + + +## Usages + +The Kafka sink should be configured as part of Data Prepper pipeline yaml file. + +## Configuration Options + +``` +log-pipeline : + source : + random: + sink : + - kafka: + bootstrap_servers: + - "pkc-l7pr2.ap-south-1.aws.confluent.cloud:9092" + thread_wait_time: 1000 + dlq: + s3: + bucket: "mydlqtestbucket" + key_path_prefix: "dlq-files/" + sts_role_arn: "arn:aws:iam::XXXXXXX:role/dataprepper" + region: "ap-south-1" + serde_format: json + record_key: "rec1" + topic: + name: rajesh_test_json + is_create: true + number_of_partitions: 1 + replication_factor: 3 + schema: + registry_url: https://psrc-znpo0.ap-southeast-2.aws.confluent.cloud + version: 1 + is_create: true + version: 1 + schema_file_location: /home/labuser/project/avroschema.json + basic_auth_credentials_source: USER_INFO + schema_registry_api_key: XXXXXXX + schema_registry_api_secret: XXXXXXXXXXXXXXXX+moAdK8xXM5/sp7eMaoiQ/sj5AKtbVFEjlu + inline_schema: {json schema string} + s3_file_config: + bucket_name: schemaconfuration + file_key: json_schema.txt + region: "ap-south-1" + + producer_properties: + buffer_memory: 102400000 + compression_type: gzip + retries: 3 + batch_size: 16384 + client_dns_lookup: use_all_dns_ips + connections_max_idle_ms: 540000 + delivery_timeout_ms: 120000 + linger_ms: 0 + max_block_ms: 60000 + max_request_size: 1048576 + partitioner_class: org.apache.kafka.clients.producer.internals.DefaultPartitioner + partitioner_ignore_keys: false + receive_buffer_bytes: 32768 + request_timeout_ms: 60000 + send_buffer_bytes: 131072 + socket_connection_setup_timeout_max_ms: 120000 + socket_connection_setup_timeout_ms: 10000 + acks: all + enable_idempotence: true + interceptor_classes: + - "" + max_in_flight_requests_per_connection: 5 + metadata_max_age_ms: 300000 + metadata_max_idle_ms: 300000 + metric_reporters: + - "" + metrics_num_samples: 2 + metrics_recording_level: INFO + metrics_sample_window_ms: 30000 + partitioner_adaptive_partitioning_enable: true + partitioner_availability_timeout_ms: 5000 + reconnect_backoff_max_ms: 1000 + reconnect_backoff_ms: 100 + retry_backoff_ms: 100 + authentication: + sasl: + oauth: + oauth_client_id: XXXXXX + oauth_client_secret: XXXXXXXXXXXXXXXXXXX + oauth_login_server: https://dev-XXXXXXX.okta.com + oauth_login_endpoint: /oauth2/default/v1/token + oauth_login_grant_type: refresh_token + oauth_login_scope: kafka + oauth_introspect_server: https://dev-XXXXXXX.okta.com + oauth_introspect_endpoint: /oauth2/default/v1/introspect + oauth_token_endpoint_url: https://dev-XXXXXXXXXX.okta.com/oauth2/default/v1/token + oauth_sasl_mechanism: OAUTHBEARER + oauth_security_protocol: SASL_SSL + oauth_sasl_login_callback_handler_class: org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerLoginCallbackHandler + oauth_jwks_endpoint_url: https://dev-XXXXXXXXXX.okta.com/oauth2/default/v1/keys + extension_logicalCluster: lkc-0jd9kq + extension_identityPoolId: pool-RXzn +``` + +## Configuration + +- `bootstrap_servers` (Required) : It is a host/port to use for establishing the initial connection to the Kafka cluster. Multiple brokers can be configured. + +- `thread_waiting_time` (Optional) : It is the time for thread to wait until other thread completes the task and signal it. + +- `serde_format` (Optional) : It must provide serde_format(Serializer/Deserializer) for the data types of record keys and record values. + Possible values can be plaintext or json. + +### Topic Configuration +- `name` (Required) : The topic in which kafka source plugin associated with to write the messages. + +- `is_create` (Optional) : If this flag is enabled then topic will be created based on the given configuration. + +- `number_of_partitions` (Optional) : This represents number of partitions to be configured againts the topic. + +- `replication_factor` (Optional) : This replication factor tells in how many brokers to configure this topic. + + + + +### Dead Letter Queue (DLQ) Configuration + +- `bucket` (Optional) : + +- `key_path_prefix` (Optional) : + +- `sts_role_arn` (Optional) : + +- `region` (Optional) : + +### Schema Configuration + +- `registry_url` (Optional) : This can be the local schema registry URL or any other remote or cloud schema registry URL. + +- `version` (Optional) : This schema version ensuring that different versions of the schema can be used simultaneously without causing compatibility issues. + +- `schema_registry_api_key` (Optional) : Schema Registry API key is the username required to access the Schema Registry. + +- `schema_registry_api_secret` (Optional) : Schema Registry API secret is the password required to access the Schema Registry. + +- `basic_auth_credentials_source` (Optional) : It is the security configuration to authenticate for a schema registry.USER_INFO is the default value. + +- `is_create` (Optional) : This flag will be enabled when schema has to created on the fly, By default it is false. + +From below configurations one of them is mandatory when is_create flag is enabled. + +- `schema_file_location` (Optional): This accepts a valid schema definition file path. + +- `s3_file_config` (Optional): This schema configuration from s3 bucket. + +- `inline_schema` (Optional): This accepts an inline json string for schema defintion. + + + + +### Producer Configuration +- `max.request.size` (Optional) : The maximum size of a request in bytes. + +- `retry.backoff.ms` (Optional) : The amount of time to wait before attempting to retry a failed request to a given topic partition. + +- `compression.type` (Optional) : This configuration accepts the standard compression codecs 'gzip' and 'snappy'. + +- `delivery.timeout.ms` (Optional) : This limits the total time that a record will be delayed prior to sending, the time to await acknowledgement from the broker, and the time allowed for retriable send failures. + +- `max.block.ms` (Optional) : The configuration controls how long the KafkaProducer's send(), partitionsFor(), initTransactions(), sendOffsetsToTransaction(), commitTransaction() and abortTransaction() methods will block. + +- `max.request.size` (Optional) : The maximum size of a request in bytes. + +- `partitioner.class` (Optional) : A class to use to determine which partition to be send to when produce the records. + +- `partitioner.ignore.keys` (Optional) : When set to 'true' the producer won't use record keys to choose a partition. If 'false', producer would choose a partition based on a hash of the key when a key is present. + +- `receive.buffer.bytes` (Optional) : The size of the TCP receive buffer (SO_RCVBUF) to use when reading data. If the value is -1, the OS default will be used. + +- `request.timeout.ms` (Optional) : The configuration controls the maximum amount of time the client will wait for the response of a request. + +- `send.buffer.bytes` (Optional) : The size of the TCP send buffer (SO_SNDBUF) to use when sending data. If the value is -1, the OS default will be use. + +- `acks` (Optional) : The number of acknowledgments the producer requires the leader to have received before considering a request complete. + The following settings are allowed:acks=0, acks=1 and acks=all + +- `enable.idempotence` (Optional) : When set to 'true', the producer will ensure that exactly one copy of each message is written in the stream. If 'false', producer retries due to broker failures, etc. + +- `interceptor.classes` (Optional) : A list of classes to use as interceptors.By default, there are no interceptors. + +- `max.in.flight.requests.per.connection` (Optional) : The maximum number of unacknowledged requests the client will send on a single connection before blocking. + +- `metadata.max.age.ms` (Optional) : The period of time in milliseconds after which we force a refresh of metadata even if we haven't seen any partition leadership changes to proactively discover any new brokers or partitions. + +- `metadata.max.idle.ms` (Optional) : Controls how long the producer will cache metadata for a topic that's idle. + +- `metric.reporters` (Optional) : A list of classes to use as metrics reporters. Implementing the org.apache.kafka.common.metrics.MetricsReporter interface allows plugging in classes that will be notified of new metric creation. + +- `metrics.num.samples` (Optional) : The number of samples maintained to compute metrics. + +- `metrics.recording.level` (Optional) : The highest recording level for metrics. + +- `metrics.sample.window.ms` (Optional) : The window of time a metrics sample is computed over. + +- `partitioner.adaptive.partitioning.enable` (Optional) : When set to 'true', the producer will try to adapt to broker performance and produce more messages to partitions hosted on faster brokers. If 'false', producer will try to distribute messages uniformly. + +- `partitioner.availability.timeout.ms` (Optional) : If a broker cannot process produce requests from a partition for partitioner.availability.timeout.ms time, the partitioner treats that partition as not available. If the value is 0, this logic is disabled. + +- `reconnect.backoff.max.ms` (Optional) : The maximum amount of time in milliseconds to wait when reconnecting to a broker that has repeatedly failed to connect. + +- `reconnect.backoff.ms` (Optional) : The base amount of time to wait before attempting to reconnect to a given host. + +- `socket.connection.setup.timeout.max.ms` (Optional) : The maximum amount of time the client will wait for the socket connection to be established. + +- `socket.connection.setup.timeout.ms` (Optional) : The amount of time the client will wait for the socket connection to be established. If the connection is not built before the timeout elapses, clients will close the socket channel. + +- `linger.ms` (Optional) : The producer groups together any records that arrive in between request transmissions into a single batched request. + +- `connections_max_idle_ms` (Optional) : Idle connections timeout: the server socket processor threads close the connections that idle more than this. + +- `buffer_memory` (Optional) : The total bytes of memory the producer can use to buffer records waiting to be sent to the server. + +- `batch_size` (Optional) : The producer will attempt to batch records together into fewer requests whenever multiple records are being sent to the same partition. + +- `client_dns_lookup` (Optional) : Controls how the client uses DNS lookups. If set to use_all_dns_ips, connect to each returned IP address in sequence until a successful connection is established.Valid Values: [use_all_dns_ips and resolve_canonical_bootstrap_servers_only]. + +### Auth Configuration for SASL PLAINTEXT + +- `username` (Optional) : The username for the Plaintext authentication. + +- `password` (Optional) : The password for the Plaintext authentication. + +### Auth Configuration for SASL OAUTH + +- `oauth_client_id`: It is the client id is the public identifier of your authorization server. + +- `oauth_client_secret` : It is a secret known only to the application and the authorization server. + +- `oauth_login_server` : The URL of the OAuth server.(Eg: https://dev.okta.com) + +- `oauth_login_endpoint`: The End point URL of the OAuth server.(Eg: /oauth2/default/v1/token) + +- `oauth_login_grant_type` (Optional) : This grant type refers to the way an application gets an access token. + +- `oauth_login_scope` (Optional) : This scope limit an application's access to a user's account. + +- `oauth_introspect_server` (Optional) : The URL of the introspect server. Most of the cases it should be similar to the oauth_login_server URL (Eg:https://dev.okta.com) + +- `oauth_introspect_endpoint` (Optional) : The end point of the introspect server URL.(Eg: /oauth2/default/v1/introspect) + +- `oauth_sasl_mechanism` (Optional) : It describes the authentication mechanism. + +- `oauth_security_protocol` (Optional) : It is the SASL security protocol like PLAINTEXT or SSL. + +- `oauth_sasl_login_callback_handler_class` (Optional) : It is the user defined or built in Kafka class to handle login and its callbeck. + +- `oauth_jwks_endpoint_url` (Optional) : The absolute URL for the oauth token refresh. + +- `extension_logicalCluster` (Optional) : This is the Cluster ID. + +- `extension_identityPoolId` (Optional) : This is the Pool ID which is created in the Schema Registry. + +### SSL with plaintext + - `username` (Optional) : The username for the Plaintext authentication. + + - `password` (Optional) : The password for the Plaintext authentication. + +## Integration Tests + +Before running the integration tests, make sure Kafka server is started +1. Start Zookeeper +``` +bin/zookeeper-server-start.sh config/zookeeper.properties +``` +2. Start Kafka Server with the following configuration +Configuration in config/server.properties +``` +isteners=SASL_SSL://localhost:9093,PLAINTEXT://localhost:9092,SSL://localhost:9094,SASL_PLAINTEXT://localhost:9095 +security.inter.broker.protocol=SASL_SSL +sasl.mechanism.inter.broker.protocol=PLAIN +sasl.enabled.mechanisms=PLAIN +ssl.truststore.location= +ssl.truststore.password= +ssl.keystore.location= +ssl.keystore.password= +``` +The truststore must have "localhost" certificates in them. + +Command to start kafka server +``` +bin/kafka-server-start.sh config/server.properties +``` + +3. Command to run multi auth type integration tests + +``` +./gradlew data-prepper-plugins:kafka-plugins:integrationTest -Dtests.kafka.bootstrap_servers= -Dtests.kafka.trust_store_location= -Dtests.kafka.trust_store_password= -Dtests.kafka.saslssl_bootstrap_servers= -Dtests.kafka.ssl_bootstrap_servers= -Dtests.kafka.saslplain_bootstrap_servers= -Dtests.kafka.username= -Dtests.kafka.password= --tests "*KafkaSourceMultipleAuthTypeIT*" +``` + +4. Command to run msk glue integration tests + +``` +./gradlew data-prepper-plugins:kafka-plugins:integrationTest -Dtests.kafka.bootstrap_servers= -Dtests.kafka.glue_registry_name= -Dtests.kafka.glue_avro_schema_name= -Dtests.kafka.glue_json_schema_name= -Dtests.msk.region= -Dtests.msk.arn= --tests "*TestAvroRecordConsumer*" + +``` + +## Developer Guide + +This plugin is compatible with Java 11. 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) From 574c5f90145a14592db9464a53d19c01fe5e82f4 Mon Sep 17 00:00:00 2001 From: rajeshLovesToCode Date: Fri, 11 Aug 2023 12:26:51 +0530 Subject: [PATCH 5/9] -Support for kafka-sink Signed-off-by: rajeshLovesToCode --- .../kafka-plugins/build.gradle | 3 + .../kafka/sink/KafkaSinkAvroTypeIT.java | 294 ++++++++++++++++++ .../kafka/sink/KafkaSinkJsonTypeIT.java | 266 ++++++++++++++++ .../kafka/sink/KafkaSinkPlainTextTypeIT.java | 263 ++++++++++++++++ 4 files changed, 826 insertions(+) create mode 100644 data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkaSinkAvroTypeIT.java create mode 100644 data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkaSinkJsonTypeIT.java create mode 100644 data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkaSinkPlainTextTypeIT.java diff --git a/data-prepper-plugins/kafka-plugins/build.gradle b/data-prepper-plugins/kafka-plugins/build.gradle index e5215d8c3b..3436de0133 100644 --- a/data-prepper-plugins/kafka-plugins/build.gradle +++ b/data-prepper-plugins/kafka-plugins/build.gradle @@ -91,6 +91,9 @@ task integrationTest(type: Test) { systemProperty 'tests.kafka.glue_avro_schema_name', System.getProperty('tests.kafka.glue_avro_schema_name') systemProperty 'tests.msk.region', System.getProperty('tests.msk.region') systemProperty 'tests.msk.arn', System.getProperty('tests.msk.arn') + systemProperty 'tests.kafka.confluent.registry_url', System.getProperty('tests.kafka.confluent.registry_url') + systemProperty 'tests.kafka.authconfig.username', System.getProperty('tests.kafka.authconfig.username') + systemProperty 'tests.kafka.authconfig.password', System.getProperty('tests.kafka.authconfig.password') filter { includeTestsMatching '*IT' diff --git a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkaSinkAvroTypeIT.java b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkaSinkAvroTypeIT.java new file mode 100644 index 0000000000..2f0e718cc2 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkaSinkAvroTypeIT.java @@ -0,0 +1,294 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.sink; + +import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException; +import io.confluent.kafka.serializers.KafkaAvroDeserializer; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.AdminClientConfig; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.config.SaslConfigs; +import org.apache.kafka.common.security.auth.SecurityProtocol; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.hamcrest.CoreMatchers; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.opensearch.dataprepper.expression.ExpressionEvaluator; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.log.JacksonLog; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.sink.SinkContext; +import org.opensearch.dataprepper.plugins.dlq.DlqProvider; +import org.opensearch.dataprepper.plugins.dlq.DlqWriter; +import org.opensearch.dataprepper.plugins.kafka.configuration.AuthConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSinkConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.PlainTextAuthConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.SchemaConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; +import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat; + +import java.io.IOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class KafkaSinkAvroTypeIT { + private static final int TEST_ID = 123456; + + @Mock + SchemaConfig schemaConfig; + + @Mock + private KafkaSinkConfig kafkaSinkConfig; + + @Mock + private TopicConfig topicConfig; + + private KafkaSink kafkaSink; + + private String bootstrapServers; + private String testTopic; + + private PluginSetting pluginSetting; + + @Mock + private PluginFactory pluginFactory; + + private SinkContext sinkContext; + + private String registryUrl; + + @Mock + private DlqProvider dlqProvider; + + @Mock + private DlqWriter dlqWriter; + + @Mock + private ExpressionEvaluator evaluator; + + private AuthConfig authConfig; + private AuthConfig.SaslAuthConfig saslAuthConfig; + private PlainTextAuthConfig plainTextAuthConfig; + private static final Properties props = new Properties(); + + + public KafkaSink createObjectUnderTest() { + return new KafkaSink(pluginSetting, kafkaSinkConfig, pluginFactory, evaluator, sinkContext); + } + + @BeforeEach + public void setup() throws RestClientException, IOException { + plainTextAuthConfig = mock(PlainTextAuthConfig.class); + authConfig = mock(AuthConfig.class); + saslAuthConfig = mock(AuthConfig.SaslAuthConfig.class); + schemaConfig = mock(SchemaConfig.class); + evaluator = mock(ExpressionEvaluator.class); + dlqWriter = mock(DlqWriter.class); + dlqProvider = mock(DlqProvider.class); + sinkContext = mock(SinkContext.class); + pluginFactory = mock(PluginFactory.class); + pluginSetting = mock(PluginSetting.class); + when(pluginSetting.getName()).thenReturn("name"); + when(pluginSetting.getPipelineName()).thenReturn("pipelinename"); + + when(pluginFactory.loadPlugin(any(Class.class), any(PluginSetting.class))).thenReturn(dlqProvider); + when(dlqProvider.getDlqWriter(anyString())).thenReturn(Optional.of(dlqWriter)); + + kafkaSinkConfig = mock(KafkaSinkConfig.class); + + registryUrl = System.getProperty("tests.kafka.confluent.registry_url"); + when(schemaConfig.getRegistryURL()).thenReturn(registryUrl); + String schemaStr = "{\"type\": \"record\", \"name\": \"Example\",\"fields\": " + + "[{\"name\": \"name\",\"type\": \"string\"}," + + "{\"name\": \"id\",\"type\": \"string\"} ]}"; + Schema schema = Schema.parse("{\"type\": \"record\", \"name\": \"Example\",\"fields\": " + + "[{\"name\": \"name\",\"type\": \"string\"}," + + "{\"name\": \"id\",\"type\": \"string\"} ]}"); + + when(schemaConfig.getInlineSchema()).thenReturn(schemaStr); + when(schemaConfig.isCreate()).thenReturn(true); + when(kafkaSinkConfig.getSchemaConfig()).thenReturn(schemaConfig); + + when(kafkaSinkConfig.getSerdeFormat()).thenReturn(MessageFormat.AVRO.toString()); + when(kafkaSinkConfig.getPartitionKey()).thenReturn("test-${name}"); + + final String testGroup = "TestGroup_" + RandomStringUtils.randomAlphabetic(5); + testTopic = "TestTopic_" + RandomStringUtils.randomAlphabetic(5); + + topicConfig = mock(TopicConfig.class); + when(topicConfig.getName()).thenReturn(testTopic); + when(topicConfig.getGroupId()).thenReturn(testGroup); + when(topicConfig.getWorkers()).thenReturn(1); + when(topicConfig.getSessionTimeOut()).thenReturn(Duration.ofSeconds(45)); + when(topicConfig.getHeartBeatInterval()).thenReturn(Duration.ofSeconds(3)); + when(topicConfig.getAutoCommit()).thenReturn(false); + when(topicConfig.getAutoOffsetReset()).thenReturn("earliest"); + when(topicConfig.getThreadWaitingTime()).thenReturn(Duration.ofSeconds(1)); + + bootstrapServers = System.getProperty("tests.kafka.bootstrap_servers"); + when(kafkaSinkConfig.getBootStrapServers()).thenReturn(Collections.singletonList(bootstrapServers)); + + props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); + + } + + @Test + public void TestPollRecordsAvroSASLPlainText() throws Exception { + + configureJasConfForSASLPlainText(); + + final int numRecords = 1; + when(topicConfig.getConsumerMaxPollRecords()).thenReturn(numRecords); + when(topicConfig.isCreate()).thenReturn(false); + when(kafkaSinkConfig.getTopic()).thenReturn(topicConfig); + + when(kafkaSinkConfig.getAuthConfig()).thenReturn(authConfig); + kafkaSink = createObjectUnderTest(); + + AtomicBoolean created = new AtomicBoolean(false); + final String topicName = topicConfig.getName(); + + createTopic(created, topicName); + + final List> records = new ArrayList<>(); + + for (int i = 0; i < numRecords; i++) { + final Map eventData = new HashMap<>(); + eventData.put("name", "testName"); + eventData.put("id", "" + TEST_ID + i); + final JacksonEvent event = JacksonLog.builder().withData(eventData).build(); + records.add(new Record<>(event)); + } + + kafkaSink.doInitialize(); + kafkaSink.doOutput(records); + + Thread.sleep(4000); + + consumeTestMessages(records); + + deleteTopic(created, topicName); + } + + private void configureJasConfForSASLPlainText() { + String username = System.getProperty("tests.kafka.authconfig.username"); + String password = System.getProperty("tests.kafka.authconfig.password"); + when(plainTextAuthConfig.getUsername()).thenReturn(username); + when(plainTextAuthConfig.getPassword()).thenReturn(password); + when(saslAuthConfig.getPlainTextAuthConfig()).thenReturn(plainTextAuthConfig); + when(authConfig.getSaslAuthConfig()).thenReturn(saslAuthConfig); + + + String jasConf = "org.apache.kafka.common.security.plain.PlainLoginModule required username=\"" + username + "\" password=\"" + password + "\";"; + props.put(SaslConfigs.SASL_JAAS_CONFIG, jasConf); + props.put(SaslConfigs.SASL_MECHANISM, "PLAIN"); + props.put("security.protocol", SecurityProtocol.SASL_PLAINTEXT.toString()); + } + + private void createTopic(AtomicBoolean created, String topicName) throws InterruptedException { + try (AdminClient adminClient = AdminClient.create(props)) { + try { + adminClient.createTopics( + Collections.singleton(new NewTopic(topicName, 1, (short) 1))) + .all().get(30, TimeUnit.SECONDS); + } catch (Exception e) { + throw new RuntimeException(e); + } + created.set(true); + } + while (created.get() != true) { + Thread.sleep(1000); + } + } + + private void deleteTopic(AtomicBoolean created, String topicName) throws InterruptedException { + try (AdminClient adminClient = AdminClient.create(props)) { + try { + adminClient.deleteTopics(Collections.singleton(topicName)) + .all().get(30, TimeUnit.SECONDS); + } catch (Exception e) { + throw new RuntimeException(e); + } + created.set(false); + } + while (created.get() != false) { + Thread.sleep(1000); + } + } + + private void consumeTestMessages(List> recList) { + + props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, + topicConfig.getCommitInterval().toSecondsPart()); + props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, + topicConfig.getAutoOffsetReset()); + props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, + topicConfig.getAutoCommit()); + props.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, + topicConfig.getConsumerMaxPollRecords()); + props.put(ConsumerConfig.GROUP_ID_CONFIG, topicConfig.getGroupId()); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + StringDeserializer.class); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + KafkaAvroDeserializer.class); + props.put("schema.registry.url", schemaConfig.getRegistryURL()); + + KafkaConsumer kafkaConsumer = new KafkaConsumer(props); + + kafkaConsumer.subscribe(Arrays.asList(topicConfig.getName())); + + pollRecords(recList, kafkaConsumer); + } + + private void pollRecords(List> recList, KafkaConsumer kafkaConsumer) { + int recListCounter = 0; + boolean isPollNext = true; + while (isPollNext) { + ConsumerRecords records = kafkaConsumer.poll(1000); + if (!records.isEmpty() && records.count() > 0) { + for (ConsumerRecord record : records) { + Record recordEvent = recList.get(recListCounter); + String inputJsonStr = recordEvent.getData().toJsonString(); + + GenericRecord recValue = record.value(); + String recValueStr = recValue.toString().replaceAll("\\s", ""); + assertThat(recValueStr, CoreMatchers.containsString(inputJsonStr)); + if (recListCounter + 1 == recList.size()) { + isPollNext = false; + } + recListCounter++; + break; + } + } + } + } +} \ No newline at end of file diff --git a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkaSinkJsonTypeIT.java b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkaSinkJsonTypeIT.java new file mode 100644 index 0000000000..278b63ce1e --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkaSinkJsonTypeIT.java @@ -0,0 +1,266 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.sink; + +import com.fasterxml.jackson.databind.JsonNode; +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.AdminClientConfig; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.config.SaslConfigs; +import org.apache.kafka.common.security.auth.SecurityProtocol; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.connect.json.JsonDeserializer; +import org.hamcrest.CoreMatchers; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.opensearch.dataprepper.expression.ExpressionEvaluator; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.log.JacksonLog; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.sink.SinkContext; +import org.opensearch.dataprepper.plugins.dlq.DlqProvider; +import org.opensearch.dataprepper.plugins.dlq.DlqWriter; +import org.opensearch.dataprepper.plugins.kafka.configuration.AuthConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSinkConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.PlainTextAuthConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; +import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class KafkaSinkJsonTypeIT { + private static final int TEST_ID = 123456; + @Mock + private KafkaSinkConfig kafkaSinkConfig; + + @Mock + private TopicConfig topicConfig; + + private KafkaSink kafkaSink; + + private String bootstrapServers; + private String testTopic; + + private PluginSetting pluginSetting; + + @Mock + private PluginFactory pluginFactory; + + private SinkContext sinkContext; + + @Mock + private DlqProvider dlqProvider; + + @Mock + private DlqWriter dlqWriter; + + @Mock + private ExpressionEvaluator evaluator; + + private PlainTextAuthConfig plainTextAuthConfig; + private AuthConfig.SaslAuthConfig saslAuthConfig; + private AuthConfig authConfig; + + private static final Properties props = new Properties(); + + + public KafkaSink createObjectUnderTest() { + return new KafkaSink(pluginSetting, kafkaSinkConfig, pluginFactory, evaluator, sinkContext); + } + + @BeforeEach + public void setup() { + plainTextAuthConfig = mock(PlainTextAuthConfig.class); + saslAuthConfig = mock(AuthConfig.SaslAuthConfig.class); + authConfig = mock(AuthConfig.class); + + evaluator = mock(ExpressionEvaluator.class); + dlqWriter = mock(DlqWriter.class); + dlqProvider = mock(DlqProvider.class); + sinkContext = mock(SinkContext.class); + pluginFactory = mock(PluginFactory.class); + pluginSetting = mock(PluginSetting.class); + when(pluginSetting.getName()).thenReturn("name"); + when(pluginSetting.getPipelineName()).thenReturn("pipelinename"); + + when(pluginFactory.loadPlugin(any(Class.class), any(PluginSetting.class))).thenReturn(dlqProvider); + when(dlqProvider.getDlqWriter(anyString())).thenReturn(Optional.of(dlqWriter)); + + kafkaSinkConfig = mock(KafkaSinkConfig.class); + when(kafkaSinkConfig.getSchemaConfig()).thenReturn(null); + when(kafkaSinkConfig.getSerdeFormat()).thenReturn(MessageFormat.JSON.toString()); + when(kafkaSinkConfig.getPartitionKey()).thenReturn("test-${name}"); + + final String testGroup = "TestGroup_" + RandomStringUtils.randomAlphabetic(5); + testTopic = "TestTopic_" + RandomStringUtils.randomAlphabetic(5); + + topicConfig = mock(TopicConfig.class); + when(topicConfig.getName()).thenReturn(testTopic); + when(topicConfig.getGroupId()).thenReturn(testGroup); + when(topicConfig.getWorkers()).thenReturn(1); + when(topicConfig.getSessionTimeOut()).thenReturn(Duration.ofSeconds(45)); + when(topicConfig.getHeartBeatInterval()).thenReturn(Duration.ofSeconds(3)); + when(topicConfig.getAutoCommit()).thenReturn(false); + when(topicConfig.getAutoOffsetReset()).thenReturn("earliest"); + when(topicConfig.getThreadWaitingTime()).thenReturn(Duration.ofSeconds(1)); + bootstrapServers = System.getProperty("tests.kafka.bootstrap_servers"); + when(kafkaSinkConfig.getBootStrapServers()).thenReturn(Collections.singletonList(bootstrapServers)); + props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); + } + + @Test + public void TestPollRecordsJsonSASLPlainText() throws Exception { + configureJasConfForSASLPlainText(); + + final int numRecords = 1; + when(topicConfig.getConsumerMaxPollRecords()).thenReturn(numRecords); + when(topicConfig.isCreate()).thenReturn(false); + when(kafkaSinkConfig.getTopic()).thenReturn(topicConfig); + when(kafkaSinkConfig.getAuthConfig()).thenReturn(authConfig); + kafkaSink = createObjectUnderTest(); + + + AtomicBoolean created = new AtomicBoolean(false); + final String topicName = topicConfig.getName(); + + createTopic(created, topicName); + + final List> records = new ArrayList<>(); + + for (int i = 0; i < numRecords; i++) { + final Map eventData = new HashMap<>(); + eventData.put("name", "testName"); + eventData.put("id", "" + TEST_ID + i); + final JacksonEvent event = JacksonLog.builder().withData(eventData).build(); + records.add(new Record<>(event)); + } + + kafkaSink.doInitialize(); + kafkaSink.doOutput(records); + + Thread.sleep(4000); + + consumeTestMessages(records); + + deleteTopic(created, topicName); + } + + private void createTopic(AtomicBoolean created, String topicName) throws InterruptedException { + try (AdminClient adminClient = AdminClient.create(props)) { + try { + adminClient.createTopics( + Collections.singleton(new NewTopic(topicName, 1, (short) 1))) + .all().get(30, TimeUnit.SECONDS); + } catch (Exception e) { + throw new RuntimeException(e); + } + created.set(true); + } + while (created.get() != true) { + Thread.sleep(1000); + } + } + + private void deleteTopic(AtomicBoolean created, String topicName) throws InterruptedException { + try (AdminClient adminClient = AdminClient.create(props)) { + try { + adminClient.deleteTopics(Collections.singleton(topicName)) + .all().get(30, TimeUnit.SECONDS); + } catch (Exception e) { + throw new RuntimeException(e); + } + created.set(false); + } + while (created.get() != false) { + Thread.sleep(1000); + } + } + + private void configureJasConfForSASLPlainText() { + String username = System.getProperty("tests.kafka.authconfig.username"); + String password = System.getProperty("tests.kafka.authconfig.password"); + when(plainTextAuthConfig.getUsername()).thenReturn(username); + when(plainTextAuthConfig.getPassword()).thenReturn(password); + when(saslAuthConfig.getPlainTextAuthConfig()).thenReturn(plainTextAuthConfig); + when(authConfig.getSaslAuthConfig()).thenReturn(saslAuthConfig); + + String jasConf = "org.apache.kafka.common.security.plain.PlainLoginModule required username=\"" + username + "\" password=\"" + password + "\";"; + props.put(SaslConfigs.SASL_JAAS_CONFIG, jasConf); + props.put(SaslConfigs.SASL_MECHANISM, "PLAIN"); + props.put("security.protocol", SecurityProtocol.SASL_PLAINTEXT.toString()); + } + + private void consumeTestMessages(List> recList) { + + props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, + topicConfig.getCommitInterval().toSecondsPart()); + props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, + topicConfig.getAutoOffsetReset()); + props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, + topicConfig.getAutoCommit()); + props.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, + topicConfig.getConsumerMaxPollRecords()); + props.put(ConsumerConfig.GROUP_ID_CONFIG, topicConfig.getGroupId()); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + StringDeserializer.class); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + JsonDeserializer.class); + + KafkaConsumer kafkaConsumer = new KafkaConsumer<>(props); + + kafkaConsumer.subscribe(Arrays.asList(topicConfig.getName())); + + pollRecords(recList, kafkaConsumer); + } + + private void pollRecords(List> recList, KafkaConsumer kafkaConsumer) { + int recListCounter = 0; + boolean isPollNext = true; + while (isPollNext) { + ConsumerRecords records = kafkaConsumer.poll(1000); + if (!records.isEmpty() && records.count() > 0) { + for (ConsumerRecord record : records) { + Record recordEvent = recList.get(recListCounter); + String inputJsonStr = recordEvent.getData().toJsonString(); + + JsonNode recValue = record.value(); + String ss = recValue.asText(); + assertThat(ss, CoreMatchers.containsString(inputJsonStr)); + if (recListCounter + 1 == recList.size()) { + isPollNext = false; + } + recListCounter++; + break; + } + } + } + } +} \ No newline at end of file diff --git a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkaSinkPlainTextTypeIT.java b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkaSinkPlainTextTypeIT.java new file mode 100644 index 0000000000..dc8ae83063 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/sink/KafkaSinkPlainTextTypeIT.java @@ -0,0 +1,263 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.sink; + +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.AdminClientConfig; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.config.SaslConfigs; +import org.apache.kafka.common.security.auth.SecurityProtocol; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.hamcrest.CoreMatchers; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.opensearch.dataprepper.expression.ExpressionEvaluator; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.log.JacksonLog; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.sink.SinkContext; +import org.opensearch.dataprepper.plugins.dlq.DlqProvider; +import org.opensearch.dataprepper.plugins.dlq.DlqWriter; +import org.opensearch.dataprepper.plugins.kafka.configuration.AuthConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSinkConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.PlainTextAuthConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; +import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class KafkaSinkPlainTextTypeIT { + private static final int TEST_ID = 123456; + @Mock + private KafkaSinkConfig kafkaSinkConfig; + + @Mock + private TopicConfig topicConfig; + + private KafkaSink kafkaSink; + + private String bootstrapServers; + private String testTopic; + + private PluginSetting pluginSetting; + + @Mock + private PluginFactory pluginFactory; + + private SinkContext sinkContext; + + @Mock + private DlqProvider dlqProvider; + + @Mock + private DlqWriter dlqWriter; + + @Mock + private ExpressionEvaluator evaluator; + + private PlainTextAuthConfig plainTextAuthConfig; + private AuthConfig.SaslAuthConfig saslAuthConfig; + private AuthConfig authConfig; + + private static final Properties props = new Properties(); + + + public KafkaSink createObjectUnderTest() { + return new KafkaSink(pluginSetting, kafkaSinkConfig, pluginFactory, evaluator, sinkContext); + } + + @BeforeEach + public void setup() { + plainTextAuthConfig = mock(PlainTextAuthConfig.class); + saslAuthConfig = mock(AuthConfig.SaslAuthConfig.class); + authConfig = mock(AuthConfig.class); + + evaluator = mock(ExpressionEvaluator.class); + dlqWriter = mock(DlqWriter.class); + dlqProvider = mock(DlqProvider.class); + sinkContext = mock(SinkContext.class); + pluginFactory = mock(PluginFactory.class); + pluginSetting = mock(PluginSetting.class); + when(pluginSetting.getName()).thenReturn("name"); + when(pluginSetting.getPipelineName()).thenReturn("pipelinename"); + + when(pluginFactory.loadPlugin(any(Class.class), any(PluginSetting.class))).thenReturn(dlqProvider); + when(dlqProvider.getDlqWriter(anyString())).thenReturn(Optional.of(dlqWriter)); + + kafkaSinkConfig = mock(KafkaSinkConfig.class); + when(kafkaSinkConfig.getSchemaConfig()).thenReturn(null); + when(kafkaSinkConfig.getSerdeFormat()).thenReturn(MessageFormat.PLAINTEXT.toString()); + when(kafkaSinkConfig.getPartitionKey()).thenReturn("test-${name}"); + + final String testGroup = "TestGroup_" + RandomStringUtils.randomAlphabetic(5); + testTopic = "TestTopic_" + RandomStringUtils.randomAlphabetic(5); + + topicConfig = mock(TopicConfig.class); + when(topicConfig.getName()).thenReturn(testTopic); + when(topicConfig.getGroupId()).thenReturn(testGroup); + when(topicConfig.getWorkers()).thenReturn(1); + when(topicConfig.getSessionTimeOut()).thenReturn(Duration.ofSeconds(45)); + when(topicConfig.getHeartBeatInterval()).thenReturn(Duration.ofSeconds(3)); + when(topicConfig.getAutoCommit()).thenReturn(false); + when(topicConfig.getAutoOffsetReset()).thenReturn("earliest"); + when(topicConfig.getThreadWaitingTime()).thenReturn(Duration.ofSeconds(1)); + bootstrapServers = System.getProperty("tests.kafka.bootstrap_servers"); + when(kafkaSinkConfig.getBootStrapServers()).thenReturn(Collections.singletonList(bootstrapServers)); + props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); + } + + @Test + public void TestPollRecordsPlainText() throws Exception { + + configureJasConfForSASLPlainText(); + + final int numRecords = 1; + when(topicConfig.getConsumerMaxPollRecords()).thenReturn(numRecords); + when(topicConfig.isCreate()).thenReturn(false); + when(kafkaSinkConfig.getTopic()).thenReturn(topicConfig); + when(kafkaSinkConfig.getAuthConfig()).thenReturn(authConfig); + kafkaSink = createObjectUnderTest(); + + AtomicBoolean created = new AtomicBoolean(false); + final String topicName = topicConfig.getName(); + + createTopic(created, topicName); + + final List> records = new ArrayList<>(); + + for (int i = 0; i < numRecords; i++) { + final Map eventData = new HashMap<>(); + eventData.put("name", "testName"); + eventData.put("id", "" + TEST_ID + i); + final JacksonEvent event = JacksonLog.builder().withData(eventData).build(); + records.add(new Record<>(event)); + } + + kafkaSink.doInitialize(); + kafkaSink.doOutput(records); + + Thread.sleep(4000); + + consumeTestMessages(records); + + deleteTopic(created, topicName); + } + + private void configureJasConfForSASLPlainText() { + String username = System.getProperty("tests.kafka.authconfig.username"); + String password = System.getProperty("tests.kafka.authconfig.password"); + when(plainTextAuthConfig.getUsername()).thenReturn(username); + when(plainTextAuthConfig.getPassword()).thenReturn(password); + when(saslAuthConfig.getPlainTextAuthConfig()).thenReturn(plainTextAuthConfig); + when(authConfig.getSaslAuthConfig()).thenReturn(saslAuthConfig); + + String jasConf = "org.apache.kafka.common.security.plain.PlainLoginModule required username=\"" + username + "\" password=\"" + password + "\";"; + props.put(SaslConfigs.SASL_JAAS_CONFIG, jasConf); + props.put(SaslConfigs.SASL_MECHANISM, "PLAIN"); + props.put("security.protocol", SecurityProtocol.SASL_PLAINTEXT.toString()); + } + + private void createTopic(AtomicBoolean created, String topicName) throws InterruptedException { + try (AdminClient adminClient = AdminClient.create(props)) { + try { + adminClient.createTopics( + Collections.singleton(new NewTopic(topicName, 1, (short) 1))) + .all().get(30, TimeUnit.SECONDS); + } catch (Exception e) { + throw new RuntimeException(e); + } + created.set(true); + } + while (created.get() != true) { + Thread.sleep(1000); + } + } + + private void deleteTopic(AtomicBoolean created, String topicName) throws InterruptedException { + try (AdminClient adminClient = AdminClient.create(props)) { + try { + adminClient.deleteTopics(Collections.singleton(topicName)) + .all().get(30, TimeUnit.SECONDS); + } catch (Exception e) { + throw new RuntimeException(e); + } + created.set(false); + } + while (created.get() != false) { + Thread.sleep(1000); + } + } + + private void consumeTestMessages(List> recList) { + + props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, + topicConfig.getCommitInterval().toSecondsPart()); + props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, + topicConfig.getAutoOffsetReset()); + props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, + topicConfig.getAutoCommit()); + props.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, + topicConfig.getConsumerMaxPollRecords()); + props.put(ConsumerConfig.GROUP_ID_CONFIG, topicConfig.getGroupId()); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + StringDeserializer.class); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + StringDeserializer.class); + + KafkaConsumer kafkaConsumer = new KafkaConsumer(props); + + kafkaConsumer.subscribe(Arrays.asList(topicConfig.getName())); + + pollRecords(recList, kafkaConsumer); + } + + private void pollRecords(List> recList, KafkaConsumer kafkaConsumer) { + int recListCounter = 0; + boolean isPollNext = true; + while (isPollNext) { + ConsumerRecords records = kafkaConsumer.poll(1000); + if (!records.isEmpty() && records.count() > 0) { + for (ConsumerRecord record : records) { + Record recordEvent = recList.get(recListCounter); + String inputJsonStr = recordEvent.getData().toJsonString(); + + String recValue = record.value(); + assertThat(recValue, CoreMatchers.containsString(inputJsonStr)); + if (recListCounter + 1 == recList.size()) { + isPollNext = false; + } + recListCounter++; + break; + } + } + } + } +} \ No newline at end of file From 3a7301941b4903a6da714b43654e1b5ef45afa7a Mon Sep 17 00:00:00 2001 From: rajeshLovesToCode Date: Fri, 11 Aug 2023 13:03:37 +0530 Subject: [PATCH 6/9] -Support for kafka-sink Signed-off-by: rajeshLovesToCode --- .../plugins/kafka/configuration/TopicConfig.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfig.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfig.java index 0c1ff2f892..87f98c1956 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfig.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfig.java @@ -40,8 +40,8 @@ public class TopicConfig { static final Duration DEFAULT_HEART_BEAT_INTERVAL_DURATION = Duration.ofSeconds(5); - private static final Integer NUM_OF_PARTITIONS = 3; - private static final Short REPLICATION_FACTOR = 1; + private static final Integer DEFAULT_NUM_OF_PARTITIONS = 3; + private static final Short DEFAULT_REPLICATION_FACTOR = 1; @JsonProperty("name") @@ -146,10 +146,10 @@ public class TopicConfig { private Boolean isCreate=Boolean.FALSE; @JsonProperty("number_of_partitions") - private Integer numberOfPartions = NUM_OF_PARTITIONS; + private Integer numberOfPartions = DEFAULT_NUM_OF_PARTITIONS; @JsonProperty("replication_factor") - private Short replicationFactor = REPLICATION_FACTOR; + private Short replicationFactor = DEFAULT_REPLICATION_FACTOR; public String getGroupId() { return groupId; From d427f801fdc7182f61b95572228bf64694747e76 Mon Sep 17 00:00:00 2001 From: rajeshLovesToCode Date: Mon, 21 Aug 2023 16:06:36 +0530 Subject: [PATCH 7/9] -Support for kafka-sink Signed-off-by: rajeshLovesToCode --- .../dataprepper/plugins/kafka/source/KafkaSource.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 f67720f5db..ea021f8474 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 @@ -9,7 +9,7 @@ import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException; import io.confluent.kafka.serializers.KafkaAvroDeserializerConfig; import io.confluent.kafka.serializers.KafkaJsonDeserializer; -//import kafka.common.BrokerEndPointNotAvailableException; +import kafka.common.BrokerEndPointNotAvailableException; import org.apache.avro.generic.GenericRecord; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; @@ -148,12 +148,12 @@ public void start(Buffer> buffer) { executorService.submit(consumer); }); } catch (Exception e) { - /* if (e instanceof BrokerNotAvailableException || + if (e instanceof BrokerNotAvailableException || e instanceof BrokerEndPointNotAvailableException || e instanceof TimeoutException) { LOG.error("The kafka broker is not available..."); } else { LOG.error("Failed to setup the Kafka Source Plugin.", e); - }*/ + } throw new RuntimeException(e); } LOG.info("Started Kafka source for topic " + topic.getName()); From c09e92aed90aa8a255f62684f2f84555efaad6cb Mon Sep 17 00:00:00 2001 From: rajeshLovesToCode Date: Mon, 21 Aug 2023 16:59:37 +0530 Subject: [PATCH 8/9] -Support for kafka-sink Signed-off-by: rajeshLovesToCode --- .../kafka-plugins/src/test/resources/sample-pipelines-sink.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-sink.yaml b/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-sink.yaml index f7dfd082bb..e417522854 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-sink.yaml +++ b/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-sink.yaml @@ -15,7 +15,7 @@ log-pipeline : serde_format: plaintext topic: name: plaintext_test_topic - is_create: false + is_topic_create: false producer_properties: buffer_memory: 10mb compression_type: gzip From eb97d1cbc677fa7450a29c16f3a95d468e4ba110 Mon Sep 17 00:00:00 2001 From: rajeshLovesToCode Date: Mon, 21 Aug 2023 18:46:13 +0530 Subject: [PATCH 9/9] -Support for kafka-sink Signed-off-by: rajeshLovesToCode --- .../plugins/kafka/producer/KafkaSinkProducerTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/producer/KafkaSinkProducerTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/producer/KafkaSinkProducerTest.java index c3d9625dcd..fc43ce2f13 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/producer/KafkaSinkProducerTest.java +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/producer/KafkaSinkProducerTest.java @@ -94,7 +94,7 @@ public void producePlainTextRecordsTest() throws ExecutionException, Interrupted @Test public void produceJsonRecordsTest() throws RestClientException, IOException { - when(kafkaSinkConfig.getSerdeFormat()).thenReturn("json"); + when(kafkaSinkConfig.getSerdeFormat()).thenReturn("JSON"); MockProducer mockProducer = new MockProducer<>(true, new StringSerializer(), new JsonSerializer()); producer = new KafkaSinkProducer(mockProducer, kafkaSinkConfig, dlqSink, mock(ExpressionEvaluator.class), null); SchemaMetadata schemaMetadata = mock(SchemaMetadata.class); @@ -117,7 +117,7 @@ public void produceJsonRecordsTest() throws RestClientException, IOException { @Test public void produceAvroRecordsTest() throws Exception { - when(kafkaSinkConfig.getSerdeFormat()).thenReturn("avro"); + when(kafkaSinkConfig.getSerdeFormat()).thenReturn("AVRO"); MockProducer mockProducer = new MockProducer<>(true, new StringSerializer(), new StringSerializer()); producer = new KafkaSinkProducer(mockProducer, kafkaSinkConfig, dlqSink, mock(ExpressionEvaluator.class), null); SchemaMetadata schemaMetadata = mock(SchemaMetadata.class);