From d93ead9f39588a3fb5d7c0d20df11a8875070fce Mon Sep 17 00:00:00 2001 From: wanghd89 Date: Wed, 5 Jul 2023 10:42:07 +0800 Subject: [PATCH 01/10] Define multiple keys for type conversion (#2934) * feat: add include_key options to KeyValueProcessor Signed-off-by: Haidong --------- Signed-off-by: Haidong Co-authored-by: Haidong --- .../mutate-event-processors/README.md | 5 ++- .../ConvertEntryTypeProcessor.java | 36 ++++++++++++--- .../ConvertEntryTypeProcessorConfig.java | 7 ++- .../ConvertEntryTypeProcessorTests.java | 44 ++++++++++++++++++- ...vertEntryTypeProcessor_NullValueTests.java | 27 ++++++++++-- 5 files changed, 104 insertions(+), 15 deletions(-) diff --git a/data-prepper-plugins/mutate-event-processors/README.md b/data-prepper-plugins/mutate-event-processors/README.md index 8d81f9eb94..bcc869ba08 100644 --- a/data-prepper-plugins/mutate-event-processors/README.md +++ b/data-prepper-plugins/mutate-event-processors/README.md @@ -261,8 +261,11 @@ and the type conversion processor will change it to the following output, where {"message": "10.10.10.10 [19/Feb/2015:15:50:36 -0500] 200", "clientip":"10.10.10.10", "timestamp": "19/Feb/2015:15:50:36 -0500", "response_status": 200} ``` ### Configuration -* `key` - (required) - keys whose value needs to be converted to a different type +* `key` - keys whose value needs to be converted to a different type. Required if `keys` option is not defined. +* `keys` - list of keys whose value needs to be converted to a different type. Required if `key` option is not defined. * `type` - target type for the value of the key. Possible values are `integer`, `double`, `string`, and `boolean`. Default is `integer`. +* `null_values` - treat any value in the null_values list as null. + * Example: `null_values` is `["-"]` and `key` is `key1`. `{"key1": "-", "key2": "value2"}` will parse into `{"key2": "value2"}` ## List-to-map Processor A processor that converts a list of objects from an event, where each object has a key field, to a map of keys to objects. diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessor.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessor.java index 17b7074fc1..d8806bde6c 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessor.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessor.java @@ -16,12 +16,13 @@ import org.opensearch.dataprepper.typeconverter.TypeConverter; import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Objects; @DataPrepperPlugin(name = "convert_entry_type", pluginType = Processor.class, pluginConfigurationType = ConvertEntryTypeProcessorConfig.class) public class ConvertEntryTypeProcessor extends AbstractProcessor, Record> { - private final String key; + private final List convertEntryKeys; private final TypeConverter converter; private final String convertWhen; private final List nullValues; @@ -33,7 +34,7 @@ public ConvertEntryTypeProcessor(final PluginMetrics pluginMetrics, final ConvertEntryTypeProcessorConfig convertEntryTypeProcessorConfig, final ExpressionEvaluator expressionEvaluator) { super(pluginMetrics); - this.key = convertEntryTypeProcessorConfig.getKey(); + this.convertEntryKeys = getKeysToConvert(convertEntryTypeProcessorConfig); this.converter = convertEntryTypeProcessorConfig.getType().getTargetConverter(); this.convertWhen = convertEntryTypeProcessorConfig.getConvertWhen(); this.nullValues = convertEntryTypeProcessorConfig.getNullValues() @@ -50,11 +51,13 @@ public Collection> doExecute(final Collection> recor continue; } - Object keyVal = recordEvent.get(key, Object.class); - if (keyVal != null) { - recordEvent.delete(key); - if (!nullValues.contains(keyVal.toString())){ - recordEvent.put(key, this.converter.convert(keyVal)); + for(final String key : convertEntryKeys) { + Object keyVal = recordEvent.get(key, Object.class); + if (keyVal != null) { + recordEvent.delete(key); + if (!nullValues.contains(keyVal.toString())) { + recordEvent.put(key, this.converter.convert(keyVal)); + } } } } @@ -73,6 +76,25 @@ public boolean isReadyForShutdown() { @Override public void shutdown() { } + + private List getKeysToConvert(final ConvertEntryTypeProcessorConfig convertEntryTypeProcessorConfig) { + final String key = convertEntryTypeProcessorConfig.getKey(); + final List keys = convertEntryTypeProcessorConfig.getKeys(); + if (key == null && keys == null) { + throw new IllegalArgumentException("key and keys cannot both be null. One must be provided."); + } + if (key != null && keys != null) { + throw new IllegalArgumentException("key and keys cannot both be defined."); + } + if (key != null) { + if (key.isEmpty()) { + throw new IllegalArgumentException("key cannot be empty."); + } else { + return Collections.singletonList(key); + } + } + return keys; + } } diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorConfig.java index 983fe57fcf..16f53b324d 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorConfig.java @@ -6,16 +6,17 @@ package org.opensearch.dataprepper.plugins.processor.mutateevent; import com.fasterxml.jackson.annotation.JsonProperty; -import jakarta.validation.constraints.NotEmpty; import java.util.List; import java.util.Optional; public class ConvertEntryTypeProcessorConfig { @JsonProperty("key") - @NotEmpty private String key; + @JsonProperty("keys") + private List keys; + @JsonProperty("type") private TargetType type = TargetType.INTEGER; @@ -29,6 +30,8 @@ public String getKey() { return key; } + public List getKeys() { return keys; } + public TargetType getType() { return type; } diff --git a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorTests.java b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorTests.java index 02f8712deb..1bddb03718 100644 --- a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorTests.java +++ b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorTests.java @@ -26,6 +26,7 @@ import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.lenient; import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) @@ -51,8 +52,9 @@ static Record buildRecordWithEvent(final Map data) { @BeforeEach private void setup() { - when(mockConfig.getKey()).thenReturn(TEST_KEY); - when(mockConfig.getConvertWhen()).thenReturn(null); + lenient().when(mockConfig.getKey()).thenReturn(TEST_KEY); + lenient().when(mockConfig.getKeys()).thenReturn(null); + lenient().when(mockConfig.getConvertWhen()).thenReturn(null); } private Record getMessage(String message, String key, Object value) { @@ -196,4 +198,42 @@ void testNoConversionWhenConvertWhenIsFalse() { Event event = executeAndGetProcessedEvent(record); assertThat(event.get(TEST_KEY, Integer.class), equalTo(testValue)); } + + @Test + void testMultipleKeysConvertEntryTypeProcessor() { + Integer testValue = 123; + String expectedValue = testValue.toString(); + String testKey1 = UUID.randomUUID().toString(); + String testKey2 = UUID.randomUUID().toString(); + when(mockConfig.getKey()).thenReturn(null); + when(mockConfig.getKeys()).thenReturn(List.of(testKey1, testKey2)); + when(mockConfig.getType()).thenReturn(TargetType.fromOptionValue("string")); + final Map testData = new HashMap(); + testData.put("message", "testMessage"); + testData.put(testKey1, testValue); + testData.put(testKey2, testValue); + Record record = buildRecordWithEvent(testData); + typeConversionProcessor = new ConvertEntryTypeProcessor(pluginMetrics, mockConfig, expressionEvaluator); + Event event = executeAndGetProcessedEvent(record); + assertThat(event.get(testKey1, String.class), equalTo(expectedValue)); + assertThat(event.get(testKey2, String.class), equalTo(expectedValue)); + } + + @Test + void testKeyAndKeysBothNullConvertEntryTypeProcessor() { + when(mockConfig.getKey()).thenReturn(null); + assertThrows(IllegalArgumentException.class, () -> new ConvertEntryTypeProcessor(pluginMetrics, mockConfig, expressionEvaluator)); + } + + @Test + void testKeyAndKeysBothDefinedConvertEntryTypeProcessor() { + when(mockConfig.getKeys()).thenReturn(Collections.singletonList(TEST_KEY)); + assertThrows(IllegalArgumentException.class, () -> new ConvertEntryTypeProcessor(pluginMetrics, mockConfig, expressionEvaluator)); + } + + @Test + void testEmptyKeyConvertEntryTypeProcessor() { + when(mockConfig.getKey()).thenReturn(""); + assertThrows(IllegalArgumentException.class, () -> new ConvertEntryTypeProcessor(pluginMetrics, mockConfig, expressionEvaluator)); + } } diff --git a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessor_NullValueTests.java b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessor_NullValueTests.java index b156b2b3b6..5fd9df9bfb 100644 --- a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessor_NullValueTests.java +++ b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessor_NullValueTests.java @@ -22,6 +22,7 @@ import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.CoreMatchers.nullValue; import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.Mockito.lenient; import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) @@ -42,9 +43,10 @@ public class ConvertEntryTypeProcessor_NullValueTests { @BeforeEach private void setup() { - when(mockConfig.getKey()).thenReturn(TEST_KEY); - when(mockConfig.getType()).thenReturn(TargetType.fromOptionValue("integer")); - when(mockConfig.getConvertWhen()).thenReturn(null); + lenient().when(mockConfig.getKey()).thenReturn(TEST_KEY); + lenient().when(mockConfig.getKeys()).thenReturn(null); + lenient().when(mockConfig.getType()).thenReturn(TargetType.fromOptionValue("integer")); + lenient().when(mockConfig.getConvertWhen()).thenReturn(null); } private Event executeAndGetProcessedEvent(final Object testValue) { @@ -117,4 +119,23 @@ void testMultipleElementNullValues() { assertThat(event.get(TEST_KEY, Integer.class), equalTo(testNumber)); } + @Test + void testMultipleKeysNullValues() { + String testValue = "-"; + String testKey1 = UUID.randomUUID().toString(); + String testKey2 = UUID.randomUUID().toString(); + when(mockConfig.getKey()).thenReturn(null); + when(mockConfig.getKeys()).thenReturn(List.of(testKey1, testKey2)); + when(mockConfig.getNullValues()).thenReturn(Optional.of(List.of("-"))); + final Map testData = new HashMap(); + testData.put("message", "testMessage"); + testData.put(testKey1, testValue); + testData.put(testKey2, testValue); + Record record = buildRecordWithEvent(testData); + nullValuesProcessor = new ConvertEntryTypeProcessor(pluginMetrics, mockConfig, expressionEvaluator); + Event event = executeAndGetProcessedEvent(record); + assertThat(event.get(testKey1, String.class), nullValue()); + assertThat(event.get(testKey2, String.class), nullValue()); + } + } From 252fea94f2ae946d095c00fdadc2a3ff83b8b208 Mon Sep 17 00:00:00 2001 From: kkondaka <41027584+kkondaka@users.noreply.github.com> Date: Tue, 4 Jul 2023 21:25:09 -0700 Subject: [PATCH 02/10] Added Kafka config to support acknowledgments and MSK arn (#2976) * Added Kafka config to support acknowledgments and MSK arn Signed-off-by: Krishna Kondaka * Modified to use data-prepper-core in testImplementation Signed-off-by: Krishna Kondaka * Addressed review comments Signed-off-by: Krishna Kondaka * Addressed failing test Signed-off-by: Krishna Kondaka --------- Signed-off-by: Krishna Kondaka Co-authored-by: Krishna Kondaka --- .../kafka-plugins/build.gradle | 1 + .../kafka/configuration/AwsConfig.java | 19 +++ .../configuration/KafkaSourceConfig.java | 19 +++ .../consumer/KafkaSourceCustomConsumer.java | 138 ++++++++++++++---- .../consumer/TopicPartitionCommitTracker.java | 79 ++++++++++ .../plugins/kafka/source/KafkaSource.java | 35 +++-- .../kafka/configuration/AwsConfigTest.java | 42 ++++++ .../configuration/KafkaSourceConfigTest.java | 11 +- .../KafkaSourceCustomConsumerTest.java | 79 +++++++++- .../TopicPartitionCommitTrackerTest.java | 84 +++++++++++ .../plugins/kafka/source/KafkaSourceTest.java | 7 +- 11 files changed, 468 insertions(+), 46 deletions(-) create mode 100644 data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/AwsConfig.java create mode 100644 data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/TopicPartitionCommitTracker.java create mode 100644 data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/AwsConfigTest.java create mode 100644 data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/consumer/TopicPartitionCommitTrackerTest.java diff --git a/data-prepper-plugins/kafka-plugins/build.gradle b/data-prepper-plugins/kafka-plugins/build.gradle index 9baba37795..8335b761f9 100644 --- a/data-prepper-plugins/kafka-plugins/build.gradle +++ b/data-prepper-plugins/kafka-plugins/build.gradle @@ -26,6 +26,7 @@ dependencies { testImplementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.14.2' testImplementation project(':data-prepper-test-common') testImplementation project(':data-prepper-plugins:blocking-buffer') + testImplementation project(':data-prepper-core') testImplementation 'org.mockito:mockito-inline:4.1.0' testImplementation 'org.apache.kafka:kafka_2.13:3.4.0' testImplementation 'org.apache.kafka:kafka_2.13:3.4.0:test' diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/AwsConfig.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/AwsConfig.java new file mode 100644 index 0000000000..4f4fa91cb5 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/AwsConfig.java @@ -0,0 +1,19 @@ +/* + * 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.constraints.Size; + +public class AwsConfig { + @JsonProperty("msk_arn") + @Size(min = 20, max = 2048, message = "mskArn length should be between 20 and 2048 characters") + private String awsMskArn; + + public String getAwsMskArn() { + return awsMskArn; + } +} diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaSourceConfig.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaSourceConfig.java index d4ba49d14c..bf8645c200 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaSourceConfig.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaSourceConfig.java @@ -11,6 +11,7 @@ import jakarta.validation.constraints.Size; import java.util.List; +import java.time.Duration; /** * * A helper class that helps to read user configuration values from @@ -18,6 +19,7 @@ */ public class KafkaSourceConfig { + public static final Duration DEFAULT_ACKNOWLEDGEMENTS_TIMEOUT = Duration.ofSeconds(30); @JsonProperty("bootstrap_servers") @NotNull @@ -36,6 +38,23 @@ public class KafkaSourceConfig { @JsonProperty("authentication") private AuthConfig authConfig; + @JsonProperty("aws") + private AwsConfig awsConfig; + + @JsonProperty("acknowledgments") + private Boolean acknowledgementsEnabled = false; + + @JsonProperty("acknowledgments_timeout") + private Duration acknowledgementsTimeout = DEFAULT_ACKNOWLEDGEMENTS_TIMEOUT; + + public Boolean getAcknowledgementsEnabled() { + return acknowledgementsEnabled; + } + + public Duration getAcknowledgementsTimeout() { + return acknowledgementsTimeout; + } + public List getTopics() { return topics; } diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaSourceCustomConsumer.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaSourceCustomConsumer.java index e5129e030e..6bd35391b9 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaSourceCustomConsumer.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaSourceCustomConsumer.java @@ -7,6 +7,7 @@ import com.fasterxml.jackson.core.JsonFactory; import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.databind.ObjectMapper; +import io.micrometer.core.instrument.Counter; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -19,7 +20,10 @@ import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig; import org.opensearch.dataprepper.buffer.common.BufferAccumulator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -31,8 +35,10 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat; +import org.apache.commons.lang3.Range; /** * * A utility class which will handle the core Kafka consumer operation. @@ -41,6 +47,10 @@ public class KafkaSourceCustomConsumer implements Runnable, ConsumerRebalanceLis private static final Logger LOG = LoggerFactory.getLogger(KafkaSourceCustomConsumer.class); private static final Long COMMIT_OFFSET_INTERVAL_MS = 300000L; private static final int DEFAULT_NUMBER_OF_RECORDS_TO_ACCUMULATE = 1; + static final String POSITIVE_ACKNOWLEDGEMENT_METRIC_NAME = "positiveAcknowledgementSetCounter"; + static final String NEGATIVE_ACKNOWLEDGEMENT_METRIC_NAME = "negativeAcknowledgementSetCounter"; + static final String DEFAULT_KEY = "message"; + private volatile long lastCommitTime; private KafkaConsumer consumer= null; private AtomicBoolean shutdownInProgress; @@ -53,12 +63,20 @@ public class KafkaSourceCustomConsumer implements Runnable, ConsumerRebalanceLis private static final ObjectMapper objectMapper = new ObjectMapper(); private final JsonFactory jsonFactory = new JsonFactory(); private Map offsetsToCommit; + private final AcknowledgementSetManager acknowledgementSetManager; + private final Map partitionCommitTrackerMap; + private final Counter positiveAcknowledgementSetCounter; + private final Counter negativeAcknowledgementSetCounter; + private final boolean acknowledgementsEnabled; + private final Duration acknowledgementsTimeout; public KafkaSourceCustomConsumer(final KafkaConsumer consumer, final AtomicBoolean shutdownInProgress, final Buffer> buffer, + final KafkaSourceConfig sourceConfig, final TopicConfig topicConfig, final String schemaType, + final AcknowledgementSetManager acknowledgementSetManager, final PluginMetrics pluginMetrics) { this.topicName = topicConfig.getName(); this.topicConfig = topicConfig; @@ -66,20 +84,92 @@ public KafkaSourceCustomConsumer(final KafkaConsumer consumer, this.consumer = consumer; this.buffer = buffer; this.offsetsToCommit = new HashMap<>(); + this.acknowledgementsTimeout = sourceConfig.getAcknowledgementsTimeout(); + // If the timeout value is different from default value, then enable acknowledgements automatically. + this.acknowledgementsEnabled = sourceConfig.getAcknowledgementsEnabled() || acknowledgementsTimeout != KafkaSourceConfig.DEFAULT_ACKNOWLEDGEMENTS_TIMEOUT; + this.acknowledgementSetManager = acknowledgementSetManager; this.pluginMetrics = pluginMetrics; - schema = MessageFormat.getByMessageFormatByName(schemaType); + this.partitionCommitTrackerMap = new HashMap<>(); + this.schema = MessageFormat.getByMessageFormatByName(schemaType); Duration bufferTimeout = Duration.ofSeconds(1); - bufferAccumulator = BufferAccumulator.create(buffer, DEFAULT_NUMBER_OF_RECORDS_TO_ACCUMULATE, bufferTimeout); - lastCommitTime = System.currentTimeMillis(); + this.bufferAccumulator = BufferAccumulator.create(buffer, DEFAULT_NUMBER_OF_RECORDS_TO_ACCUMULATE, bufferTimeout); + this.lastCommitTime = System.currentTimeMillis(); + this.positiveAcknowledgementSetCounter = pluginMetrics.counter(POSITIVE_ACKNOWLEDGEMENT_METRIC_NAME); + this.negativeAcknowledgementSetCounter = pluginMetrics.counter(NEGATIVE_ACKNOWLEDGEMENT_METRIC_NAME); + } + + public void updateOffsetsToCommit(final TopicPartition partition, final OffsetAndMetadata offsetAndMetadata) { + if (Objects.isNull(offsetAndMetadata)) { + return; + } + synchronized (this) { + offsetsToCommit.put(partition, offsetAndMetadata); + } + } + + private AcknowledgementSet createAcknowledgementSet(Map> offsets) { + AcknowledgementSet acknowledgementSet = + acknowledgementSetManager.create((result) -> { + if (result == true) { + positiveAcknowledgementSetCounter.increment(); + offsets.forEach((partition, offsetRange) -> { + int partitionId = partition.partition(); + if (!partitionCommitTrackerMap.containsKey(partitionId)) { + OffsetAndMetadata committedOffsetAndMetadata = consumer.committed(partition); + Long committedOffset = Objects.nonNull(committedOffsetAndMetadata) ? committedOffsetAndMetadata.offset() : null; + + partitionCommitTrackerMap.put(partitionId, new TopicPartitionCommitTracker(partition, committedOffset)); + } + OffsetAndMetadata offsetAndMetadata = partitionCommitTrackerMap.get(partitionId).addCompletedOffsets(offsetRange); + updateOffsetsToCommit(partition, offsetAndMetadata); + }); + } else { + positiveAcknowledgementSetCounter.increment(); + } + }, acknowledgementsTimeout); + return acknowledgementSet; + } + + double getPositiveAcknowledgementsCount() { + return positiveAcknowledgementSetCounter.count(); } public void consumeRecords() throws Exception { ConsumerRecords records = consumer.poll(topicConfig.getThreadWaitingTime().toMillis()/2); if (!records.isEmpty() && records.count() > 0) { - Map offsets = iterateRecordPartitions(records); - offsets.forEach((partition, offset) -> - offsetsToCommit.put(partition, offset)); + Map> offsets = new HashMap<>(); + AcknowledgementSet acknowledgementSet = null; + if (acknowledgementsEnabled) { + acknowledgementSet = createAcknowledgementSet(offsets); + } + iterateRecordPartitions(records, acknowledgementSet, offsets); + if (!acknowledgementsEnabled) { + offsets.forEach((partition, offsetRange) -> + updateOffsetsToCommit(partition, new OffsetAndMetadata(offsetRange.getMaximum() + 1))); + } + } + } + + private void commitOffsets() { + if (topicConfig.getAutoCommit()) { + return; + } + long currentTimeMillis = System.currentTimeMillis(); + if ((currentTimeMillis - lastCommitTime) < COMMIT_OFFSET_INTERVAL_MS) { + return; + } + synchronized (this) { + if (offsetsToCommit.isEmpty()) { + return; + } + try { + consumer.commitSync(); + offsetsToCommit.clear(); + lastCommitTime = currentTimeMillis; + } catch (CommitFailedException e) { + LOG.error("Failed to commit offsets in topic "+topicName); + } } } @@ -93,17 +183,7 @@ public void run() { consumer.subscribe(Arrays.asList(topicName)); while (!shutdownInProgress.get()) { consumeRecords(); - long currentTimeMillis = System.currentTimeMillis(); - if (!topicConfig.getAutoCommit() && !offsetsToCommit.isEmpty() && - (currentTimeMillis - lastCommitTime) >= COMMIT_OFFSET_INTERVAL_MS) { - try { - consumer.commitSync(offsetsToCommit); - offsetsToCommit.clear(); - lastCommitTime = currentTimeMillis; - } catch (CommitFailedException e) { - LOG.error("Failed to commit offsets in topic "+topicName); - } - } + commitOffsets(); } } catch (Exception exp) { LOG.error("Error while reading the records from the topic...", exp); @@ -113,25 +193,29 @@ public void run() { private Record getRecord(ConsumerRecord consumerRecord) { Map data = new HashMap<>(); Event event; + Object value; + String key = (String)consumerRecord.key(); + if (Objects.isNull(key)) { + key = DEFAULT_KEY; + } if (schema == MessageFormat.JSON || schema == MessageFormat.AVRO) { - Map message = new HashMap<>(); + value = new HashMap<>(); try { final JsonParser jsonParser = jsonFactory.createParser((String)consumerRecord.value().toString()); - message = objectMapper.readValue(jsonParser, Map.class); + value = objectMapper.readValue(jsonParser, Map.class); } catch (Exception e){ LOG.error("Failed to parse JSON or AVRO record"); return null; } - data.put(consumerRecord.key(), message); } else { - data.put(consumerRecord.key(), (String)consumerRecord.value()); + value = (String)consumerRecord.value(); } + data.put(key, value); event = JacksonLog.builder().withData(data).build(); return new Record(event); } - private Map iterateRecordPartitions(ConsumerRecords records) throws Exception { - Map offsets = new HashMap<>(); + private void iterateRecordPartitions(ConsumerRecords records, final AcknowledgementSet acknowledgementSet, Map> offsets) throws Exception { for (TopicPartition topicPartition : records.partitions()) { List> kafkaRecords = new ArrayList<>(); List> partitionRecords = records.records(topicPartition); @@ -139,12 +223,16 @@ private Map iterateRecordPartitions(Consu Record record = getRecord(consumerRecord); if (record != null) { bufferAccumulator.add(record); + if (acknowledgementSet != null) { + acknowledgementSet.add(record.getData()); + } } } long lastOffset = partitionRecords.get(partitionRecords.size() - 1).offset(); - offsets.put(topicPartition, new OffsetAndMetadata(lastOffset + 1)); + long firstOffset = partitionRecords.get(0).offset(); + Range offsetRange = Range.between(firstOffset, lastOffset); + offsets.put(topicPartition, offsetRange); } - return offsets; } public void closeConsumer(){ diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/TopicPartitionCommitTracker.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/TopicPartitionCommitTracker.java new file mode 100644 index 0000000000..9d10b46611 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/TopicPartitionCommitTracker.java @@ -0,0 +1,79 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.consumer; + +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +import org.apache.commons.lang3.Range; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; + +public class TopicPartitionCommitTracker { + private long committedOffset; + private final TopicPartition topicPartition; + private final Map> offsetMaxMap; + private final Map> offsetMinMap; + + public TopicPartitionCommitTracker(final TopicPartition topicPartition, Long committedOffset) { + this.topicPartition = topicPartition; + this.committedOffset = Objects.nonNull(committedOffset) ? committedOffset : -1L; + this.offsetMaxMap = new HashMap<>(); + this.offsetMinMap = new HashMap<>(); + this.offsetMaxMap.put(this.committedOffset, Range.between(this.committedOffset, this.committedOffset)); + } + + public TopicPartitionCommitTracker(final String topic, final int partition, Long committedOffset) { + this(new TopicPartition(topic, partition), committedOffset); + } + + public OffsetAndMetadata addCompletedOffsets(final Range offsetRange) { + Long min = offsetRange.getMinimum(); + Long max = offsetRange.getMaximum(); + boolean merged = false; + if (offsetMaxMap.containsKey(min - 1)) { + Range entry = offsetMaxMap.get(min - 1); + offsetMaxMap.remove(min - 1); + offsetMinMap.remove(entry.getMinimum()); + min = entry.getMinimum(); + Range newEntry = Range.between(min, max); + offsetMaxMap.put(max, newEntry); + offsetMinMap.put(min, newEntry); + merged = true; + } + if (offsetMinMap.containsKey(max + 1)) { + Range entry = offsetMinMap.get(max + 1); + offsetMinMap.remove(max + 1); + if (merged) { + offsetMinMap.remove(min); + offsetMaxMap.remove(max); + } + max = entry.getMaximum(); + offsetMaxMap.remove(max); + Range newEntry = Range.between(min, max); + offsetMaxMap.put(max, newEntry); + offsetMinMap.put(min, newEntry); + merged = true; + } + if (!merged) { + offsetMaxMap.put(max, offsetRange); + offsetMinMap.put(min, offsetRange); + return null; + } + if (offsetMinMap.containsKey(committedOffset)) { + Long maxValue = offsetMinMap.get(committedOffset).getMaximum(); + if (maxValue != committedOffset) { + offsetMinMap.remove(committedOffset); + committedOffset = maxValue; + offsetMaxMap.put(committedOffset, Range.between(committedOffset, committedOffset)); + return new OffsetAndMetadata(committedOffset + 1); + } + } + return null; + } + +} 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 9a1d96023f..6c69c259c2 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 @@ -22,6 +22,7 @@ import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.source.Source; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; import org.opensearch.dataprepper.plugins.kafka.consumer.KafkaSourceCustomConsumer; @@ -66,12 +67,16 @@ public class KafkaSource implements Source> { private String pipelineName; private String schemaType = MessageFormat.PLAINTEXT.toString(); private static final String SCHEMA_TYPE= "schemaType"; + private final AcknowledgementSetManager acknowledgementSetManager; @DataPrepperPluginConstructor - public KafkaSource(final KafkaSourceConfig sourceConfig, final PluginMetrics pluginMetrics, + public KafkaSource(final KafkaSourceConfig sourceConfig, + final PluginMetrics pluginMetrics, + final AcknowledgementSetManager acknowledgementSetManager, final PipelineDescription pipelineDescription) { this.sourceConfig = sourceConfig; this.pluginMetrics = pluginMetrics; + this.acknowledgementSetManager = acknowledgementSetManager; this.pipelineName = pipelineDescription.getPipelineName(); this.kafkaWorkerThreadProcessingErrors = pluginMetrics.counter(KAFKA_WORKER_THREAD_PROCESSING_ERRORS); shutdownInProgress = new AtomicBoolean(false); @@ -82,25 +87,25 @@ public void start(Buffer> buffer) { sourceConfig.getTopics().forEach(topic -> { Properties consumerProperties = getConsumerProperties(topic); MessageFormat schema = MessageFormat.getByMessageFormatByName(schemaType); - KafkaConsumer kafkaConsumer; - switch (schema) { - case JSON: - kafkaConsumer = new KafkaConsumer(consumerProperties); - break; - case AVRO: - kafkaConsumer = new KafkaConsumer(consumerProperties); - break; - case PLAINTEXT: - default: - kafkaConsumer = new KafkaConsumer(consumerProperties); - break; - } try { int numWorkers = topic.getWorkers(); executorService = Executors.newFixedThreadPool(numWorkers); IntStream.range(0, numWorkers + 1).forEach(index -> { - consumer = new KafkaSourceCustomConsumer(kafkaConsumer, shutdownInProgress, buffer, topic, schemaType, pluginMetrics); + KafkaConsumer kafkaConsumer; + switch (schema) { + case JSON: + kafkaConsumer = new KafkaConsumer(consumerProperties); + break; + case AVRO: + kafkaConsumer = new KafkaConsumer(consumerProperties); + break; + case PLAINTEXT: + default: + kafkaConsumer = new KafkaConsumer(consumerProperties); + break; + } + consumer = new KafkaSourceCustomConsumer(kafkaConsumer, shutdownInProgress, buffer, sourceConfig, topic, schemaType, acknowledgementSetManager, pluginMetrics); executorService.submit(consumer); }); diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/AwsConfigTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/AwsConfigTest.java new file mode 100644 index 0000000000..d3facb361f --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/AwsConfigTest.java @@ -0,0 +1,42 @@ +/* + * 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 java.lang.reflect.Field; +import java.util.UUID; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; + +class AwsConfigTest { + + private AwsConfig awsConfig; + + @BeforeEach + void setUp() { + awsConfig = new AwsConfig(); + } + + @Test + void getMskArn_notNull() throws NoSuchFieldException, IllegalAccessException { + final String testArn = UUID.randomUUID().toString(); + reflectivelySetField(awsConfig, "awsMskArn", testArn); + assertThat(awsConfig.getAwsMskArn(), equalTo(testArn)); + } + + private void reflectivelySetField(final AwsConfig awsConfig, final String fieldName, final Object value) throws NoSuchFieldException, IllegalAccessException { + final Field field = AwsConfig.class.getDeclaredField(fieldName); + try { + field.setAccessible(true); + field.set(awsConfig, value); + } finally { + field.setAccessible(false); + } + } +} diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaSourceConfigTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaSourceConfigTest.java index e7e23c9a04..552baed388 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaSourceConfigTest.java +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/KafkaSourceConfigTest.java @@ -17,6 +17,7 @@ import java.util.Arrays; import java.util.Map; import java.util.stream.Collectors; +import java.time.Duration; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.hasItem; @@ -24,6 +25,7 @@ import static org.hamcrest.Matchers.notNullValue; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.mockito.Mockito.mock; +import static org.opensearch.dataprepper.test.helper.ReflectivelySetField.setField; class KafkaSourceConfigTest { @@ -70,11 +72,13 @@ void test_bootStrapServers_not_null(){ @Test void test_topics_not_null(){ + assertEquals(false, kafkaSourceConfig.getAcknowledgementsEnabled()); + assertEquals(KafkaSourceConfig.DEFAULT_ACKNOWLEDGEMENTS_TIMEOUT, kafkaSourceConfig.getAcknowledgementsTimeout()); assertThat(kafkaSourceConfig.getTopics(), notNullValue()); } @Test - void test_setters(){ + void test_setters() throws NoSuchFieldException, IllegalAccessException { kafkaSourceConfig = new KafkaSourceConfig(); kafkaSourceConfig.setBootStrapServers(new ArrayList<>(Arrays.asList("127.0.0.1:9092"))); TopicConfig topicConfig = mock(TopicConfig.class); @@ -82,5 +86,10 @@ void test_setters(){ assertEquals(Arrays.asList("127.0.0.1:9092"), kafkaSourceConfig.getBootStrapServers()); assertEquals(Collections.singletonList(topicConfig), kafkaSourceConfig.getTopics()); + setField(KafkaSourceConfig.class, kafkaSourceConfig, "acknowledgementsEnabled", true); + Duration testTimeout = Duration.ofSeconds(10); + setField(KafkaSourceConfig.class, kafkaSourceConfig, "acknowledgementsTimeout", testTimeout); + assertEquals(true, kafkaSourceConfig.getAcknowledgementsEnabled()); + assertEquals(testTimeout, kafkaSourceConfig.getAcknowledgementsTimeout()); } } diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaSourceCustomConsumerTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaSourceCustomConsumerTest.java index 22172aece5..4f4f35d23e 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaSourceCustomConsumerTest.java +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaSourceCustomConsumerTest.java @@ -1,3 +1,8 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + package org.opensearch.dataprepper.plugins.kafka.consumer; import com.fasterxml.jackson.databind.ObjectMapper; @@ -16,11 +21,17 @@ import org.opensearch.dataprepper.plugins.buffer.blockingbuffer.BlockingBuffer; import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.acknowledgements.DefaultAcknowledgementSetManager; +import io.micrometer.core.instrument.Counter; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import static org.mockito.Mockito.doAnswer; import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.any; import static org.hamcrest.Matchers.hasEntry; import static org.hamcrest.MatcherAssert.assertThat; import org.junit.jupiter.api.Assertions; @@ -40,6 +51,8 @@ import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; import java.time.Duration; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; @ExtendWith(MockitoExtension.class) @MockitoSettings(strictness = Strictness.LENIENT) @@ -55,6 +68,9 @@ public class KafkaSourceCustomConsumerTest { @Mock private KafkaSourceConfig sourceConfig; + private ExecutorService callbackExecutor; + private AcknowledgementSetManager acknowledgementSetManager; + @Mock private TopicConfig topicConfig; @@ -77,22 +93,33 @@ public class KafkaSourceCustomConsumerTest { private final String testJsonValue2 = "{ \"key3\": \"value3\", \"key4\": false}"; private final int testPartition = 0; private final int testJsonPartition = 1; + private Counter counter; @BeforeEach public void setUp() { kafkaConsumer = mock(KafkaConsumer.class); pluginMetrics = mock(PluginMetrics.class); + counter = mock(Counter.class); topicConfig = mock(TopicConfig.class); when(topicConfig.getThreadWaitingTime()).thenReturn(Duration.ofSeconds(1)); when(topicConfig.getAutoCommit()).thenReturn(false); + when(kafkaConsumer.committed(any(TopicPartition.class))).thenReturn(null); + + when(pluginMetrics.counter(anyString())).thenReturn(counter); + doAnswer((i)-> {return null;}).when(counter).increment(); + callbackExecutor = Executors.newFixedThreadPool(2); + acknowledgementSetManager = new DefaultAcknowledgementSetManager(callbackExecutor, Duration.ofMillis(2000)); + sourceConfig = mock(KafkaSourceConfig.class); buffer = getBuffer(); shutdownInProgress = new AtomicBoolean(false); when(topicConfig.getName()).thenReturn("topic1"); } - public KafkaSourceCustomConsumer createObjectUnderTest(String schemaType) { - return new KafkaSourceCustomConsumer(kafkaConsumer, shutdownInProgress, buffer, topicConfig, schemaType, pluginMetrics); + public KafkaSourceCustomConsumer createObjectUnderTest(String schemaType, boolean acknowledgementsEnabled) { + when(sourceConfig.getAcknowledgementsEnabled()).thenReturn(acknowledgementsEnabled); + when(sourceConfig.getAcknowledgementsTimeout()).thenReturn(KafkaSourceConfig.DEFAULT_ACKNOWLEDGEMENTS_TIMEOUT); + return new KafkaSourceCustomConsumer(kafkaConsumer, shutdownInProgress, buffer, sourceConfig, topicConfig, schemaType, acknowledgementSetManager, pluginMetrics); } private BlockingBuffer> getBuffer() { @@ -109,7 +136,7 @@ public void testPlainTextConsumeRecords() throws InterruptedException { String topic = topicConfig.getName(); consumerRecords = createPlainTextRecords(topic); when(kafkaConsumer.poll(anyLong())).thenReturn(consumerRecords); - consumer = createObjectUnderTest("plaintext"); + consumer = createObjectUnderTest("plaintext", false); try { consumer.consumeRecords(); @@ -118,6 +145,7 @@ public void testPlainTextConsumeRecords() throws InterruptedException { ArrayList> bufferedRecords = new ArrayList<>(bufferRecords.getKey()); Assertions.assertEquals(consumerRecords.count(), bufferedRecords.size()); Map offsetsToCommit = consumer.getOffsetsToCommit(); + Assertions.assertEquals(offsetsToCommit.size(), 1); offsetsToCommit.forEach((topicPartition, offsetAndMetadata) -> { Assertions.assertEquals(topicPartition.partition(), testPartition); Assertions.assertEquals(topicPartition.topic(), topic); @@ -138,12 +166,55 @@ public void testPlainTextConsumeRecords() throws InterruptedException { } } + @Test + public void testPlainTextConsumeRecordsWithAcknowledgements() throws InterruptedException { + String topic = topicConfig.getName(); + consumerRecords = createPlainTextRecords(topic); + when(kafkaConsumer.poll(anyLong())).thenReturn(consumerRecords); + consumer = createObjectUnderTest("plaintext", true); + + try { + consumer.consumeRecords(); + } catch (Exception e){} + final Map.Entry>, CheckpointState> bufferRecords = buffer.read(1000); + ArrayList> bufferedRecords = new ArrayList<>(bufferRecords.getKey()); + Assertions.assertEquals(consumerRecords.count(), bufferedRecords.size()); + Map offsetsToCommit = consumer.getOffsetsToCommit(); + Assertions.assertEquals(offsetsToCommit.size(), 0); + + for (Record record: bufferedRecords) { + Event event = record.getData(); + String value1 = event.get(testKey1, String.class); + String value2 = event.get(testKey2, String.class); + assertTrue(value1 != null || value2 != null); + if (value1 != null) { + Assertions.assertEquals(value1, testValue1); + } + if (value2 != null) { + Assertions.assertEquals(value2, testValue2); + } + event.getEventHandle().release(true); + } + // Wait for acknowledgement callback function to run + try { + Thread.sleep(10000); + } catch (Exception e){} + + offsetsToCommit = consumer.getOffsetsToCommit(); + Assertions.assertEquals(offsetsToCommit.size(), 1); + offsetsToCommit.forEach((topicPartition, offsetAndMetadata) -> { + Assertions.assertEquals(topicPartition.partition(), testPartition); + Assertions.assertEquals(topicPartition.topic(), topic); + Assertions.assertEquals(offsetAndMetadata.offset(), 2L); + }); + } + @Test public void testJsonConsumeRecords() throws InterruptedException, Exception { String topic = topicConfig.getName(); consumerRecords = createJsonRecords(topic); when(kafkaConsumer.poll(anyLong())).thenReturn(consumerRecords); - consumer = createObjectUnderTest("json"); + consumer = createObjectUnderTest("json", false); consumer.consumeRecords(); final Map.Entry>, CheckpointState> bufferRecords = buffer.read(1000); diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/consumer/TopicPartitionCommitTrackerTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/consumer/TopicPartitionCommitTrackerTest.java new file mode 100644 index 0000000000..387ffb909e --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/consumer/TopicPartitionCommitTrackerTest.java @@ -0,0 +1,84 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.consumer; + +import org.apache.commons.lang3.Range; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.mockito.junit.jupiter.MockitoExtension; + +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; + + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.stream.Stream; + +@ExtendWith(MockitoExtension.class) +class TopicPartitionCommitTrackerTest { + private final String testTopic = "test_topic"; + private final int testPartition = 1; + private TopicPartitionCommitTracker topicPartitionCommitTracker; + public TopicPartitionCommitTracker createObjectUnderTest(String topic, int partition, Long offset) { + return new TopicPartitionCommitTracker(topic, partition, offset); + } + + @ParameterizedTest + @MethodSource("getInputOrder") + public void test(List order) { + topicPartitionCommitTracker = createObjectUnderTest(testTopic, testPartition, -1L); + List> ranges = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + ranges.add(Range.between(i*10L, i*10L+9L)); + } + OffsetAndMetadata result = null; + Long expectedOffset = 10L; + for (Integer i: order) { + result = topicPartitionCommitTracker.addCompletedOffsets(ranges.get(i)); + if (ranges.get(i).getMaximum() == (expectedOffset - 1)) { + assertThat(result.offset(), greaterThanOrEqualTo(expectedOffset)); + expectedOffset = result.offset() + 10L; + } + } + assertTrue(Objects.nonNull(result)); + assertThat(result.offset(), equalTo(100L)); + } + + private static Stream getInputOrder() { + List> orderList = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + List order = new ArrayList<>(); + for (int j = 0; j < 10; j++) { + order.add(j); + } + Collections.shuffle(order); + orderList.add(order); + } + return Stream.of( + Arguments.of(List.of(0, 1, 2, 3, 4, 5, 6, 7, 8, 9)), + Arguments.of(List.of(9, 8, 7, 6, 5, 4, 3, 2, 1, 0)), + Arguments.of(orderList.get(0)), + Arguments.of(orderList.get(1)), + Arguments.of(orderList.get(2)), + Arguments.of(orderList.get(3)), + Arguments.of(orderList.get(4)), + Arguments.of(orderList.get(5)), + Arguments.of(orderList.get(6)), + Arguments.of(orderList.get(7)), + Arguments.of(orderList.get(8)), + Arguments.of(orderList.get(9)) + ); + } + +} diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceTest.java index ba08ec5dbd..2dbe126691 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceTest.java +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceTest.java @@ -14,6 +14,7 @@ import org.opensearch.dataprepper.plugins.kafka.configuration.SchemaConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.PlainTextAuthConfig; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; @@ -44,6 +45,9 @@ class KafkaSourceTest { @Mock private SchemaConfig schemaConfig; + @Mock + private AcknowledgementSetManager acknowledgementSetManager; + @Mock private TopicConfig topicConfig; @Mock @@ -61,7 +65,7 @@ class KafkaSourceTest { public KafkaSource createObjectUnderTest() { - return new KafkaSource(sourceConfig, pluginMetrics, pipelineDescription); + return new KafkaSource(sourceConfig, pluginMetrics, acknowledgementSetManager, pipelineDescription); } @BeforeEach @@ -69,6 +73,7 @@ void setUp() throws Exception { sourceConfig = mock(KafkaSourceConfig.class); pipelineDescription = mock(PipelineDescription.class); pluginMetrics = mock(PluginMetrics.class); + acknowledgementSetManager = mock(AcknowledgementSetManager.class); when(topic1.getName()).thenReturn("topic1"); when(topic2.getName()).thenReturn("topic2"); when(topic1.getWorkers()).thenReturn(2); From 3231313b998a62e91c7e127f2a1b8429bbe6bafe Mon Sep 17 00:00:00 2001 From: Omkar <133762828+omkarmmore95@users.noreply.github.com> Date: Wed, 5 Jul 2023 21:23:44 +0530 Subject: [PATCH 03/10] -Support for Sink Codecs (#2881) -Support for Sink Codecs Signed-off-by: omkarmmore95 --- .../dataprepper/model/codec/OutputCodec.java | 21 ++++- .../model/codec/OutputCodecTest.java | 69 +++++++++++++++ .../plugins/codec/avro/AvroOutputCodec.java | 2 +- .../plugins/codec/csv/CsvOutputCodec.java | 2 +- data-prepper-plugins/newline-codecs/README.md | 57 ++++++++++++ .../newline/NewlineDelimitedOutputCodec.java | 51 +++++++++-- .../newline/NewlineDelimitedOutputConfig.java | 25 ++++++ .../NewlineDelimitedOutputCodecTest.java | 87 +++++++++++++++++++ .../codec/parquet/ParquetOutputCodec.java | 2 +- .../plugins/codec/json/JsonOutputCodec.java | 2 +- data-prepper-plugins/s3-sink/build.gradle | 2 + .../plugins/sink/S3SinkServiceIT.java | 45 +++++++--- .../dataprepper/plugins/sink/S3Sink.java | 6 +- .../plugins/sink/S3SinkService.java | 27 +++--- .../plugins/sink/accumulator/Buffer.java | 7 +- .../sink/accumulator/InMemoryBuffer.java | 25 +++--- .../sink/accumulator/LocalFileBuffer.java | 23 +++-- .../plugins/sink/accumulator/ObjectKey.java | 8 +- .../plugins/sink/S3SinkServiceTest.java | 67 +++++++++----- .../dataprepper/plugins/sink/S3SinkTest.java | 5 +- .../plugins/sink/ThresholdCheckTest.java | 17 +++- .../sink/accumulator/InMemoryBufferTest.java | 12 ++- .../sink/accumulator/LocalFileBufferTest.java | 11 ++- .../sink/accumulator/ObjectKeyTest.java | 9 +- 24 files changed, 476 insertions(+), 106 deletions(-) create mode 100644 data-prepper-api/src/test/java/org/opensearch/dataprepper/model/codec/OutputCodecTest.java create mode 100644 data-prepper-plugins/newline-codecs/README.md create mode 100644 data-prepper-plugins/newline-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/newline/NewlineDelimitedOutputConfig.java create mode 100644 data-prepper-plugins/newline-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/newline/NewlineDelimitedOutputCodecTest.java diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/codec/OutputCodec.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/codec/OutputCodec.java index 06056f5608..39c7a1490f 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/codec/OutputCodec.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/codec/OutputCodec.java @@ -5,14 +5,21 @@ package org.opensearch.dataprepper.model.codec; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.log.JacksonLog; import org.opensearch.dataprepper.model.sink.Sink; import java.io.IOException; import java.io.OutputStream; +import java.util.Map; public interface OutputCodec { + static final ObjectMapper objectMapper = new ObjectMapper(); + /** * this method get called from {@link Sink} to do initial wrapping in {@link OutputStream} * Implementors should do initial wrapping according to the implementation @@ -26,11 +33,12 @@ public interface OutputCodec { * this method get called from {@link Sink} to write event in {@link OutputStream} * Implementors should do get data from event and write to the {@link OutputStream} * - * @param event event Record event - * @param outputStream outputStream param to hold the event data + * @param event event Record event + * @param outputStream outputStream param to hold the event data + * @param tagsTargetKey to add tags to the record * @throws IOException throws IOException when not able to write data to {@link OutputStream} */ - void writeEvent(Event event, OutputStream outputStream) throws IOException; + void writeEvent(Event event, OutputStream outputStream, String tagsTargetKey) throws IOException; /** * this method get called from {@link Sink} to do final wrapping in {@link OutputStream} @@ -47,4 +55,11 @@ public interface OutputCodec { * @return String */ String getExtension(); + + default 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-api/src/test/java/org/opensearch/dataprepper/model/codec/OutputCodecTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/codec/OutputCodecTest.java new file mode 100644 index 0000000000..461907d9a4 --- /dev/null +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/codec/OutputCodecTest.java @@ -0,0 +1,69 @@ +package org.opensearch.dataprepper.model.codec; + +import com.fasterxml.jackson.core.JsonProcessingException; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.model.event.DefaultEventMetadata; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventMetadata; +import org.opensearch.dataprepper.model.event.EventType; +import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.log.JacksonLog; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.Arrays; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Set; +import java.util.UUID; + +import static org.junit.Assert.assertNotEquals; + +public class OutputCodecTest { + + @BeforeEach + public void setUp() { + } + + @Test + public void testWriteMetrics() throws JsonProcessingException { + OutputCodec outputCodec = new OutputCodec() { + @Override + public void start(OutputStream outputStream) throws IOException { + } + + @Override + public void writeEvent(Event event, OutputStream outputStream, String tagsTargetKey) throws IOException { + } + + @Override + public void complete(OutputStream outputStream) throws IOException { + } + + @Override + public String getExtension() { + return null; + } + }; + + final Set testTags = Set.of("tag1"); + final EventMetadata defaultEventMetadata = DefaultEventMetadata.builder(). + withEventType(EventType.LOG.toString()). + withTags(testTags).build(); + Map json = generateJson(); + final JacksonEvent event = JacksonLog.builder().withData(json).withEventMetadata(defaultEventMetadata).build(); + Event tagsToEvent = outputCodec.addTagsToEvent(event, "Tag"); + assertNotEquals(event.toJsonString(), tagsToEvent.toJsonString()); + } + + private static Map generateJson() { + final Map jsonObject = new LinkedHashMap<>(); + for (int i = 0; i < 2; i++) { + jsonObject.put(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + } + jsonObject.put(UUID.randomUUID().toString(), Arrays.asList(UUID.randomUUID().toString(), + UUID.randomUUID().toString(), UUID.randomUUID().toString())); + return jsonObject; + } +} diff --git a/data-prepper-plugins/avro-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/avro/AvroOutputCodec.java b/data-prepper-plugins/avro-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/avro/AvroOutputCodec.java index 081c6dd0e2..8129594162 100644 --- a/data-prepper-plugins/avro-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/avro/AvroOutputCodec.java +++ b/data-prepper-plugins/avro-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/avro/AvroOutputCodec.java @@ -31,7 +31,7 @@ public void start(final OutputStream outputStream) throws IOException { } @Override - public void writeEvent(final Event event, final OutputStream outputStream) throws IOException { + public void writeEvent(final Event event, final OutputStream outputStream,final String tagsTargetKey) throws IOException { // TODO: write event data to the outputstream } diff --git a/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/codec/csv/CsvOutputCodec.java b/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/codec/csv/CsvOutputCodec.java index 91a82454b2..ef3cc98225 100644 --- a/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/codec/csv/CsvOutputCodec.java +++ b/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/codec/csv/CsvOutputCodec.java @@ -30,7 +30,7 @@ public void start(final OutputStream outputStream) throws IOException { } @Override - public void writeEvent(final Event event, final OutputStream outputStream) throws IOException { + public void writeEvent(final Event event, final OutputStream outputStream, String tagsTargetKey) throws IOException { // TODO: validate data according to header and write event data to the outputstream } diff --git a/data-prepper-plugins/newline-codecs/README.md b/data-prepper-plugins/newline-codecs/README.md new file mode 100644 index 0000000000..094e57d40f --- /dev/null +++ b/data-prepper-plugins/newline-codecs/README.md @@ -0,0 +1,57 @@ +# Newline Delimited JSON Sink/Output Codec + +This is an implementation of Newline Sink Codec that parses the Dataprepper Events into Newline rows and writes them into the underlying OutputStream. + +## Usages + +Newline Output Codec can be configured with sink plugins (e.g. S3 Sink) in the Pipeline file. + +## Configuration Options + +``` +pipeline: + ... + sink: + - s3: + aws: + region: us-east-1 + sts_role_arn: arn:aws:iam::123456789012:role/Data-Prepper + sts_header_overrides: + max_retries: 5 + bucket: bucket_name + object_key: + path_prefix: my-elb/%{yyyy}/%{MM}/%{dd}/ + threshold: + event_count: 2000 + maximum_size: 50mb + event_collect_timeout: 15s + codec: + newline: + header_destination: header + exclude_keys: + - s3 + buffer_type: in_memory +``` + +## AWS Configuration + +### Codec Configuration: + +1) `header_destination`: The key corresponding to which the header value has to be placed by the codec. +2) `exclude_keys`: Those keys of the events that the user wants to exclude while converting them to newline rows. + + +## Developer Guide + +This plugin is compatible with Java 11. See below + +- [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) + +The integration tests for this plugin do not run as part of the Data Prepper build. + +The following command runs the integration tests: + +``` +./gradlew :data-prepper-plugins:s3-sink:integrationTest -Dtests.s3sink.region= -Dtests.s3sink.bucket= +``` diff --git a/data-prepper-plugins/newline-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/newline/NewlineDelimitedOutputCodec.java b/data-prepper-plugins/newline-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/newline/NewlineDelimitedOutputCodec.java index 46a0db6789..9e07f60e9b 100644 --- a/data-prepper-plugins/newline-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/newline/NewlineDelimitedOutputCodec.java +++ b/data-prepper-plugins/newline-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/newline/NewlineDelimitedOutputCodec.java @@ -4,42 +4,75 @@ */ package org.opensearch.dataprepper.plugins.codec.newline; +import com.fasterxml.jackson.databind.ObjectMapper; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; +import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.codec.OutputCodec; import org.opensearch.dataprepper.model.event.Event; import java.io.IOException; import java.io.OutputStream; +import java.util.Map; +import java.util.Objects; /** * An implementation of {@link OutputCodec} which deserializes Data-Prepper events * and writes them to Output Stream as ND-JSON data */ -@DataPrepperPlugin(name = "newline", pluginType = OutputCodec.class) +@DataPrepperPlugin(name = "newline", pluginType = OutputCodec.class, pluginConfigurationType = NewlineDelimitedOutputConfig.class) public class NewlineDelimitedOutputCodec implements OutputCodec { + private static final String NDJSON = "ndjson"; + private static final ObjectMapper objectMapper = new ObjectMapper(); + private final NewlineDelimitedOutputConfig config; + + @DataPrepperPluginConstructor + public NewlineDelimitedOutputCodec(final NewlineDelimitedOutputConfig config) { + Objects.requireNonNull(config); + this.config = config; + } @Override public void start(final OutputStream outputStream) throws IOException { - // TODO: implement + Objects.requireNonNull(outputStream); } @Override - public void writeEvent(final Event event, final OutputStream outputStream) throws IOException { - // TODO: get the event data and - // get the header record and message record and write event data to the outputstream + public void writeEvent(final Event event, final OutputStream outputStream, String tagsTargetKey) throws IOException { + Objects.requireNonNull(event); + Map eventMap; + if (tagsTargetKey != null) { + eventMap = addTagsToEvent(event, tagsTargetKey).toMap(); + } else { + eventMap = event.toMap(); + } + writeToOutputStream(outputStream, eventMap); } @Override public void complete(final OutputStream outputStream) throws IOException { - // TODO: Close the output stream + outputStream.close(); } - private void writeByteArrayToOutputStream(final OutputStream outputStream, final Object object) throws IOException { - // TODO: common method to write byte array data to OutputStream + private void writeToOutputStream(final OutputStream outputStream, final Object object) throws IOException { + byte[] byteArr = null; + if (object instanceof Map) { + Map map = objectMapper.convertValue(object, Map.class); + for (String key : config.getExcludeKeys()) { + if (map.containsKey(key)) { + map.remove(key); + } + } + String json = objectMapper.writeValueAsString(map); + byteArr = json.getBytes(); + } else { + byteArr = object.toString().getBytes(); + } + outputStream.write(byteArr); + outputStream.write(System.lineSeparator().getBytes()); } @Override public String getExtension() { - return null; + return NDJSON; } } diff --git a/data-prepper-plugins/newline-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/newline/NewlineDelimitedOutputConfig.java b/data-prepper-plugins/newline-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/newline/NewlineDelimitedOutputConfig.java new file mode 100644 index 0000000000..d3e91267e6 --- /dev/null +++ b/data-prepper-plugins/newline-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/newline/NewlineDelimitedOutputConfig.java @@ -0,0 +1,25 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.codec.newline; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.ArrayList; +import java.util.List; + +/** + * Configuration class for the newline delimited codec. + */ +public class NewlineDelimitedOutputConfig { + private static final List DEFAULT_EXCLUDE_KEYS = new ArrayList<>(); + + @JsonProperty("exclude_keys") + private List excludeKeys = DEFAULT_EXCLUDE_KEYS; + + public List getExcludeKeys() { + return excludeKeys; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/newline-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/newline/NewlineDelimitedOutputCodecTest.java b/data-prepper-plugins/newline-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/newline/NewlineDelimitedOutputCodecTest.java new file mode 100644 index 0000000000..b31ebaf6f0 --- /dev/null +++ b/data-prepper-plugins/newline-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/newline/NewlineDelimitedOutputCodecTest.java @@ -0,0 +1,87 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.codec.newline; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.hamcrest.Matchers; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.log.JacksonLog; +import org.opensearch.dataprepper.model.record.Record; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.MatcherAssert.assertThat; + + +public class NewlineDelimitedOutputCodecTest { + private ByteArrayOutputStream outputStream; + + private static NewlineDelimitedOutputConfig config; + + private static int numberOfRecords; + private static final String REGEX = "\\r?\\n"; + private static ObjectMapper objectMapper = new ObjectMapper(); + + private NewlineDelimitedOutputCodec createObjectUnderTest() { + config = new NewlineDelimitedOutputConfig(); + return new NewlineDelimitedOutputCodec(config); + } + + @ParameterizedTest + @ValueSource(ints = {1, 3, 10, 100}) + void test_happy_case(final int numberOfRecords) throws IOException { + this.numberOfRecords = numberOfRecords; + NewlineDelimitedOutputCodec newlineDelimitedOutputCodec = createObjectUnderTest(); + outputStream = new ByteArrayOutputStream(); + newlineDelimitedOutputCodec.start(outputStream); + for (int index = 0; index < numberOfRecords; index++) { + final Event event = (Event) getRecord(index).getData(); + newlineDelimitedOutputCodec.writeEvent(event, outputStream, null); + } + newlineDelimitedOutputCodec.complete(outputStream); + byte[] byteArray = outputStream.toByteArray(); + String jsonString = null; + try { + jsonString = new String(byteArray, StandardCharsets.UTF_8); + } catch (Exception e) { + e.printStackTrace(); + } + int index = 0; + List expectedRecords = generateRecords(numberOfRecords); + String[] jsonObjects = jsonString.split(REGEX); + for (String jsonObject : jsonObjects) { + Object expectedMap = expectedRecords.get(index); + Object actualMap = objectMapper.readValue(jsonObject, Map.class); + assertThat(expectedMap, Matchers.equalTo(actualMap)); + index++; + } + } + + private static Record getRecord(int index) { + List recordList = generateRecords(numberOfRecords); + final Event event = JacksonLog.builder().withData(recordList.get(index)).build(); + return new Record<>(event); + } + + private static List generateRecords(int numberOfRecords) { + + List recordList = new ArrayList<>(); + for (int rows = 0; rows < numberOfRecords; rows++) { + HashMap eventData = new HashMap<>(); + eventData.put("name", "Person" + rows); + eventData.put("age", rows); + recordList.add(eventData); + } + return recordList; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodec.java b/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodec.java index 2cbce41c7f..c8be600499 100644 --- a/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodec.java +++ b/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodec.java @@ -35,7 +35,7 @@ public void complete(final OutputStream outputStream) throws IOException { } @Override - public void writeEvent(final Event event, final OutputStream outputStream) throws IOException { + public void writeEvent(final Event event, final OutputStream outputStream, String tagsTargetKey) throws IOException { // TODO: get the event data and write in output stream } diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/codec/json/JsonOutputCodec.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/codec/json/JsonOutputCodec.java index e37be538c0..bc68761f07 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/codec/json/JsonOutputCodec.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/codec/json/JsonOutputCodec.java @@ -24,7 +24,7 @@ public void start(final OutputStream outputStream) throws IOException { } @Override - public void writeEvent(final Event event, final OutputStream outputStream) throws IOException { + public void writeEvent(final Event event, final OutputStream outputStream, String tagsTargetKey) throws IOException { // TODO: get the event data and write event data to the outputstream } diff --git a/data-prepper-plugins/s3-sink/build.gradle b/data-prepper-plugins/s3-sink/build.gradle index 6d8b44cdb8..4827a75b02 100644 --- a/data-prepper-plugins/s3-sink/build.gradle +++ b/data-prepper-plugins/s3-sink/build.gradle @@ -20,6 +20,8 @@ dependencies { implementation 'org.jetbrains.kotlin:kotlin-stdlib-common:1.8.21' implementation 'org.apache.commons:commons-lang3:3.12.0' testImplementation project(':data-prepper-test-common') + + implementation project(':data-prepper-plugins:newline-codecs') } test { diff --git a/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/S3SinkServiceIT.java b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/S3SinkServiceIT.java index cfa9d3657b..7468e6bf44 100644 --- a/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/S3SinkServiceIT.java +++ b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/S3SinkServiceIT.java @@ -14,16 +14,21 @@ import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.codec.OutputCodec; +import org.opensearch.dataprepper.model.event.DefaultEventMetadata; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventHandle; +import org.opensearch.dataprepper.model.event.EventMetadata; +import org.opensearch.dataprepper.model.event.EventType; import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.model.log.JacksonLog; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.types.ByteCount; +import org.opensearch.dataprepper.plugins.codec.newline.NewlineDelimitedOutputCodec; +import org.opensearch.dataprepper.plugins.codec.newline.NewlineDelimitedOutputConfig; import org.opensearch.dataprepper.plugins.sink.accumulator.BufferFactory; import org.opensearch.dataprepper.plugins.sink.accumulator.InMemoryBufferFactory; import org.opensearch.dataprepper.plugins.sink.accumulator.ObjectKey; -import org.opensearch.dataprepper.plugins.sink.codec.JsonCodec; import org.opensearch.dataprepper.plugins.sink.configuration.AwsAuthenticationOptions; import org.opensearch.dataprepper.plugins.sink.configuration.ObjectKeyOptions; import org.opensearch.dataprepper.plugins.sink.configuration.ThresholdOptions; @@ -37,12 +42,14 @@ import software.amazon.awssdk.services.s3.model.S3Object; import java.time.Duration; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.LinkedHashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.UUID; import static org.hamcrest.MatcherAssert.assertThat; @@ -57,7 +64,7 @@ class S3SinkServiceIT { private static final String PATH_PREFIX = UUID.randomUUID().toString() + "/%{yyyy}/%{MM}/%{dd}/"; private S3Client s3Client; private String bucketName; - private JsonCodec codec; + private BufferFactory bufferFactory; @Mock private S3SinkConfig s3SinkConfig; @@ -80,6 +87,11 @@ class S3SinkServiceIT { @Mock private DistributionSummary s3ObjectSizeSummary; + private OutputCodec codec; + + @Mock + NewlineDelimitedOutputConfig newlineDelimitedOutputConfig; + @BeforeEach public void setUp() { @@ -88,7 +100,6 @@ public void setUp() { s3Client = S3Client.builder().region(Region.of(s3region)).build(); bucketName = System.getProperty("tests.s3sink.bucket"); - codec = new JsonCodec(); bufferFactory = new InMemoryBufferFactory(); when(objectKeyOptions.getNamePattern()).thenReturn("elb-log-%{yyyy-MM-dd'T'hh-mm-ss}"); @@ -111,6 +122,7 @@ public void setUp() { @Test void verify_flushed_object_count_into_s3_bucket() { + configureNewLineCodec(); int s3ObjectCountBeforeIngest = gets3ObjectCount(); S3SinkService s3SinkService = createObjectUnderTest(); s3SinkService.output(setEventQueue()); @@ -118,23 +130,31 @@ void verify_flushed_object_count_into_s3_bucket() { assertThat(s3ObjectCountAfterIngest, equalTo(s3ObjectCountBeforeIngest + 1)); } - @Test - void verify_flushed_records_into_s3_bucket() { + void configureNewLineCodec() { + codec = new NewlineDelimitedOutputCodec(newlineDelimitedOutputConfig); + when(newlineDelimitedOutputConfig.getExcludeKeys()).thenReturn(new ArrayList<>()); + } + @Test + void verify_flushed_records_into_s3_bucketNewLine() { + configureNewLineCodec(); S3SinkService s3SinkService = createObjectUnderTest(); Collection> recordsData = setEventQueue(); s3SinkService.output(recordsData); String objectData = getS3Object(); + int count = 0; + String[] objectDataArr = objectData.split("\r\n"); for (Record recordData : recordsData) { String objectRecord = recordData.getData().toJsonString(); - assertThat(objectData, CoreMatchers.containsString(objectRecord)); + assertThat(objectDataArr[count], CoreMatchers.containsString(objectRecord)); + count++; } } private S3SinkService createObjectUnderTest() { - return new S3SinkService(s3SinkConfig, bufferFactory, codec, s3Client, null, pluginMetrics); + return new S3SinkService(s3SinkConfig, bufferFactory, codec, s3Client, "Tag", pluginMetrics); } private int gets3ObjectCount() { @@ -183,19 +203,24 @@ private Collection> setEventQueue() { } private static Record createRecord() { - Map json = generateJson(); - final JacksonEvent event = JacksonLog.builder().withData(json).build(); + final Set testTags = Set.of("tag1"); + final EventMetadata defaultEventMetadata = DefaultEventMetadata.builder(). + withEventType(EventType.LOG.toString()). + withTags(testTags).build(); + Map json = generateJson(testTags); + final JacksonEvent event = JacksonLog.builder().withData(json).withEventMetadata(defaultEventMetadata).build(); event.setEventHandle(mock(EventHandle.class)); return new Record<>(event); } - private static Map generateJson() { + private static Map generateJson(Set testTags) { final Map jsonObject = new LinkedHashMap<>(); for (int i = 0; i < 2; i++) { jsonObject.put(UUID.randomUUID().toString(), UUID.randomUUID().toString()); } jsonObject.put(UUID.randomUUID().toString(), Arrays.asList(UUID.randomUUID().toString(), UUID.randomUUID().toString(), UUID.randomUUID().toString())); + jsonObject.put("Tag", testTags.toArray()); return jsonObject; } } diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/S3Sink.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/S3Sink.java index a4baa538ae..828fd2e414 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/S3Sink.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/S3Sink.java @@ -8,6 +8,7 @@ import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; +import org.opensearch.dataprepper.model.codec.OutputCodec; import org.opensearch.dataprepper.model.configuration.PluginModel; import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.event.Event; @@ -21,7 +22,6 @@ import org.opensearch.dataprepper.plugins.sink.accumulator.BufferTypeOptions; import org.opensearch.dataprepper.plugins.sink.accumulator.InMemoryBufferFactory; import org.opensearch.dataprepper.plugins.sink.accumulator.LocalFileBufferFactory; -import org.opensearch.dataprepper.plugins.sink.codec.Codec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.services.s3.S3Client; @@ -38,7 +38,7 @@ public class S3Sink extends AbstractSink> { private static final Logger LOG = LoggerFactory.getLogger(S3Sink.class); private final S3SinkConfig s3SinkConfig; - private final Codec codec; + private final OutputCodec codec; private volatile boolean sinkInitialized; private final S3SinkService s3SinkService; private final BufferFactory bufferFactory; @@ -61,7 +61,7 @@ public S3Sink(final PluginSetting pluginSetting, final PluginModel codecConfiguration = s3SinkConfig.getCodec(); final PluginSetting codecPluginSettings = new PluginSetting(codecConfiguration.getPluginName(), codecConfiguration.getPluginSettings()); - codec = pluginFactory.loadPlugin(Codec.class, codecPluginSettings); + codec = pluginFactory.loadPlugin(OutputCodec.class, codecPluginSettings); sinkInitialized = Boolean.FALSE; if (s3SinkConfig.getBufferType().equals(BufferTypeOptions.LOCALFILE)) { diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/S3SinkService.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/S3SinkService.java index 34c49a9b25..0ed2e8bb79 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/S3SinkService.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/S3SinkService.java @@ -8,6 +8,7 @@ import io.micrometer.core.instrument.Counter; import io.micrometer.core.instrument.DistributionSummary; import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.codec.OutputCodec; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventHandle; import org.opensearch.dataprepper.model.record.Record; @@ -15,7 +16,6 @@ import org.opensearch.dataprepper.plugins.sink.accumulator.Buffer; import org.opensearch.dataprepper.plugins.sink.accumulator.BufferFactory; import org.opensearch.dataprepper.plugins.sink.accumulator.ObjectKey; -import org.opensearch.dataprepper.plugins.sink.codec.Codec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.awscore.exception.AwsServiceException; @@ -23,6 +23,7 @@ import software.amazon.awssdk.services.s3.S3Client; import java.io.IOException; +import java.io.OutputStream; import java.util.Collection; import java.util.LinkedList; import java.util.concurrent.locks.Lock; @@ -44,7 +45,7 @@ public class S3SinkService { private final Lock reentrantLock; private final BufferFactory bufferFactory; private final Collection bufferedEventHandles; - private final Codec codec; + private final OutputCodec codec; private final S3Client s3Client; private Buffer currentBuffer; private final int maxEvents; @@ -67,7 +68,7 @@ public class S3SinkService { * @param pluginMetrics metrics. */ public S3SinkService(final S3SinkConfig s3SinkConfig, final BufferFactory bufferFactory, - final Codec codec, final S3Client s3Client, final String tagsTargetKey, final PluginMetrics pluginMetrics) { + final OutputCodec codec, final S3Client s3Client, final String tagsTargetKey, final PluginMetrics pluginMetrics) { this.s3SinkConfig = s3SinkConfig; this.bufferFactory = bufferFactory; this.codec = codec; @@ -100,19 +101,25 @@ void output(Collection> records) { currentBuffer = bufferFactory.getBuffer(); } try { + OutputStream outputStream = currentBuffer.getOutputStream(); + for (Record record : records) { + if(currentBuffer.getEventCount() == 0) { + codec.start(outputStream); + } + final Event event = record.getData(); - final String encodedEvent; - encodedEvent = codec.parse(event, tagsTargetKey); - final byte[] encodedBytes = encodedEvent.getBytes(); + codec.writeEvent(event, outputStream, tagsTargetKey); + int count = currentBuffer.getEventCount() +1; + currentBuffer.setEventCount(count); - currentBuffer.writeEvent(encodedBytes); if(event.getEventHandle() != null) { bufferedEventHandles.add(event.getEventHandle()); } if (ThresholdCheck.checkThresholdExceed(currentBuffer, maxEvents, maxBytes, maxCollectionDuration)) { - final String s3Key = generateKey(); + codec.complete(outputStream); + final String s3Key = generateKey(codec); LOG.info("Writing {} to S3 with {} events and size of {} bytes.", s3Key, currentBuffer.getEventCount(), currentBuffer.getSize()); final boolean isFlushToS3 = retryFlushToS3(currentBuffer, s3Key); @@ -178,9 +185,9 @@ protected boolean retryFlushToS3(final Buffer currentBuffer, final String s3Key) * Generate the s3 object path prefix and object file name. * @return object key path. */ - protected String generateKey() { + protected String generateKey(OutputCodec codec) { final String pathPrefix = ObjectKey.buildingPathPrefix(s3SinkConfig); - final String namePattern = ObjectKey.objectFileName(s3SinkConfig); + final String namePattern = ObjectKey.objectFileName(s3SinkConfig, codec.getExtension()); return (!pathPrefix.isEmpty()) ? pathPrefix + namePattern : namePattern; } } diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/Buffer.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/Buffer.java index ebb3886eac..df09ea42e1 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/Buffer.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/Buffer.java @@ -6,7 +6,7 @@ package org.opensearch.dataprepper.plugins.sink.accumulator; import software.amazon.awssdk.services.s3.S3Client; -import java.io.IOException; +import java.io.OutputStream; /** * A buffer can hold data before flushing it to S3. @@ -23,5 +23,8 @@ public interface Buffer { long getDuration(); void flushToS3(S3Client s3Client, String bucket, String key) ; - void writeEvent(byte[] bytes) throws IOException; + + OutputStream getOutputStream(); + + void setEventCount(int eventCount); } diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/InMemoryBuffer.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/InMemoryBuffer.java index 370b1c1882..4a892233de 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/InMemoryBuffer.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/InMemoryBuffer.java @@ -9,8 +9,9 @@ import software.amazon.awssdk.core.sync.RequestBody; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.model.PutObjectRequest; + import java.io.ByteArrayOutputStream; -import java.io.IOException; +import java.io.OutputStream; import java.util.concurrent.TimeUnit; /** @@ -39,6 +40,15 @@ public int getEventCount() { return eventCount; } + @Override + public void setEventCount(int eventCount) { + this.eventCount = eventCount; + } + @Override + public OutputStream getOutputStream() { + return byteArrayOutputStream; + } + public long getDuration() { return watch.getTime(TimeUnit.SECONDS); } @@ -58,16 +68,5 @@ public void flushToS3(S3Client s3Client, String bucket, String key) { RequestBody.fromBytes(byteArray)); } - /** - * write byte array to output stream. - * - * @param bytes byte array. - * @throws IOException while writing to output stream fails. - */ - @Override - public void writeEvent(byte[] bytes) throws IOException { - byteArrayOutputStream.write(bytes); - byteArrayOutputStream.write(System.lineSeparator().getBytes()); - eventCount++; - } + } \ No newline at end of file diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/LocalFileBuffer.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/LocalFileBuffer.java index da5c9faa1a..843be56090 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/LocalFileBuffer.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/LocalFileBuffer.java @@ -75,18 +75,6 @@ public void flushToS3(S3Client s3Client, String bucket, String key) { removeTemporaryFile(); } - /** - * write byte array to output stream. - * @param bytes byte array. - * @throws IOException while writing to output stream fails. - */ - @Override - public void writeEvent(byte[] bytes) throws IOException { - outputStream.write(bytes); - outputStream.write(System.lineSeparator().getBytes()); - eventCount++; - } - /** * Flushing the buffered data into the output stream. */ @@ -111,4 +99,15 @@ protected void removeTemporaryFile() { } } } + + + @Override + public void setEventCount(int eventCount) { + this.eventCount = eventCount; + } + + @Override + public OutputStream getOutputStream() { + return outputStream; + } } \ No newline at end of file diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/ObjectKey.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/ObjectKey.java index b3e85d8381..4eb0a12923 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/ObjectKey.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/accumulator/ObjectKey.java @@ -50,16 +50,18 @@ public static String buildingPathPrefix(final S3SinkConfig s3SinkConfig) { *Get the object file name with the extension * * @param s3SinkConfig s3 sink configuration + * @param codecExtension extension * @return s3 object name with prefix */ - public static String objectFileName(S3SinkConfig s3SinkConfig) { + public static String objectFileName(S3SinkConfig s3SinkConfig, String codecExtension) { String configNamePattern = s3SinkConfig.getObjectKeyOptions().getNamePattern(); int extensionIndex = configNamePattern.lastIndexOf('.'); if (extensionIndex > 0) { return S3ObjectIndexUtility.getObjectNameWithDateTimeId(configNamePattern.substring(0, extensionIndex)) + "." - + configNamePattern.substring(extensionIndex + 1); + + (codecExtension!=null? codecExtension :configNamePattern.substring(extensionIndex + 1)); } else { - return S3ObjectIndexUtility.getObjectNameWithDateTimeId(configNamePattern) + "." + DEFAULT_CODEC_FILE_EXTENSION; + return S3ObjectIndexUtility.getObjectNameWithDateTimeId(configNamePattern) + "." + + (codecExtension!=null? codecExtension : DEFAULT_CODEC_FILE_EXTENSION); } } } \ No newline at end of file diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/S3SinkServiceTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/S3SinkServiceTest.java index 36302133a6..9c8a4e30ab 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/S3SinkServiceTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/S3SinkServiceTest.java @@ -10,6 +10,7 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.codec.OutputCodec; import org.opensearch.dataprepper.model.configuration.PluginModel; import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.event.Event; @@ -23,8 +24,6 @@ import org.opensearch.dataprepper.plugins.sink.accumulator.BufferTypeOptions; import org.opensearch.dataprepper.plugins.sink.accumulator.InMemoryBuffer; import org.opensearch.dataprepper.plugins.sink.accumulator.InMemoryBufferFactory; -import org.opensearch.dataprepper.plugins.sink.codec.Codec; -import org.opensearch.dataprepper.plugins.sink.codec.JsonCodec; import org.opensearch.dataprepper.plugins.sink.configuration.AwsAuthenticationOptions; import org.opensearch.dataprepper.plugins.sink.configuration.ObjectKeyOptions; import org.opensearch.dataprepper.plugins.sink.configuration.ThresholdOptions; @@ -34,6 +33,7 @@ import org.apache.commons.lang3.RandomStringUtils; import java.io.IOException; +import java.io.OutputStream; import java.time.Duration; import java.time.LocalDateTime; import java.time.ZoneId; @@ -78,7 +78,7 @@ class S3SinkServiceTest { public static final String PATH_PREFIX = "logdata/"; private S3SinkConfig s3SinkConfig; private S3Client s3Client; - private JsonCodec codec; + private OutputCodec codec; private PluginMetrics pluginMetrics; private BufferFactory bufferFactory; private Counter snapshotSuccessCounter; @@ -100,7 +100,7 @@ void setUp() { pluginMetrics = mock(PluginMetrics.class); PluginModel pluginModel = mock(PluginModel.class); PluginFactory pluginFactory = mock(PluginFactory.class); - codec = mock(JsonCodec.class); + codec = mock(OutputCodec.class); snapshotSuccessCounter = mock(Counter.class); Counter snapshotFailedCounter = mock(Counter.class); Counter numberOfRecordsSuccessCounter = mock(Counter.class); @@ -123,7 +123,7 @@ void setUp() { when(awsAuthenticationOptions.getAwsRegion()).thenReturn(Region.of(S3_REGION)); when(s3SinkConfig.getCodec()).thenReturn(pluginModel); when(pluginModel.getPluginName()).thenReturn(CODEC_PLUGIN_NAME); - when(pluginFactory.loadPlugin(Codec.class, pluginSetting)).thenReturn(codec); + when(pluginFactory.loadPlugin(OutputCodec.class, pluginSetting)).thenReturn(codec); lenient().when(pluginMetrics.counter(S3SinkService.OBJECTS_SUCCEEDED)).thenReturn(snapshotSuccessCounter); lenient().when(pluginMetrics.counter(S3SinkService.OBJECTS_FAILED)).thenReturn(snapshotFailedCounter); @@ -150,7 +150,7 @@ void test_generateKey_with_general_prefix() { String pathPrefix = "events/"; when(s3SinkConfig.getObjectKeyOptions().getPathPrefix()).thenReturn(pathPrefix); S3SinkService s3SinkService = createObjectUnderTest(); - String key = s3SinkService.generateKey(); + String key = s3SinkService.generateKey(codec); assertNotNull(key); assertThat(key, true); assertThat(key, key.contains(pathPrefix)); @@ -169,7 +169,7 @@ void test_generateKey_with_date_prefix() { when(s3SinkConfig.getObjectKeyOptions() .getPathPrefix()).thenReturn(pathPrefix + datePattern); S3SinkService s3SinkService = createObjectUnderTest(); - String key = s3SinkService.generateKey(); + String key = s3SinkService.generateKey(codec); assertNotNull(key); assertThat(key, true); assertThat(key, key.contains(pathPrefix + dateString)); @@ -184,7 +184,9 @@ void test_output_with_threshold_set_as_more_then_zero_event_count() throws IOExc when(bufferFactory.getBuffer()).thenReturn(buffer); when(s3SinkConfig.getThresholdOptions().getEventCount()).thenReturn(5); - when(codec.parse(any(), anyString())).thenReturn("{\"message\":\"31824252-adba-4c47-a2ac-05d16c5b8140\"}"); + final OutputStream outputStream = mock(OutputStream.class); + final Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + doNothing().when(codec).writeEvent(event, outputStream, null); S3SinkService s3SinkService = createObjectUnderTest(); assertNotNull(s3SinkService); s3SinkService.output(generateRandomStringEventRecord()); @@ -205,7 +207,9 @@ void test_output_with_threshold_set_as_zero_event_count() throws IOException { when(s3SinkConfig.getThresholdOptions().getEventCount()).thenReturn(0); when(s3SinkConfig.getThresholdOptions().getMaximumSize()).thenReturn(ByteCount.parse("2kb")); - when(codec.parse(any(), anyString())).thenReturn("{\"message\":\"31824252-adba-4c47-a2ac-05d16c5b8140\"}"); + final OutputStream outputStream = mock(OutputStream.class); + final Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + doNothing().when(codec).writeEvent(event, outputStream, null); S3SinkService s3SinkService = createObjectUnderTest(); assertNotNull(s3SinkService); s3SinkService.output(generateRandomStringEventRecord()); @@ -221,7 +225,9 @@ void test_output_with_uploadedToS3_success() throws IOException { doNothing().when(buffer).flushToS3(any(S3Client.class), anyString(), any(String.class)); when(bufferFactory.getBuffer()).thenReturn(buffer); - when(codec.parse(any(), anyString())).thenReturn("{\"message\":\"31824252-adba-4c47-a2ac-05d16c5b8140\"}"); + final OutputStream outputStream = mock(OutputStream.class); + final Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + doNothing().when(codec).writeEvent(event, outputStream, null); S3SinkService s3SinkService = createObjectUnderTest(); assertNotNull(s3SinkService); assertThat(s3SinkService, instanceOf(S3SinkService.class)); @@ -239,7 +245,9 @@ void test_output_with_uploadedToS3_success_records_byte_count() throws IOExcepti final long objectSize = random.nextInt(1_000_000) + 10_000; when(buffer.getSize()).thenReturn(objectSize); - when(codec.parse(any(), anyString())).thenReturn(UUID.randomUUID().toString()); + final OutputStream outputStream = mock(OutputStream.class); + final Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + doNothing().when(codec).writeEvent(event, outputStream, null); final S3SinkService s3SinkService = createObjectUnderTest(); s3SinkService.output(generateRandomStringEventRecord()); @@ -250,7 +258,9 @@ void test_output_with_uploadedToS3_success_records_byte_count() throws IOExcepti void test_output_with_uploadedToS3_failed() throws IOException { when(s3SinkConfig.getBucketName()).thenReturn(UUID.randomUUID().toString()); when(s3SinkConfig.getMaxUploadRetries()).thenReturn(3); - when(codec.parse(any(), anyString())).thenReturn("{\"message\":\"31824252-adba-4c47-a2ac-05d16c5b8140\"}"); + final OutputStream outputStream = mock(OutputStream.class); + final Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + doNothing().when(codec).writeEvent(event, outputStream, null); S3SinkService s3SinkService = createObjectUnderTest(); assertNotNull(s3SinkService); assertThat(s3SinkService, instanceOf(S3SinkService.class)); @@ -270,9 +280,10 @@ void test_output_with_uploadedToS3_failure_does_not_record_byte_count() throws I final long objectSize = random.nextInt(1_000_000) + 10_000; when(buffer.getSize()).thenReturn(objectSize); - when(codec.parse(any(), anyString())).thenReturn(UUID.randomUUID().toString()); final S3SinkService s3SinkService = createObjectUnderTest(); final Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + final OutputStream outputStream = mock(OutputStream.class); + doNothing().when(codec).writeEvent(event, outputStream, null); s3SinkService.output(Collections.singletonList(new Record<>(event))); verify(s3ObjectSizeSummary, never()).record(anyLong()); @@ -290,7 +301,9 @@ void test_retryFlushToS3_positive() throws InterruptedException, IOException { S3SinkService s3SinkService = createObjectUnderTest(); assertNotNull(s3SinkService); assertNotNull(buffer); - buffer.writeEvent(generateByteArray()); + OutputStream outputStream = buffer.getOutputStream(); + final Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + codec.writeEvent(event, outputStream, null); final String s3Key = UUID.randomUUID().toString(); boolean isUploadedToS3 = s3SinkService.retryFlushToS3(buffer, s3Key); assertTrue(isUploadedToS3); @@ -304,7 +317,9 @@ void test_retryFlushToS3_negative() throws InterruptedException, IOException { when(s3SinkConfig.getBucketName()).thenReturn(""); S3SinkService s3SinkService = createObjectUnderTest(); assertNotNull(s3SinkService); - buffer.writeEvent(generateByteArray()); + OutputStream outputStream = buffer.getOutputStream(); + final Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + codec.writeEvent(event, outputStream, null); final String s3Key = UUID.randomUUID().toString(); doThrow(AwsServiceException.class).when(buffer).flushToS3(eq(s3Client), anyString(), anyString()); boolean isUploadedToS3 = s3SinkService.retryFlushToS3(buffer, s3Key); @@ -321,7 +336,9 @@ void output_will_release_all_handles_since_a_flush() throws IOException { final long objectSize = random.nextInt(1_000_000) + 10_000; when(buffer.getSize()).thenReturn(objectSize); - when(codec.parse(any(), anyString())).thenReturn(UUID.randomUUID().toString()); + final OutputStream outputStream = mock(OutputStream.class); + final Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + doNothing().when(codec).writeEvent(event, outputStream, null); final S3SinkService s3SinkService = createObjectUnderTest(); final Collection> records = generateRandomStringEventRecord(); s3SinkService.output(records); @@ -342,7 +359,9 @@ void output_will_skip_releasing_events_without_EventHandle_objects() throws IOEx final long objectSize = random.nextInt(1_000_000) + 10_000; when(buffer.getSize()).thenReturn(objectSize); - when(codec.parse(any(), anyString())).thenReturn(UUID.randomUUID().toString()); + final OutputStream outputStream = mock(OutputStream.class); + final Event event1 = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + doNothing().when(codec).writeEvent(event1, outputStream, null); final S3SinkService s3SinkService = createObjectUnderTest(); final Collection> records = generateRandomStringEventRecord(); records.stream() @@ -373,7 +392,9 @@ void output_will_release_all_handles_since_a_flush_when_S3_fails() throws IOExce final long objectSize = random.nextInt(1_000_000) + 10_000; when(buffer.getSize()).thenReturn(objectSize); - when(codec.parse(any(), anyString())).thenReturn(UUID.randomUUID().toString()); + final OutputStream outputStream = mock(OutputStream.class); + final Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + doNothing().when(codec).writeEvent(event, outputStream, null); final S3SinkService s3SinkService = createObjectUnderTest(); final List> records = generateEventRecords(1); s3SinkService.output(records); @@ -394,7 +415,9 @@ void output_will_release_only_new_handles_since_a_flush() throws IOException { final long objectSize = random.nextInt(1_000_000) + 10_000; when(buffer.getSize()).thenReturn(objectSize); - when(codec.parse(any(), anyString())).thenReturn(UUID.randomUUID().toString()); + final OutputStream outputStream = mock(OutputStream.class); + final Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + doNothing().when(codec).writeEvent(event, outputStream, null); final S3SinkService s3SinkService = createObjectUnderTest(); final Collection> records = generateRandomStringEventRecord(); s3SinkService.output(records); @@ -425,7 +448,9 @@ void output_will_release_only_new_handles_since_a_flush_when_S3_fails() throws I final long objectSize = random.nextInt(1_000_000) + 10_000; when(buffer.getSize()).thenReturn(objectSize); - when(codec.parse(any(), anyString())).thenReturn(UUID.randomUUID().toString()); + final OutputStream outputStream = mock(OutputStream.class); + final Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + doNothing().when(codec).writeEvent(event, outputStream, null); final S3SinkService s3SinkService = createObjectUnderTest(); final List> records = generateEventRecords(1); s3SinkService.output(records); @@ -470,4 +495,4 @@ private byte[] generateByteArray() { } return bytes; } -} +} \ No newline at end of file diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/S3SinkTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/S3SinkTest.java index b92da61bc6..f7f8ceb0fb 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/S3SinkTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/S3SinkTest.java @@ -9,6 +9,7 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.model.codec.OutputCodec; import org.opensearch.dataprepper.model.configuration.PluginModel; import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.event.Event; @@ -17,8 +18,6 @@ import org.opensearch.dataprepper.model.sink.SinkContext; import org.opensearch.dataprepper.model.types.ByteCount; import org.opensearch.dataprepper.plugins.sink.accumulator.BufferTypeOptions; -import org.opensearch.dataprepper.plugins.sink.codec.Codec; -import org.opensearch.dataprepper.plugins.sink.codec.JsonCodec; import org.opensearch.dataprepper.plugins.sink.configuration.AwsAuthenticationOptions; import org.opensearch.dataprepper.plugins.sink.configuration.ObjectKeyOptions; import org.opensearch.dataprepper.plugins.sink.configuration.ThresholdOptions; @@ -59,7 +58,7 @@ void setUp() { sinkContext = mock(SinkContext.class); ThresholdOptions thresholdOptions = mock(ThresholdOptions.class); AwsAuthenticationOptions awsAuthenticationOptions = mock(AwsAuthenticationOptions.class); - Codec codec = mock(JsonCodec.class); + OutputCodec codec = mock(OutputCodec.class); ObjectKeyOptions objectKeyOptions = mock(ObjectKeyOptions.class); pluginSetting = mock(PluginSetting.class); PluginModel pluginModel = mock(PluginModel.class); diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/ThresholdCheckTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/ThresholdCheckTest.java index 992e17368f..18d6672b0d 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/ThresholdCheckTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/ThresholdCheckTest.java @@ -11,6 +11,8 @@ import org.opensearch.dataprepper.plugins.sink.accumulator.Buffer; import org.opensearch.dataprepper.plugins.sink.accumulator.InMemoryBufferFactory; import java.io.IOException; +import java.io.OutputStream; + import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -23,7 +25,10 @@ void setUp() throws IOException { inMemoryBuffer = new InMemoryBufferFactory().getBuffer(); while (inMemoryBuffer.getEventCount() < 100) { - inMemoryBuffer.writeEvent(generateByteArray()); + OutputStream outputStream = inMemoryBuffer.getOutputStream(); + outputStream.write(generateByteArray()); + int eventCount = inMemoryBuffer.getEventCount() +1; + inMemoryBuffer.setEventCount(eventCount); } } @@ -78,7 +83,10 @@ void test_exceedThreshold_ture_dueTo_maxCollectionDuration_is_less_than_buffered boolean isThresholdExceed = Boolean.FALSE; synchronized (this) { while (inMemoryBuffer.getEventCount() < 100) { - inMemoryBuffer.writeEvent(generateByteArray()); + OutputStream outputStream = inMemoryBuffer.getOutputStream(); + outputStream.write(generateByteArray()); + int eventCount = inMemoryBuffer.getEventCount() +1; + inMemoryBuffer.setEventCount(eventCount); isThresholdExceed = ThresholdCheck.checkThresholdExceed(inMemoryBuffer, maxEvents, maxBytes, maxCollectionDuration); if (isThresholdExceed) { @@ -102,7 +110,10 @@ void test_exceedThreshold_ture_dueTo_maxCollectionDuration_is_greater_than_buffe boolean isThresholdExceed = Boolean.FALSE; synchronized (this) { while (inMemoryBuffer.getEventCount() < 100) { - inMemoryBuffer.writeEvent(generateByteArray()); + OutputStream outputStream = inMemoryBuffer.getOutputStream(); + outputStream.write(generateByteArray()); + int eventCount = inMemoryBuffer.getEventCount() +1; + inMemoryBuffer.setEventCount(eventCount); isThresholdExceed = ThresholdCheck.checkThresholdExceed(inMemoryBuffer, maxEvents, maxBytes, maxCollectionDuration); if (isThresholdExceed) { diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/accumulator/InMemoryBufferTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/accumulator/InMemoryBufferTest.java index d97411e37c..254b32278c 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/accumulator/InMemoryBufferTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/accumulator/InMemoryBufferTest.java @@ -14,6 +14,8 @@ import org.mockito.junit.jupiter.MockitoExtension; import software.amazon.awssdk.services.s3.S3Client; import java.io.IOException; +import java.io.OutputStream; + import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -31,7 +33,10 @@ void test_with_write_event_into_buffer() throws IOException { inMemoryBuffer = new InMemoryBuffer(); while (inMemoryBuffer.getEventCount() < MAX_EVENTS) { - inMemoryBuffer.writeEvent(generateByteArray()); + OutputStream outputStream = inMemoryBuffer.getOutputStream(); + outputStream.write(generateByteArray()); + int eventCount = inMemoryBuffer.getEventCount() +1; + inMemoryBuffer.setEventCount(eventCount); } assertThat(inMemoryBuffer.getSize(), greaterThanOrEqualTo(54110L)); assertThat(inMemoryBuffer.getEventCount(), equalTo(MAX_EVENTS)); @@ -44,7 +49,10 @@ void test_with_write_event_into_buffer_and_flush_toS3() throws IOException { inMemoryBuffer = new InMemoryBuffer(); while (inMemoryBuffer.getEventCount() < MAX_EVENTS) { - inMemoryBuffer.writeEvent(generateByteArray()); + OutputStream outputStream = inMemoryBuffer.getOutputStream(); + outputStream.write(generateByteArray()); + int eventCount = inMemoryBuffer.getEventCount() +1; + inMemoryBuffer.setEventCount(eventCount); } assertDoesNotThrow(() -> { inMemoryBuffer.flushToS3(s3Client, "data-prepper", "log.txt"); diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/accumulator/LocalFileBufferTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/accumulator/LocalFileBufferTest.java index 65ad70963d..a7c30f7b13 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/accumulator/LocalFileBufferTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/accumulator/LocalFileBufferTest.java @@ -13,6 +13,7 @@ import software.amazon.awssdk.services.s3.model.PutObjectRequest; import java.io.File; import java.io.IOException; +import java.io.OutputStream; import java.util.UUID; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.notNullValue; @@ -46,7 +47,10 @@ void setUp() throws IOException { @Test void test_with_write_events_into_buffer() throws IOException { while (localFileBuffer.getEventCount() < 55) { - localFileBuffer.writeEvent(generateByteArray()); + OutputStream outputStream = localFileBuffer.getOutputStream(); + outputStream.write(generateByteArray()); + int eventCount = localFileBuffer.getEventCount() +1; + localFileBuffer.setEventCount(eventCount); } assertThat(localFileBuffer.getSize(), greaterThan(1l)); assertThat(localFileBuffer.getEventCount(), equalTo(55)); @@ -69,7 +73,10 @@ void test_without_write_events_into_buffer() { @Test void test_with_write_events_into_buffer_and_flush_toS3() throws IOException { while (localFileBuffer.getEventCount() < 55) { - localFileBuffer.writeEvent(generateByteArray()); + OutputStream outputStream = localFileBuffer.getOutputStream(); + outputStream.write(generateByteArray()); + int eventCount = localFileBuffer.getEventCount() +1; + localFileBuffer.setEventCount(eventCount); } assertThat(localFileBuffer.getSize(), greaterThan(1l)); assertThat(localFileBuffer.getEventCount(), equalTo(55)); diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/accumulator/ObjectKeyTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/accumulator/ObjectKeyTest.java index c2ee676e5d..f70cba4d3a 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/accumulator/ObjectKeyTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/accumulator/ObjectKeyTest.java @@ -15,7 +15,6 @@ import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.plugin.PluginFactory; import org.opensearch.dataprepper.plugins.sink.S3SinkConfig; -import org.opensearch.dataprepper.plugins.sink.codec.JsonCodec; import org.opensearch.dataprepper.plugins.sink.configuration.ObjectKeyOptions; import static org.hamcrest.CoreMatchers.startsWith; @@ -36,8 +35,6 @@ class ObjectKeyTest { @Mock private PluginFactory pluginFactory; @Mock - private JsonCodec codec; - @Mock private ObjectKeyOptions objectKeyOptions; @BeforeEach @@ -58,7 +55,7 @@ void test_buildingPathPrefix() { void test_objectFileName() { when(objectKeyOptions.getNamePattern()).thenReturn("my-elb-%{yyyy-MM-dd'T'hh-mm-ss}"); - String objectFileName = ObjectKey.objectFileName(s3SinkConfig); + String objectFileName = ObjectKey.objectFileName(s3SinkConfig, null); Assertions.assertNotNull(objectFileName); assertThat(objectFileName, startsWith("my-elb")); } @@ -68,7 +65,7 @@ void test_objectFileName_with_fileExtension() { when(s3SinkConfig.getObjectKeyOptions().getNamePattern()) .thenReturn("events-%{yyyy-MM-dd'T'hh-mm-ss}.pdf"); - String objectFileName = ObjectKey.objectFileName(s3SinkConfig); + String objectFileName = ObjectKey.objectFileName(s3SinkConfig, null); Assertions.assertNotNull(objectFileName); Assertions.assertTrue(objectFileName.contains(".pdf")); } @@ -78,7 +75,7 @@ void test_objectFileName_default_fileExtension() { when(s3SinkConfig.getObjectKeyOptions().getNamePattern()) .thenReturn("events-%{yyyy-MM-dd'T'hh-mm-ss}"); - String objectFileName = ObjectKey.objectFileName(s3SinkConfig); + String objectFileName = ObjectKey.objectFileName(s3SinkConfig, null); Assertions.assertNotNull(objectFileName); Assertions.assertTrue(objectFileName.contains(".json")); } From a435a99a0b4ae45a1525fbe16dec74f7795bdfaa Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 5 Jul 2023 10:55:11 -0500 Subject: [PATCH 04/10] Bump com.fasterxml.jackson.datatype:jackson-datatype-jsr310 (#2796) Bumps com.fasterxml.jackson.datatype:jackson-datatype-jsr310 from 2.14.2 to 2.15.2. --- updated-dependencies: - dependency-name: com.fasterxml.jackson.datatype:jackson-datatype-jsr310 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- data-prepper-plugins/kafka-plugins/build.gradle | 2 +- data-prepper-plugins/opensearch-source/build.gradle | 2 +- data-prepper-plugins/s3-source/build.gradle | 2 +- data-prepper-plugins/sqs-source/build.gradle | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/data-prepper-plugins/kafka-plugins/build.gradle b/data-prepper-plugins/kafka-plugins/build.gradle index 8335b761f9..70c635bd08 100644 --- a/data-prepper-plugins/kafka-plugins/build.gradle +++ b/data-prepper-plugins/kafka-plugins/build.gradle @@ -23,7 +23,7 @@ dependencies { testImplementation 'org.mockito:mockito-inline:4.1.0' testImplementation 'org.yaml:snakeyaml:2.0' testImplementation testLibs.spring.test - testImplementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.14.2' + testImplementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.15.2' testImplementation project(':data-prepper-test-common') testImplementation project(':data-prepper-plugins:blocking-buffer') testImplementation project(':data-prepper-core') diff --git a/data-prepper-plugins/opensearch-source/build.gradle b/data-prepper-plugins/opensearch-source/build.gradle index 22a02beeda..cd88114792 100644 --- a/data-prepper-plugins/opensearch-source/build.gradle +++ b/data-prepper-plugins/opensearch-source/build.gradle @@ -8,7 +8,7 @@ dependencies { implementation project(':data-prepper-plugins:aws-plugin-api') implementation 'software.amazon.awssdk:apache-client' implementation 'com.fasterxml.jackson.core:jackson-databind' - implementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.14.2' + implementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.15.2' implementation 'software.amazon.awssdk:s3' implementation 'software.amazon.awssdk:sts' testImplementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' diff --git a/data-prepper-plugins/s3-source/build.gradle b/data-prepper-plugins/s3-source/build.gradle index 70c27e342a..d2fb01a822 100644 --- a/data-prepper-plugins/s3-source/build.gradle +++ b/data-prepper-plugins/s3-source/build.gradle @@ -29,7 +29,7 @@ dependencies { implementation 'joda-time:joda-time:2.11.1' implementation 'org.hibernate.validator:hibernate-validator:7.0.5.Final' implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-csv' - implementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.14.2' + implementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.15.2' implementation 'org.xerial.snappy:snappy-java:1.1.10.1' implementation 'org.apache.parquet:parquet-common:1.12.3' testImplementation 'org.apache.commons:commons-lang3:3.12.0' diff --git a/data-prepper-plugins/sqs-source/build.gradle b/data-prepper-plugins/sqs-source/build.gradle index 893a1faf8d..199271317f 100644 --- a/data-prepper-plugins/sqs-source/build.gradle +++ b/data-prepper-plugins/sqs-source/build.gradle @@ -9,7 +9,7 @@ dependencies { implementation 'software.amazon.awssdk:sts' implementation 'io.micrometer:micrometer-core' implementation 'com.fasterxml.jackson.core:jackson-annotations' - implementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.14.2' + implementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.15.2' testImplementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' testImplementation project(':data-prepper-plugins:blocking-buffer') } From 906eef895c17da707a0d63e7ec8ebae0ea66f82a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 5 Jul 2023 10:57:43 -0500 Subject: [PATCH 05/10] Bump commons-io:commons-io from 2.11.0 to 2.13.0 in /data-prepper-api (#2900) Bumps commons-io:commons-io from 2.11.0 to 2.13.0. --- updated-dependencies: - dependency-name: commons-io:commons-io dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- data-prepper-api/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/data-prepper-api/build.gradle b/data-prepper-api/build.gradle index 6234c99214..42032769b9 100644 --- a/data-prepper-api/build.gradle +++ b/data-prepper-api/build.gradle @@ -14,7 +14,7 @@ dependencies { testImplementation testLibs.junit.vintage testImplementation project(':data-prepper-test-common') testImplementation 'org.skyscreamer:jsonassert:1.5.1' - testImplementation 'commons-io:commons-io:2.11.0' + testImplementation 'commons-io:commons-io:2.13.0' } jacocoTestCoverageVerification { From e66b0916ffdb120aa33dced3d562adefac6b497b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 5 Jul 2023 10:59:41 -0500 Subject: [PATCH 06/10] Bump org.apache.commons:commons-compress in /data-prepper-plugins/common (#2960) Bumps org.apache.commons:commons-compress from 1.21 to 1.23.0. --- updated-dependencies: - dependency-name: org.apache.commons:commons-compress dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- data-prepper-plugins/common/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/data-prepper-plugins/common/build.gradle b/data-prepper-plugins/common/build.gradle index 87b15dcd31..9a2041e5c6 100644 --- a/data-prepper-plugins/common/build.gradle +++ b/data-prepper-plugins/common/build.gradle @@ -13,7 +13,7 @@ dependencies { implementation "commons-io:commons-io:2.12.0" implementation 'software.amazon.awssdk:s3' implementation 'software.amazon.awssdk:acm' - implementation 'org.apache.commons:commons-compress:1.21' + implementation 'org.apache.commons:commons-compress:1.23.0' implementation "org.apache.commons:commons-lang3:3.12.0" implementation "org.bouncycastle:bcprov-jdk15on:1.70" implementation "org.bouncycastle:bcpkix-jdk15on:1.70" From 4aa35a3d39fa54779a9dd317ef6000d722f83cc0 Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo <95880281+MaGonzalMayedo@users.noreply.github.com> Date: Wed, 5 Jul 2023 09:12:03 -0700 Subject: [PATCH 07/10] GitHub-Issue#2778: Added CloudWatchLogs Sink Config Files (#2922) * Elasticsearch client implementation with pit and no context search (#2910) Create Elasticsearch client, implement search and pit apis for ElasticsearchAccessor Signed-off-by: Taylor Gray Signed-off-by: Marcos Gonzalez Mayedo * GitHub-Issue#2778: Refactoring config files for CloudWatchLogs Sink (#4) Added Config Files for CloudWatchLogs Sink. Signed-off-by: Marcos Gonzalez Mayedo * Added fixes from comments to code (including pathing and nomenclature syntax) Signed-off-by: Marcos Gonzalez Mayedo * Refactoring config (#5) Added default params for back_off and log_send_interval alongside test cases for ThresholdConfig. Signed-off-by: Marcos Gonzalez Mayedo * Fixed deleted AwsConfig file Signed-off-by: Marcos Gonzalez Mayedo * Removed the s3 dependency from build.gradle, replaced the AwsAuth.. with AwsConfig. Signed-off-by: Marcos Gonzalez Mayedo * Added modifiable back_off_timer, added threshold test for back_off_timer and params to AwsConfig Signed-off-by: Marcos Gonzalez Mayedo * Added fixes to gradle file, added tests to AwsConfig, and used Reflective mapping to tests CwlSink Signed-off-by: Marcos Gonzalez Mayedo * Added default value test to ThresholdConfig and renamed getter for maxRequestSize Signed-off-by: Marcos Gonzalez Mayedo * Removed unnecessary imports Signed-off-by: Marcos Gonzalez Mayedo * Added cloudwatch-logs to settings.gradle Signed-off-by: Marcos Gonzalez Mayedo * Added a quick fix to the back_off_time range Signed-off-by: Marcos Gonzalez Mayedo --------- Signed-off-by: Taylor Gray Signed-off-by: Marcos Gonzalez Mayedo Signed-off-by: Marcos Gonzalez Mayedo <95880281+MaGonzalMayedo@users.noreply.github.com> Co-authored-by: Taylor Gray Co-authored-by: Marcos --- .../cloudwatch-logs/build.gradle | 38 ++++++++ .../plugins/sink/config/AwsConfig.java | 48 ++++++++++ .../plugins/sink/config/CwlSinkConfig.java | 52 +++++++++++ .../plugins/sink/config/ThresholdConfig.java | 66 ++++++++++++++ .../sink/configuration/AwsConfigTest.java | 87 +++++++++++++++++++ .../sink/configuration/CwlSinkConfigTest.java | 63 ++++++++++++++ .../configuration/ThresholdConfigTest.java | 82 +++++++++++++++++ .../client/OpenSearchClientFactory.java | 1 - settings.gradle | 1 + 9 files changed, 437 insertions(+), 1 deletion(-) create mode 100644 data-prepper-plugins/cloudwatch-logs/build.gradle create mode 100644 data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfig.java create mode 100644 data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/CwlSinkConfig.java create mode 100644 data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java create mode 100644 data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/AwsConfigTest.java create mode 100644 data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java create mode 100644 data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/ThresholdConfigTest.java diff --git a/data-prepper-plugins/cloudwatch-logs/build.gradle b/data-prepper-plugins/cloudwatch-logs/build.gradle new file mode 100644 index 0000000000..2d00cbd2d0 --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/build.gradle @@ -0,0 +1,38 @@ +plugins { + id 'java' + id 'java-library' +} + +repositories { + mavenCentral() +} + +dependencies { + api project(':data-prepper-api') + implementation project(':data-prepper-plugins:aws-plugin-api') + implementation project(path: ':data-prepper-plugins:common') + testImplementation 'org.junit.jupiter:junit-jupiter' + implementation 'com.fasterxml.jackson.core:jackson-core' + implementation 'com.fasterxml.jackson.core:jackson-databind' + implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' + implementation 'software.amazon.awssdk:cloudwatch' + implementation 'software.amazon.awssdk:cloudwatchlogs' + implementation 'org.apache.commons:commons-lang3:3.12.0' + testImplementation project(path: ':data-prepper-test-common') + testImplementation project(path: ':data-prepper-test-common') +} + +jacocoTestCoverageVerification { + dependsOn jacocoTestReport + violationRules { + rule { //in addition to core projects rule + limit { + minimum = 0.90 + } + } + } +} + +test { + useJUnitPlatform() +} \ No newline at end of file diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfig.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfig.java new file mode 100644 index 0000000000..6a2a85557e --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/AwsConfig.java @@ -0,0 +1,48 @@ +package org.opensearch.dataprepper.plugins.sink.config; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.Size; +import software.amazon.awssdk.regions.Region; + +import java.util.Map; + +/** + * AwsConfig is based on the S3-Sink AwsAuthenticationOptions + * where the configuration allows the sink to fetch Aws credentials + * and resources. + */ +public class AwsConfig { + public static int DEFAULT_CONNECTION_ATTEMPTS = 5; + + @JsonProperty("region") + @Size(min = 1, message = "Region cannot be empty string") + private String awsRegion; + + @JsonProperty("sts_role_arn") + @Size(min = 20, max = 2048, message = "awsStsRoleArn length should be between 1 and 2048 characters") + private String awsStsRoleArn; + + @JsonProperty("sts_header_overrides") + @Size(max = 5, message = "sts_header_overrides supports a maximum of 5 headers to override") + private Map awsStsHeaderOverrides; + + @JsonProperty("sts_external_id") + @Size(min = 2, max = 1224, message = "awsStsExternalId length should be between 2 and 1224 characters") + private String awsStsExternalId; + + public Region getAwsRegion() { + return awsRegion != null ? Region.of(awsRegion) : null; + } + + public String getAwsStsRoleArn() { + return awsStsRoleArn; + } + + public String getAwsStsExternalId() { + return awsStsExternalId; + } + + public Map getAwsStsHeaderOverrides() { + return awsStsHeaderOverrides; + } +} diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/CwlSinkConfig.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/CwlSinkConfig.java new file mode 100644 index 0000000000..230512bde5 --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/CwlSinkConfig.java @@ -0,0 +1,52 @@ +package org.opensearch.dataprepper.plugins.sink.config; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.Valid; +import jakarta.validation.constraints.NotEmpty; +import jakarta.validation.constraints.NotNull; + +public class CwlSinkConfig { + public static final String DEFAULT_BUFFER_TYPE = "in_memory"; + + @JsonProperty("aws") + @NotNull + @Valid + private AwsConfig awsConfig; + + @JsonProperty("threshold") + @NotNull + private ThresholdConfig thresholdConfig; + + @JsonProperty("buffer_type") + private String bufferType = DEFAULT_BUFFER_TYPE; + + @JsonProperty("log_group") + @NotEmpty + @NotNull + private String logGroup; + + @JsonProperty("log_stream") + @NotEmpty + @NotNull + private String logStream; + + public AwsConfig getAwsConfig() { + return awsConfig; + } + + public ThresholdConfig getThresholdConfig() { + return thresholdConfig; + } + + public String getBufferType() { + return bufferType; + } + + public String getLogGroup() { + return logGroup; + } + + public String getLogStream() { + return logStream; + } +} diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java new file mode 100644 index 0000000000..77571a2c29 --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/config/ThresholdConfig.java @@ -0,0 +1,66 @@ +package org.opensearch.dataprepper.plugins.sink.config; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.Size; + +/** + * The threshold config holds the different configurations for + * buffer restrictions, retransmission restrictions and timeout + * restrictions. + */ +public class ThresholdConfig { + public static final int DEFAULT_BATCH_SIZE = 100; + public static final int DEFAULT_EVENT_SIZE = 50; + public static final int DEFAULT_SIZE_OF_REQUEST = 524288; + public static final int DEFAULT_RETRY_COUNT = 5; + public static final int DEFAULT_LOG_SEND_INTERVAL_TIME = 60; + public static final int DEFAULT_BACKOFF_TIME = 5000; + + @JsonProperty("batch_size") + @Size(min = 1, max = 10000, message = "batch_size amount should be between 1 to 10000") + private int batchSize = DEFAULT_BATCH_SIZE; + + @JsonProperty("max_event_size") + @Size(min = 1, max = 256, message = "max_event_size amount should be between 1 to 256 kilobytes") + private int maxEventSize = DEFAULT_EVENT_SIZE; + + @JsonProperty("max_request_size") + @Size(min = 1, max = 1048576, message = "max_batch_request_size amount should be between 1 and 1048576 bytes") + private int maxRequestSize = DEFAULT_SIZE_OF_REQUEST; + + @JsonProperty("retry_count") + @Size(min = 1, max = 15, message = "retry_count amount should be between 1 and 15") + private int retryCount = DEFAULT_RETRY_COUNT; + + @JsonProperty("log_send_interval") + @Size(min = 5, max = 300, message = "log_send_interval amount should be between 5 and 300 seconds") + private int logSendInterval = DEFAULT_LOG_SEND_INTERVAL_TIME; + + @JsonProperty("back_off_time") + @Size(min = 500, max = 1000, message = "back_off_time amount should be between 500 and 1000 milliseconds") + private int backOffTime = DEFAULT_BACKOFF_TIME; + + public int getBatchSize() { + return batchSize; + } + + public int getMaxEventSize() { + return maxEventSize; + } + + public int getMaxRequestSize() { + return maxRequestSize; + } + + public int getRetryCount() { + return retryCount; + } + + public int getLogSendInterval() { + return logSendInterval; + } + + public int getBackOffTime() { + return backOffTime; + } +} diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/AwsConfigTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/AwsConfigTest.java new file mode 100644 index 0000000000..8ab03d575d --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/AwsConfigTest.java @@ -0,0 +1,87 @@ +package org.opensearch.dataprepper.plugins.sink.configuration; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import org.opensearch.dataprepper.plugins.sink.config.AwsConfig; +import software.amazon.awssdk.regions.Region; + +import java.util.Collections; +import java.util.Map; +import java.util.UUID; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.MatcherAssert.assertThat; + +public class AwsConfigTest { + private ObjectMapper objectMapper; + + @BeforeEach + void setUp() { + objectMapper = new ObjectMapper(); + } + + @ParameterizedTest + @ValueSource(strings = {"us-east-1", "us-west-2", "eu-central-1"}) + void getAwsRegion_returns_Region_of(final String regionString) { + final Region expectedRegionObject = Region.of(regionString); + final Map jsonMap = Map.of("region", regionString); + final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); + assertThat(objectUnderTest.getAwsRegion(), equalTo(expectedRegionObject)); + } + + @Test + void getAwsRegion_returns_null_when_region_is_null() { + final Map jsonMap = Collections.emptyMap(); + final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); + assertThat(objectUnderTest.getAwsRegion(), nullValue()); + } + + @Test + void getAwsStsRoleArn_returns_value_from_deserialized_JSON() { + final String stsRoleArn = UUID.randomUUID().toString(); + final Map jsonMap = Map.of("sts_role_arn", stsRoleArn); + final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); + assertThat(objectUnderTest.getAwsStsRoleArn(), equalTo(stsRoleArn)); + } + + @Test + void getAwsStsRoleArn_returns_null_if_not_in_JSON() { + final Map jsonMap = Collections.emptyMap(); + final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); + assertThat(objectUnderTest.getAwsStsRoleArn(), nullValue()); + } + + @Test + void getAwsStsExternalId_returns_value_from_deserialized_JSON() { + final String stsExternalId = UUID.randomUUID().toString(); + final Map jsonMap = Map.of("sts_external_id", stsExternalId); + final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); + assertThat(objectUnderTest.getAwsStsExternalId(), equalTo(stsExternalId)); + } + + @Test + void getAwsStsExternalId_returns_null_if_not_in_JSON() { + final Map jsonMap = Collections.emptyMap(); + final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); + assertThat(objectUnderTest.getAwsStsExternalId(), nullValue()); + } + + @Test + void getAwsStsHeaderOverrides_returns_value_from_deserialized_JSON() { + final Map stsHeaderOverrides = Map.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + final Map jsonMap = Map.of("sts_header_overrides", stsHeaderOverrides); + final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); + assertThat(objectUnderTest.getAwsStsHeaderOverrides(), equalTo(stsHeaderOverrides)); + } + + @Test + void getAwsStsHeaderOverrides_returns_null_if_not_in_JSON() { + final Map jsonMap = Collections.emptyMap(); + final AwsConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsConfig.class); + assertThat(objectUnderTest.getAwsStsHeaderOverrides(), nullValue()); + } +} diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java new file mode 100644 index 0000000000..9842a333ee --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/CwlSinkConfigTest.java @@ -0,0 +1,63 @@ +package org.opensearch.dataprepper.plugins.sink.configuration; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.plugins.sink.config.AwsConfig; +import org.opensearch.dataprepper.plugins.sink.config.CwlSinkConfig; +import org.opensearch.dataprepper.plugins.sink.config.ThresholdConfig; +import org.opensearch.dataprepper.test.helper.ReflectivelySetField; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; + +public class CwlSinkConfigTest { + private CwlSinkConfig cwlSinkConfig; + private AwsConfig awsConfig; + private ThresholdConfig thresholdConfig; + private final String LOG_GROUP = "testLogGroup"; + private final String LOG_STREAM = "testLogStream"; + + @BeforeEach + void setUp() { + cwlSinkConfig = new CwlSinkConfig(); + awsConfig = new AwsConfig(); + thresholdConfig = new ThresholdConfig(); + } + + @Test + void check_null_auth_config_test() { + assertThat(new CwlSinkConfig().getAwsConfig(), equalTo(null)); + } + + @Test + void check_default_buffer_type_test() { + assertThat(new CwlSinkConfig().getBufferType(), equalTo(CwlSinkConfig.DEFAULT_BUFFER_TYPE)); + } + + @Test + void check_null_log_group_test() { + assertThat(new CwlSinkConfig().getLogGroup(), equalTo(null)); + } + @Test + void check_null_log_stream_test() { + assertThat(new CwlSinkConfig().getLogStream(), equalTo(null)); + } + + @Test + void check_valid_log_group_and_log_stream_test() throws NoSuchFieldException, IllegalAccessException { + ReflectivelySetField.setField(cwlSinkConfig.getClass(), cwlSinkConfig, "logGroup", LOG_GROUP); + ReflectivelySetField.setField(cwlSinkConfig.getClass(), cwlSinkConfig, "logStream", LOG_STREAM); + + assertThat(cwlSinkConfig.getLogGroup(), equalTo(LOG_GROUP)); + assertThat(cwlSinkConfig.getLogStream(), equalTo(LOG_STREAM)); + } + + @Test + void check_valid_sub_config_test() throws NoSuchFieldException, IllegalAccessException { + ReflectivelySetField.setField(cwlSinkConfig.getClass(), cwlSinkConfig, "thresholdConfig", thresholdConfig); + ReflectivelySetField.setField(cwlSinkConfig.getClass(), cwlSinkConfig, "awsConfig", awsConfig); + + assertThat(cwlSinkConfig.getAwsConfig(), equalTo(awsConfig)); + assertThat(cwlSinkConfig.getThresholdConfig(), equalTo(thresholdConfig)); + } +} diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/ThresholdConfigTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/ThresholdConfigTest.java new file mode 100644 index 0000000000..e6af96e08d --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/configuration/ThresholdConfigTest.java @@ -0,0 +1,82 @@ +package org.opensearch.dataprepper.plugins.sink.configuration; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import org.opensearch.dataprepper.plugins.sink.config.ThresholdConfig; + +import java.util.Map; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; + +public class ThresholdConfigTest { + private ObjectMapper objectMapper; + + @BeforeEach + void setUp() { + objectMapper = new ObjectMapper(); + } + + @Test + void check_default_values() { + final ThresholdConfig thresholdConfig = new ThresholdConfig(); + + assertThat(thresholdConfig.getBackOffTime(), equalTo(ThresholdConfig.DEFAULT_BACKOFF_TIME)); + assertThat(thresholdConfig.getRetryCount(), equalTo(ThresholdConfig.DEFAULT_RETRY_COUNT)); + assertThat(thresholdConfig.getBatchSize(), equalTo(ThresholdConfig.DEFAULT_BATCH_SIZE)); + assertThat(thresholdConfig.getMaxEventSize(), equalTo(ThresholdConfig.DEFAULT_EVENT_SIZE)); + assertThat(thresholdConfig.getMaxRequestSize(), equalTo(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST)); + assertThat(thresholdConfig.getLogSendInterval(), equalTo(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME)); + } + + @ParameterizedTest + @ValueSource(ints = {1, 10, 10000}) + void check_valid_batch_size(final int batchSize) { + final Map jsonMap = Map.of("batch_size", batchSize); + final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); + assertThat(thresholdConfigTest.getBatchSize(), equalTo(batchSize)); + } + + @ParameterizedTest + @ValueSource(ints = {1, 10, 256}) + void check_valid_max_event_size(final int max_event_size) { + final Map jsonMap = Map.of("max_event_size", max_event_size); + final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); + assertThat(thresholdConfigTest.getMaxEventSize(), equalTo(max_event_size)); + } + + @ParameterizedTest + @ValueSource(ints = {1, 100, 1048576}) + void check_valid_request_size(final int max_batch_request_size) { + final Map jsonMap = Map.of("max_request_size", max_batch_request_size); + final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); + assertThat(thresholdConfigTest.getMaxRequestSize(), equalTo(max_batch_request_size)); + } + + @ParameterizedTest + @ValueSource(ints = {1, 10, 15}) + void check_valid_retry_count(final int retry_count) { + final Map jsonMap = Map.of("retry_count", retry_count); + final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); + assertThat(thresholdConfigTest.getRetryCount(), equalTo(retry_count)); + } + + @ParameterizedTest + @ValueSource(ints = {5, 10, 300}) + void check_valid_log_send_interval(final int log_send_interval) { + final Map jsonMap = Map.of("log_send_interval", log_send_interval); + final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); + assertThat(thresholdConfigTest.getLogSendInterval(), equalTo(log_send_interval)); + } + + @ParameterizedTest + @ValueSource(ints = {0, 100, 5000}) + void check_valid_back_off_time(final int back_off_time) { + final Map jsonMap = Map.of("back_off_time", back_off_time); + final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); + assertThat(thresholdConfigTest.getBackOffTime(), equalTo(back_off_time)); + } +} diff --git a/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/OpenSearchClientFactory.java b/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/OpenSearchClientFactory.java index e588e1f711..b32f4bb3a5 100644 --- a/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/OpenSearchClientFactory.java +++ b/data-prepper-plugins/opensearch-source/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/OpenSearchClientFactory.java @@ -189,7 +189,6 @@ private void attachBasicAuth(final org.elasticsearch.client.RestClientBuilder re } else { LOG.warn("Authentication was explicitly disabled for the OpenSearch source"); } - attachSSLContext(httpClientBuilder, openSearchSourceConfiguration); httpClientBuilder.addInterceptorLast( (HttpResponseInterceptor) diff --git a/settings.gradle b/settings.gradle index d668b7d12c..67c85c3a12 100644 --- a/settings.gradle +++ b/settings.gradle @@ -126,4 +126,5 @@ include 'data-prepper-plugins:parquet-codecs' include 'data-prepper-plugins:aws-sqs-common' include 'data-prepper-plugins:buffer-common' include 'data-prepper-plugins:sqs-source' +include 'data-prepper-plugins:cloudwatch-logs' include 'data-prepper-plugins:http-sink' From c78de957040e664ec8e49ff1fb3a193295c1b721 Mon Sep 17 00:00:00 2001 From: mallikagogoi7 Date: Wed, 5 Jul 2023 21:57:57 +0530 Subject: [PATCH 08/10] Boiler plate code of HttpSink for #874. (#2916) * Boiler plate code of HttpSink for #874. Signed-off-by: mallikagogoi7 * Added copyright on classes of HttpSink for #874. Signed-off-by: mallikagogoi7 * Moved Accumulator package to common for #874. Signed-off-by: mallikagogoi7 * Test cases added for accumulator related classes for #874. Signed-off-by: mallikagogoi7 * Added HttpSink related methos in accumulator for #874. Signed-off-by: mallikagogoi7 * Removed plugin specific methods from common for #874. Signed-off-by: mallikagogoi7 --- .../plugins/accumulator/Buffer.java | 25 ++++ .../plugins/accumulator/BufferFactory.java | 10 ++ .../accumulator/BufferTypeOptions.java | 40 ++++++ .../plugins/accumulator/InMemoryBuffer.java | 64 +++++++++ .../accumulator/InMemoryBufferFactory.java | 13 ++ .../plugins/accumulator/LocalFileBuffer.java | 109 +++++++++++++++ .../accumulator/LocalFileBufferFactory.java | 31 +++++ .../plugins/sink/ThresholdCheck.java | 37 +++++ .../accumulator/BufferTypeOptionsTest.java | 35 +++++ .../InMemoryBufferFactoryTest.java | 30 +++++ .../accumulator/InMemoryBufferTest.java | 56 ++++++++ .../LocalFileBufferFactoryTest.java | 29 ++++ .../accumulator/LocalFileBufferTest.java | 84 ++++++++++++ .../plugins/sink/ThresholdCheckTest.java | 126 ++++++++++++++++++ data-prepper-plugins/http-sink/build.gradle | 3 + .../plugins/sink/ClientFactory.java | 13 ++ .../dataprepper/plugins/sink/HTTPSink.java | 21 +++ .../plugins/sink/HttpAuthOptions.java | 65 +++++++++ .../plugins/sink/dlq/FailedDlqData.java | 58 ++++++++ .../handler/BasicAuthHttpSinkHandler.java | 17 +++ .../BearerTokenAuthHttpSinkHandler.java | 17 +++ .../plugins/sink/handler/HttpAuthOptions.java | 61 +++++++++ .../handler/MultiAuthHttpSinkHandler.java | 14 ++ .../handler/SecuredAuthHttpSinkHandler.java | 18 +++ .../plugins/sink/service/HttpSinkService.java | 59 ++++++++ 25 files changed, 1035 insertions(+) create mode 100644 data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/Buffer.java create mode 100644 data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/BufferFactory.java create mode 100644 data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/BufferTypeOptions.java create mode 100644 data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/InMemoryBuffer.java create mode 100644 data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/InMemoryBufferFactory.java create mode 100644 data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/LocalFileBuffer.java create mode 100644 data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/LocalFileBufferFactory.java create mode 100644 data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/ThresholdCheck.java create mode 100644 data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/BufferTypeOptionsTest.java create mode 100644 data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/InMemoryBufferFactoryTest.java create mode 100644 data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/InMemoryBufferTest.java create mode 100644 data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/LocalFileBufferFactoryTest.java create mode 100644 data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/LocalFileBufferTest.java create mode 100644 data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/sink/ThresholdCheckTest.java create mode 100644 data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/ClientFactory.java create mode 100644 data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/HttpAuthOptions.java create mode 100644 data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/dlq/FailedDlqData.java create mode 100644 data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/BasicAuthHttpSinkHandler.java create mode 100644 data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/BearerTokenAuthHttpSinkHandler.java create mode 100644 data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/HttpAuthOptions.java create mode 100644 data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/MultiAuthHttpSinkHandler.java create mode 100644 data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/SecuredAuthHttpSinkHandler.java create mode 100644 data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/service/HttpSinkService.java diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/Buffer.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/Buffer.java new file mode 100644 index 0000000000..c6f6018a4f --- /dev/null +++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/Buffer.java @@ -0,0 +1,25 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.accumulator; + +import java.io.IOException; + +/** + * A buffer can hold data before flushing it any Sink. + */ +public interface Buffer { + + /** + * Gets the current size of the buffer. This should be the number of bytes. + * @return buffer size. + */ + long getSize(); + int getEventCount(); + long getDuration(); + + byte[] getSinkBufferData() throws IOException; + void writeEvent(byte[] bytes) throws IOException; +} diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/BufferFactory.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/BufferFactory.java new file mode 100644 index 0000000000..2a99d63aaf --- /dev/null +++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/BufferFactory.java @@ -0,0 +1,10 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.accumulator; + +public interface BufferFactory { + Buffer getBuffer(); +} diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/BufferTypeOptions.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/BufferTypeOptions.java new file mode 100644 index 0000000000..e2a2ad7158 --- /dev/null +++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/BufferTypeOptions.java @@ -0,0 +1,40 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.accumulator; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.util.Arrays; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * Defines all the buffer types enumerations. + */ +public enum BufferTypeOptions { + + INMEMORY("in_memory", new InMemoryBufferFactory()), + LOCALFILE("local_file", new LocalFileBufferFactory()); + + private final String option; + private final BufferFactory bufferType; + private static final Map OPTIONS_MAP = Arrays.stream(BufferTypeOptions.values()) + .collect(Collectors.toMap(value -> value.option, value -> value)); + + BufferTypeOptions(final String option, final BufferFactory bufferType) { + this.option = option.toLowerCase(); + this.bufferType = bufferType; + } + + public BufferFactory getBufferType() { + return bufferType; + } + + @JsonCreator + static BufferTypeOptions fromOptionValue(final String option) { + return OPTIONS_MAP.get(option); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/InMemoryBuffer.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/InMemoryBuffer.java new file mode 100644 index 0000000000..e583e54260 --- /dev/null +++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/InMemoryBuffer.java @@ -0,0 +1,64 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.accumulator; + +import org.apache.commons.lang3.time.StopWatch; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.concurrent.TimeUnit; + +/** + * A buffer can hold in memory data and flushing it to any Sink. + */ +public class InMemoryBuffer implements Buffer { + + private static final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + private int eventCount; + private final StopWatch watch; + + InMemoryBuffer() { + byteArrayOutputStream.reset(); + eventCount = 0; + watch = new StopWatch(); + watch.start(); + } + + @Override + public long getSize() { + return byteArrayOutputStream.size(); + } + + @Override + public int getEventCount() { + return eventCount; + } + + public long getDuration() { + return watch.getTime(TimeUnit.SECONDS); + } + + /** + * collect current buffer data. + * @throws IOException while collecting current buffer data. + */ + @Override + public byte[] getSinkBufferData() throws IOException { + return byteArrayOutputStream.toByteArray(); + } + + /** + * write byte array to output stream. + * + * @param bytes byte array. + * @throws IOException while writing to output stream fails. + */ + @Override + public void writeEvent(byte[] bytes) throws IOException { + byteArrayOutputStream.write(bytes); + byteArrayOutputStream.write(System.lineSeparator().getBytes()); + eventCount++; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/InMemoryBufferFactory.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/InMemoryBufferFactory.java new file mode 100644 index 0000000000..ef6ec5daf9 --- /dev/null +++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/InMemoryBufferFactory.java @@ -0,0 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.accumulator; + +public class InMemoryBufferFactory implements BufferFactory { + @Override + public Buffer getBuffer() { + return new InMemoryBuffer(); + } +} diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/LocalFileBuffer.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/LocalFileBuffer.java new file mode 100644 index 0000000000..9f9b4a3aac --- /dev/null +++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/LocalFileBuffer.java @@ -0,0 +1,109 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.accumulator; + +import org.apache.commons.lang3.time.StopWatch; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.BufferedOutputStream; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.OutputStream; +import java.io.FileOutputStream; +import java.io.IOException; + +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.concurrent.TimeUnit; + +/** + * A buffer can hold local file data and flushing it to any Sink. + */ +public class LocalFileBuffer implements Buffer { + + private static final Logger LOG = LoggerFactory.getLogger(LocalFileBuffer.class); + private final OutputStream outputStream; + private int eventCount; + private final StopWatch watch; + private final File localFile; + + LocalFileBuffer(File tempFile) throws FileNotFoundException { + localFile = tempFile; + outputStream = new BufferedOutputStream(new FileOutputStream(tempFile)); + eventCount = 0; + watch = new StopWatch(); + watch.start(); + } + + @Override + public long getSize() { + try { + outputStream.flush(); + } catch (IOException e) { + LOG.error("An exception occurred while flushing data to buffered output stream :", e); + } + return localFile.length(); + } + + @Override + public int getEventCount() { + return eventCount; + } + + @Override + public long getDuration(){ + return watch.getTime(TimeUnit.SECONDS); + } + + /** + * collect current buffer data. + * @throws IOException while collecting current buffer data. + */ + @Override + public byte[] getSinkBufferData() throws IOException { + final byte[] fileData = Files.readAllBytes(localFile.toPath()); + removeTemporaryFile(); + return fileData; + } + + /** + * write byte array to output stream. + * @param bytes byte array. + * @throws IOException while writing to output stream fails. + */ + @Override + public void writeEvent(byte[] bytes) throws IOException { + outputStream.write(bytes); + outputStream.write(System.lineSeparator().getBytes()); + eventCount++; + } + + /** + * Flushing the buffered data into the output stream. + */ + protected void flushAndCloseStream(){ + try { + outputStream.flush(); + outputStream.close(); + } catch (IOException e) { + LOG.error("An exception occurred while flushing data to buffered output stream :", e); + } + } + + /** + * Remove the local temp file after flushing data to Sink. + */ + protected void removeTemporaryFile() { + if (localFile != null) { + try { + Files.deleteIfExists(Paths.get(localFile.toString())); + } catch (IOException e) { + LOG.error("Unable to delete Local file {}", localFile, e); + } + } + } +} \ No newline at end of file diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/LocalFileBufferFactory.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/LocalFileBufferFactory.java new file mode 100644 index 0000000000..cf11ba2a39 --- /dev/null +++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/accumulator/LocalFileBufferFactory.java @@ -0,0 +1,31 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.accumulator; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; + +public class LocalFileBufferFactory implements BufferFactory { + + private static final Logger LOG = LoggerFactory.getLogger(LocalFileBufferFactory.class); + public static final String PREFIX = "local"; + public static final String SUFFIX = ".log"; + @Override + public Buffer getBuffer() { + File tempFile = null; + Buffer localfileBuffer = null; + try { + tempFile = File.createTempFile(PREFIX, SUFFIX); + localfileBuffer = new LocalFileBuffer(tempFile); + } catch (IOException e) { + LOG.error("Unable to create temp file ", e); + } + return localfileBuffer; + } +} diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/ThresholdCheck.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/ThresholdCheck.java new file mode 100644 index 0000000000..1792aea483 --- /dev/null +++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/ThresholdCheck.java @@ -0,0 +1,37 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink; + +import org.opensearch.dataprepper.model.types.ByteCount; +import org.opensearch.dataprepper.plugins.accumulator.Buffer; + +/** + * Check threshold limits. + */ +public class ThresholdCheck { + + private ThresholdCheck() { + } + + /** + * Check threshold exceeds. + * @param currentBuffer current buffer. + * @param maxEvents maximum event provided by user as threshold. + * @param maxBytes maximum bytes provided by user as threshold. + * @param maxCollectionDuration maximum event collection duration provided by user as threshold. + * @return boolean value whether the threshold are met. + */ + public static boolean checkThresholdExceed(final Buffer currentBuffer, final int maxEvents, final ByteCount maxBytes, final long maxCollectionDuration) { + if (maxEvents > 0) { + return currentBuffer.getEventCount() + 1 > maxEvents || + currentBuffer.getDuration() > maxCollectionDuration || + currentBuffer.getSize() > maxBytes.getBytes(); + } else { + return currentBuffer.getDuration() > maxCollectionDuration || + currentBuffer.getSize() > maxBytes.getBytes(); + } + } +} diff --git a/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/BufferTypeOptionsTest.java b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/BufferTypeOptionsTest.java new file mode 100644 index 0000000000..2211085b33 --- /dev/null +++ b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/BufferTypeOptionsTest.java @@ -0,0 +1,35 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.accumulator; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.junit.jupiter.MockitoExtension; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertNotNull; + +@ExtendWith(MockitoExtension.class) +class BufferTypeOptionsTest { + + @Test + void notNull_test() { + assertNotNull(BufferTypeOptions.INMEMORY); + } + + @Test + void get_buffer_type_test() { + assertNotNull(BufferTypeOptions.INMEMORY.getBufferType()); + } + + @Test + void fromOptionValue_test() { + BufferTypeOptions bufferTypeOptions = BufferTypeOptions.fromOptionValue("in_memory"); + assertNotNull(bufferTypeOptions); + assertThat(bufferTypeOptions.toString(), equalTo("INMEMORY")); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/InMemoryBufferFactoryTest.java b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/InMemoryBufferFactoryTest.java new file mode 100644 index 0000000000..ab533f4015 --- /dev/null +++ b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/InMemoryBufferFactoryTest.java @@ -0,0 +1,30 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.accumulator; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.MatcherAssert.assertThat; + +class InMemoryBufferFactoryTest { + + @Test + void test_inMemoryBufferFactory_notNull(){ + InMemoryBufferFactory inMemoryBufferFactory = new InMemoryBufferFactory(); + Assertions.assertNotNull(inMemoryBufferFactory); + } + + @Test + void test_buffer_notNull(){ + InMemoryBufferFactory inMemoryBufferFactory = new InMemoryBufferFactory(); + Assertions.assertNotNull(inMemoryBufferFactory); + Buffer buffer = inMemoryBufferFactory.getBuffer(); + Assertions.assertNotNull(buffer); + assertThat(buffer, instanceOf(Buffer.class)); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/InMemoryBufferTest.java b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/InMemoryBufferTest.java new file mode 100644 index 0000000000..ad07cc4011 --- /dev/null +++ b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/InMemoryBufferTest.java @@ -0,0 +1,56 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.accumulator; + +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.IOException; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; + +@ExtendWith(MockitoExtension.class) +class InMemoryBufferTest { + + public static final int MAX_EVENTS = 55; + + private InMemoryBuffer inMemoryBuffer; + + @Test + void test_with_write_event_into_buffer() throws IOException { + inMemoryBuffer = new InMemoryBuffer(); + + while (inMemoryBuffer.getEventCount() < MAX_EVENTS) { + inMemoryBuffer.writeEvent(generateByteArray()); + } + assertThat(inMemoryBuffer.getSize(), greaterThanOrEqualTo(54110L)); + assertThat(inMemoryBuffer.getEventCount(), equalTo(MAX_EVENTS)); + assertThat(inMemoryBuffer.getDuration(), greaterThanOrEqualTo(0L)); + + } + + @Test + void test_getSinkData_success() { + inMemoryBuffer = new InMemoryBuffer(); + Assertions.assertNotNull(inMemoryBuffer); + assertDoesNotThrow(() -> { + inMemoryBuffer.getSinkBufferData(); + }); + } + + private byte[] generateByteArray() { + byte[] bytes = new byte[1000]; + for (int i = 0; i < 1000; i++) { + bytes[i] = (byte) i; + } + return bytes; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/LocalFileBufferFactoryTest.java b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/LocalFileBufferFactoryTest.java new file mode 100644 index 0000000000..def5990028 --- /dev/null +++ b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/LocalFileBufferFactoryTest.java @@ -0,0 +1,29 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.accumulator; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.MatcherAssert.assertThat; +class LocalFileBufferFactoryTest { + + @Test + void test_localFileBufferFactory_notNull() { + LocalFileBufferFactory localFileBufferFactory = new LocalFileBufferFactory(); + Assertions.assertNotNull(localFileBufferFactory); + } + + @Test + void test_buffer_notNull() { + LocalFileBufferFactory localFileBufferFactory = new LocalFileBufferFactory(); + Assertions.assertNotNull(localFileBufferFactory); + Buffer buffer = localFileBufferFactory.getBuffer(); + Assertions.assertNotNull(buffer); + assertThat(buffer, instanceOf(LocalFileBuffer.class)); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/LocalFileBufferTest.java b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/LocalFileBufferTest.java new file mode 100644 index 0000000000..53c556e75c --- /dev/null +++ b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/accumulator/LocalFileBufferTest.java @@ -0,0 +1,84 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.accumulator; + +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.junit.jupiter.MockitoExtension; + +import java.io.File; +import java.io.IOException; +import java.util.UUID; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.equalTo; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertFalse; + +@ExtendWith(MockitoExtension.class) +class LocalFileBufferTest { + + public static final String KEY = UUID.randomUUID().toString() + ".log"; + public static final String PREFIX = "local"; + public static final String SUFFIX = ".log"; + + private LocalFileBuffer localFileBuffer; + private File tempFile; + + @BeforeEach + void setUp() throws IOException { + tempFile = File.createTempFile(PREFIX, SUFFIX); + localFileBuffer = new LocalFileBuffer(tempFile); + } + + @Test + void test_with_write_events_into_buffer() throws IOException { + while (localFileBuffer.getEventCount() < 55) { + localFileBuffer.writeEvent(generateByteArray()); + } + assertThat(localFileBuffer.getSize(), greaterThan(1l)); + assertThat(localFileBuffer.getEventCount(), equalTo(55)); + assertThat(localFileBuffer.getDuration(), equalTo(0L)); + localFileBuffer.flushAndCloseStream(); + localFileBuffer.removeTemporaryFile(); + assertFalse(tempFile.exists(), "The temp file has not been deleted."); + } + + @Test + void test_without_write_events_into_buffer() { + assertThat(localFileBuffer.getSize(), equalTo(0L)); + assertThat(localFileBuffer.getEventCount(), equalTo(0)); + assertThat(localFileBuffer.getDuration(), equalTo(0L)); + localFileBuffer.flushAndCloseStream(); + localFileBuffer.removeTemporaryFile(); + assertFalse(tempFile.exists(), "The temp file has not been deleted."); + } + + @Test + void test_getSinkData_success() throws IOException{ + Assertions.assertNotNull(localFileBuffer); + assertDoesNotThrow(() -> { + localFileBuffer.getSinkBufferData(); + }); + } + + @AfterEach + void cleanup() { + tempFile.deleteOnExit(); + } + + private byte[] generateByteArray() { + byte[] bytes = new byte[1000]; + for (int i = 0; i < 1000; i++) { + bytes[i] = (byte) i; + } + return bytes; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/sink/ThresholdCheckTest.java b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/sink/ThresholdCheckTest.java new file mode 100644 index 0000000000..23494ecb3d --- /dev/null +++ b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/sink/ThresholdCheckTest.java @@ -0,0 +1,126 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.model.types.ByteCount; +import org.opensearch.dataprepper.plugins.accumulator.Buffer; +import org.opensearch.dataprepper.plugins.accumulator.InMemoryBufferFactory; + +import java.io.IOException; + +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +class ThresholdCheckTest { + + private Buffer inMemoryBuffer; + + @BeforeEach + void setUp() throws IOException { + inMemoryBuffer = new InMemoryBufferFactory().getBuffer(); + + while (inMemoryBuffer.getEventCount() < 100) { + inMemoryBuffer.writeEvent(generateByteArray()); + } + } + + @Test + void test_exceedThreshold_true_dueTo_maxEvents_is_less_than_buffered_event_count() { + final int maxEvents = 95; + final ByteCount maxBytes = ByteCount.parse("50kb"); + final long maxCollectionDuration = 15; + boolean isThresholdExceed = ThresholdCheck.checkThresholdExceed(inMemoryBuffer, maxEvents, + maxBytes, maxCollectionDuration); + assertTrue(isThresholdExceed, "Threshold not exceeded"); + } + + @Test + void test_exceedThreshold_false_dueTo_maxEvents_is_greater_than_buffered_event_count() { + final int maxEvents = 105; + final ByteCount maxBytes = ByteCount.parse("50mb"); + final long maxCollectionDuration = 50; + boolean isThresholdExceed = ThresholdCheck.checkThresholdExceed(inMemoryBuffer, maxEvents, maxBytes, + maxCollectionDuration); + assertFalse(isThresholdExceed, "Threshold exceeded"); + } + + @Test + void test_exceedThreshold_ture_dueTo_maxBytes_is_less_than_buffered_byte_count() { + final int maxEvents = 500; + final ByteCount maxBytes = ByteCount.parse("1b"); + final long maxCollectionDuration = 15; + boolean isThresholdExceed = ThresholdCheck.checkThresholdExceed(inMemoryBuffer, maxEvents, maxBytes, + maxCollectionDuration); + assertTrue(isThresholdExceed, "Threshold not exceeded"); + } + + @Test + void test_exceedThreshold_false_dueTo_maxBytes_is_greater_than_buffered_byte_count() { + final int maxEvents = 500; + final ByteCount maxBytes = ByteCount.parse("8mb"); + final long maxCollectionDuration = 15; + boolean isThresholdExceed = ThresholdCheck.checkThresholdExceed(inMemoryBuffer, maxEvents, + maxBytes, maxCollectionDuration); + assertFalse(isThresholdExceed, "Threshold exceeded"); + } + + @Test + void test_exceedThreshold_ture_dueTo_maxCollectionDuration_is_less_than_buffered_event_collection_duration() + throws IOException, InterruptedException { + final int maxEvents = 500; + final ByteCount maxBytes = ByteCount.parse("500mb"); + final long maxCollectionDuration = 10; + + inMemoryBuffer = new InMemoryBufferFactory().getBuffer(); + boolean isThresholdExceed = Boolean.FALSE; + synchronized (this) { + while (inMemoryBuffer.getEventCount() < 100) { + inMemoryBuffer.writeEvent(generateByteArray()); + isThresholdExceed = ThresholdCheck.checkThresholdExceed(inMemoryBuffer, maxEvents, + maxBytes, maxCollectionDuration); + if (isThresholdExceed) { + break; + } + wait(5000); + } + } + assertTrue(isThresholdExceed, "Threshold not exceeded"); + } + + @Test + void test_exceedThreshold_ture_dueTo_maxCollectionDuration_is_greater_than_buffered_event_collection_duration() + throws IOException, InterruptedException { + final int maxEvents = 500; + final ByteCount maxBytes = ByteCount.parse("500mb"); + final long maxCollectionDuration = 240; + + inMemoryBuffer = new InMemoryBufferFactory().getBuffer(); + + boolean isThresholdExceed = Boolean.FALSE; + synchronized (this) { + while (inMemoryBuffer.getEventCount() < 100) { + inMemoryBuffer.writeEvent(generateByteArray()); + isThresholdExceed = ThresholdCheck.checkThresholdExceed(inMemoryBuffer, + maxEvents, maxBytes, maxCollectionDuration); + if (isThresholdExceed) { + break; + } + wait(50); + } + } + assertFalse(isThresholdExceed, "Threshold exceeded"); + } + + private byte[] generateByteArray() { + byte[] bytes = new byte[10000]; + for (int i = 0; i < 10000; i++) { + bytes[i] = (byte) i; + } + return bytes; + } +} diff --git a/data-prepper-plugins/http-sink/build.gradle b/data-prepper-plugins/http-sink/build.gradle index da49c79c4b..21e678348f 100644 --- a/data-prepper-plugins/http-sink/build.gradle +++ b/data-prepper-plugins/http-sink/build.gradle @@ -2,11 +2,14 @@ dependencies { implementation project(':data-prepper-api') implementation project(':data-prepper-plugins:aws-plugin-api') + implementation project(':data-prepper-plugins:common') implementation 'com.fasterxml.jackson.core:jackson-core' implementation 'org.hibernate.validator:hibernate-validator:7.0.5.Final' implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' implementation 'software.amazon.awssdk:s3' implementation 'software.amazon.awssdk:sts' + implementation group: 'org.apache.httpcomponents.client5', name: 'httpclient5', version: '5.2' + implementation 'org.apache.commons:commons-lang3:3.12.0' } test { diff --git a/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/ClientFactory.java b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/ClientFactory.java new file mode 100644 index 0000000000..fdb3dcaaa0 --- /dev/null +++ b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/ClientFactory.java @@ -0,0 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink; + +/** + * AWS factory class required to create AWS Http Endpoint client for this plugin. + */ +public final class ClientFactory { + private ClientFactory() {} + +} diff --git a/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/HTTPSink.java b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/HTTPSink.java index 2c32235287..06d057b625 100644 --- a/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/HTTPSink.java +++ b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/HTTPSink.java @@ -4,6 +4,7 @@ */ package org.opensearch.dataprepper.plugins.sink; +import org.apache.hc.client5.http.impl.classic.CloseableHttpClient; import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; @@ -15,10 +16,13 @@ import org.opensearch.dataprepper.model.sink.AbstractSink; import org.opensearch.dataprepper.model.sink.Sink; import org.opensearch.dataprepper.plugins.sink.configuration.HttpSinkConfiguration; +import org.opensearch.dataprepper.plugins.sink.configuration.UrlConfigurationOption; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.Collection; +import java.util.List; +import java.util.Optional; @DataPrepperPlugin(name = "http", pluginType = Sink.class, pluginConfigurationType = HttpSinkConfiguration.class) public class HTTPSink extends AbstractSink> { @@ -73,4 +77,21 @@ public void doOutput(final Collection> records) { } //TODO: call Service call method } + + + public Optional getAuthHandlerByConfig(final HttpSinkConfiguration sinkConfiguration){ + //TODO: AWS Sigv4 - check + // TODO: call Auth Handlers based on auth Type + + return null; + } + + public List getClassicHttpRequestList(final List urlConfigurationOption){ + // logic for create auth handler for each url based on provided configuration - getAuthHandlerByConfig() + // logic for request preparation for each url + // logic for worker is not there in url level then verify the global workers if global workers also not defined then default 1 + // logic for get the Proxy object if url level proxy enabled else look the global proxy. + // Aws SageMaker headers if headers found in the configuration + return null; + } } \ No newline at end of file diff --git a/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/HttpAuthOptions.java b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/HttpAuthOptions.java new file mode 100644 index 0000000000..61b20993d8 --- /dev/null +++ b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/HttpAuthOptions.java @@ -0,0 +1,65 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink; + +import org.apache.hc.client5.http.impl.classic.CloseableHttpClient; +import org.apache.hc.core5.http.ClassicHttpRequest; + +public class HttpAuthOptions { + private String url; + + private CloseableHttpClient closeableHttpClient; + + private ClassicHttpRequest classicHttpRequest; + + private int workers; + + private String proxy; + + public CloseableHttpClient getCloseableHttpClient() { + return closeableHttpClient; + } + + public HttpAuthOptions setCloseableHttpClient(CloseableHttpClient closeableHttpClient) { + this.closeableHttpClient = closeableHttpClient; + return this; + } + + public ClassicHttpRequest getClassicHttpRequest() { + return classicHttpRequest; + } + + public HttpAuthOptions setClassicHttpRequest(ClassicHttpRequest classicHttpRequest) { + this.classicHttpRequest = classicHttpRequest; + return this; + } + + public int getWorkers() { + return workers; + } + + public HttpAuthOptions setWorkers(int workers) { + this.workers = workers; + return this; + } + + public String getUrl() { + return url; + } + + public HttpAuthOptions setUrl(String url) { + this.url = url; + return this; + } + + public String getProxy() { + return proxy; + } + + public HttpAuthOptions setProxy(String proxy) { + this.proxy = proxy; + return this; + } +} diff --git a/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/dlq/FailedDlqData.java b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/dlq/FailedDlqData.java new file mode 100644 index 0000000000..62a85cb709 --- /dev/null +++ b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/dlq/FailedDlqData.java @@ -0,0 +1,58 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.dlq; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import org.opensearch.dataprepper.model.event.EventHandle; + +import java.util.Objects; + +public class FailedDlqData { + + private final int status; + + private final String message; + + @JsonIgnore + private final EventHandle eventHandle; + + private FailedDlqData(final int status, + final String message, + final EventHandle eventHandle) { + this.status = status; + Objects.requireNonNull(message); + this.message = message; + this.eventHandle = eventHandle; + } + + public int getStatus() { + return status; + } + + public String getMessage() { + return message; + } + public EventHandle getEventHandle() { + return eventHandle; + } + + + public static Builder builder() { + return new Builder(); + } + + public static class Builder { + + private EventHandle eventHandle; + + private int status = 0; + + private String message; + + public FailedDlqData build() { + return new FailedDlqData(status, message, eventHandle); + } + } +} diff --git a/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/BasicAuthHttpSinkHandler.java b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/BasicAuthHttpSinkHandler.java new file mode 100644 index 0000000000..c84841c31e --- /dev/null +++ b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/BasicAuthHttpSinkHandler.java @@ -0,0 +1,17 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.handler; + +import org.opensearch.dataprepper.plugins.sink.configuration.HttpSinkConfiguration; + +import java.util.Optional; + +public class BasicAuthHttpSinkHandler implements MultiAuthHttpSinkHandler { + @Override + public Optional authenticate(HttpSinkConfiguration sinkConfiguration) { + // if ssl enabled then set connection manager + return null; + } +} diff --git a/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/BearerTokenAuthHttpSinkHandler.java b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/BearerTokenAuthHttpSinkHandler.java new file mode 100644 index 0000000000..fec474e6b8 --- /dev/null +++ b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/BearerTokenAuthHttpSinkHandler.java @@ -0,0 +1,17 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.handler; + +import org.opensearch.dataprepper.plugins.sink.configuration.HttpSinkConfiguration; + +import java.util.Optional; + +public class BearerTokenAuthHttpSinkHandler implements MultiAuthHttpSinkHandler { + @Override + public Optional authenticate(HttpSinkConfiguration sinkConfiguration) { + // if ssl enabled then set connection manager + return null; + } +} diff --git a/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/HttpAuthOptions.java b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/HttpAuthOptions.java new file mode 100644 index 0000000000..7ff5810b77 --- /dev/null +++ b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/HttpAuthOptions.java @@ -0,0 +1,61 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.handler; + +import org.apache.hc.client5.http.impl.classic.CloseableHttpClient; +import org.apache.hc.core5.http.ClassicHttpRequest; + +public class HttpAuthOptions { + private String url; + private CloseableHttpClient closeableHttpClient; + private ClassicHttpRequest classicHttpRequest; + private int workers; + private String proxy; + + public CloseableHttpClient getCloseableHttpClient() { + return closeableHttpClient; + } + + public HttpAuthOptions setCloseableHttpClient(CloseableHttpClient closeableHttpClient) { + this.closeableHttpClient = closeableHttpClient; + return this; + } + + public ClassicHttpRequest getClassicHttpRequest() { + return classicHttpRequest; + } + + public HttpAuthOptions setClassicHttpRequest(ClassicHttpRequest classicHttpRequest) { + this.classicHttpRequest = classicHttpRequest; + return this; + } + + public int getWorkers() { + return workers; + } + + public HttpAuthOptions setWorkers(int workers) { + this.workers = workers; + return this; + } + + public String getUrl() { + return url; + } + + public HttpAuthOptions setUrl(String url) { + this.url = url; + return this; + } + + public String getProxy() { + return proxy; + } + + public HttpAuthOptions setProxy(String proxy) { + this.proxy = proxy; + return this; + } +} diff --git a/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/MultiAuthHttpSinkHandler.java b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/MultiAuthHttpSinkHandler.java new file mode 100644 index 0000000000..e0db436915 --- /dev/null +++ b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/MultiAuthHttpSinkHandler.java @@ -0,0 +1,14 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.handler; + +import org.opensearch.dataprepper.plugins.sink.configuration.HttpSinkConfiguration; + +import java.util.Optional; + +public interface MultiAuthHttpSinkHandler { + Optional authenticate(final HttpSinkConfiguration sinkConfiguration); + +} diff --git a/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/SecuredAuthHttpSinkHandler.java b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/SecuredAuthHttpSinkHandler.java new file mode 100644 index 0000000000..9fb58fe223 --- /dev/null +++ b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/handler/SecuredAuthHttpSinkHandler.java @@ -0,0 +1,18 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.handler; + +import org.opensearch.dataprepper.plugins.sink.configuration.HttpSinkConfiguration; + +import java.util.Optional; + +public class SecuredAuthHttpSinkHandler implements MultiAuthHttpSinkHandler { + @Override + public Optional authenticate(HttpSinkConfiguration sinkConfiguration) { + // logic here to read the certs from ACM/S3/local + // SSL Sigv4 validation and verification and make connection + return null; + } +} diff --git a/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/service/HttpSinkService.java b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/service/HttpSinkService.java new file mode 100644 index 0000000000..8f97dbb5b0 --- /dev/null +++ b/data-prepper-plugins/http-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/service/HttpSinkService.java @@ -0,0 +1,59 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.service; + +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.types.ByteCount; +import org.opensearch.dataprepper.plugins.accumulator.BufferFactory; +import org.opensearch.dataprepper.plugins.sink.configuration.HttpSinkConfiguration; +import org.opensearch.dataprepper.plugins.sink.handler.HttpAuthOptions; +import org.opensearch.dataprepper.model.codec.OutputCodec; + +import java.util.Collection; +import java.util.List; +public class HttpSinkService { + + private final HttpSinkConfiguration httpSinkConf; + + private final BufferFactory bufferFactory; + + private final List httpAuthOptions; + private OutputCodec codec; + + public HttpSinkService(final OutputCodec codec, + final HttpSinkConfiguration httpSinkConf, + final BufferFactory bufferFactory, + final List httpAuthOptions){ + this.codec= codec; + this.httpSinkConf = httpSinkConf; + this.bufferFactory = bufferFactory; + this.httpAuthOptions = httpAuthOptions; + } + + public void processRecords(Collection> records) { + records.forEach(record -> { + try{ + // logic to fetch the records in batch as per threshold limit - checkThresholdExceed(); + // apply the codec + // push to http end point + }catch(Exception e){ + // In case of any exception, need to write the exception in dlq - logFailureForDlqObjects(); + // In case of any exception, need to push the web hook url- logFailureForWebHook(); + } + //TODO: implement end to end acknowledgement + }); + } + + public static boolean checkThresholdExceed(final Buffer currentBuffer, + final int maxEvents, + final ByteCount maxBytes, + final long maxCollectionDuration) { + // logic for checking the threshold + return true; + } + +} From 92af936d39b55d4f4fe0c413a57606113fb7150a Mon Sep 17 00:00:00 2001 From: kkondaka <41027584+kkondaka@users.noreply.github.com> Date: Wed, 5 Jul 2023 10:15:15 -0700 Subject: [PATCH 09/10] Fix Stdout and File sink (#2978) * Fix Stdout and File sink Signed-off-by: Krishna Kondaka * Fixed javadoc warnings and errors Signed-off-by: Krishna Kondaka --------- Signed-off-by: Krishna Kondaka Co-authored-by: Krishna Kondaka --- .../ArmeriaHttpAuthenticationProvider.java | 1 + .../authentication/GrpcAuthenticationProvider.java | 4 +++- .../plugins/s3keyindex/S3ObjectIndexUtility.java | 10 +++++++++- .../opensearch/dataprepper/plugins/sink/FileSink.java | 1 + .../dataprepper/plugins/sink/StdOutSink.java | 3 +++ 5 files changed, 17 insertions(+), 2 deletions(-) diff --git a/data-prepper-plugins/armeria-common/src/main/java/org/opensearch/dataprepper/armeria/authentication/ArmeriaHttpAuthenticationProvider.java b/data-prepper-plugins/armeria-common/src/main/java/org/opensearch/dataprepper/armeria/authentication/ArmeriaHttpAuthenticationProvider.java index 489ddcd125..dc1e157ca1 100644 --- a/data-prepper-plugins/armeria-common/src/main/java/org/opensearch/dataprepper/armeria/authentication/ArmeriaHttpAuthenticationProvider.java +++ b/data-prepper-plugins/armeria-common/src/main/java/org/opensearch/dataprepper/armeria/authentication/ArmeriaHttpAuthenticationProvider.java @@ -30,6 +30,7 @@ public interface ArmeriaHttpAuthenticationProvider { * Gets an authentication decorator to an Armeria {@link ServerBuilder}. * * @since 2.0 + * @return returns authentication decorator */ default Optional> getAuthenticationDecorator() { return Optional.empty(); diff --git a/data-prepper-plugins/armeria-common/src/main/java/org/opensearch/dataprepper/armeria/authentication/GrpcAuthenticationProvider.java b/data-prepper-plugins/armeria-common/src/main/java/org/opensearch/dataprepper/armeria/authentication/GrpcAuthenticationProvider.java index e933caf7c3..cde38b31fe 100644 --- a/data-prepper-plugins/armeria-common/src/main/java/org/opensearch/dataprepper/armeria/authentication/GrpcAuthenticationProvider.java +++ b/data-prepper-plugins/armeria-common/src/main/java/org/opensearch/dataprepper/armeria/authentication/GrpcAuthenticationProvider.java @@ -29,6 +29,7 @@ public interface GrpcAuthenticationProvider { /** * Returns a {@link ServerInterceptor} that does authentication * @since 1.2 + * @return returns authentication interceptor */ ServerInterceptor getAuthenticationInterceptor(); @@ -36,8 +37,9 @@ public interface GrpcAuthenticationProvider { * Allows implementors to provide an {@link HttpService} to either intercept the HTTP request prior to validation, * or to perform validation on the HTTP request. This may be optional, in which case it is not used. * @since 1.5 + * @return returns http authentication service */ default Optional> getHttpAuthenticationService() { return Optional.empty(); } -} \ No newline at end of file +} diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/s3keyindex/S3ObjectIndexUtility.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/s3keyindex/S3ObjectIndexUtility.java index 27d54cf44b..bfcac56130 100644 --- a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/s3keyindex/S3ObjectIndexUtility.java +++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/s3keyindex/S3ObjectIndexUtility.java @@ -38,6 +38,8 @@ public class S3ObjectIndexUtility { /** * Create Object Name with date,time and UniqueID prepended. + * @param indexAlias index alias + * @return returns string of object name with date time */ public static String getObjectNameWithDateTimeId(final String indexAlias) { DateTimeFormatter dateFormatter = validateAndGetDateTimeFormatter(indexAlias); @@ -48,6 +50,8 @@ public static String getObjectNameWithDateTimeId(final String indexAlias) { /** * Create Object path prefix. + * @param indexAlias index alias + * @return returns string of object path prefix */ public static String getObjectPathPrefix(final String indexAlias) { DateTimeFormatter dateFormatter = validateAndGetDateTimeFormatter(indexAlias); @@ -57,6 +61,7 @@ public static String getObjectPathPrefix(final String indexAlias) { /** * Creates epoch seconds. + * @return returns time since epoch in nanoseconds */ public static long getTimeNanos() { Instant time = Instant.now(); @@ -67,6 +72,8 @@ public static long getTimeNanos() { /** * Validate the index with the regular expression pattern. Throws exception if validation fails + * @param indexAlias index alias + * @return returns date time formatter */ public static DateTimeFormatter validateAndGetDateTimeFormatter(final String indexAlias) { final Pattern pattern = Pattern.compile(TIME_PATTERN_INTERNAL_EXTRACTOR_REGULAR_EXPRESSION); @@ -128,8 +135,9 @@ public static void validateTimePatternGranularity(String timePattern) { /** * Returns the current UTC Date and Time + * @return returns current UTC time */ public static ZonedDateTime getCurrentUtcTime() { return LocalDateTime.now().atZone(ZoneId.systemDefault()).withZoneSameInstant(UTC_ZONE_ID); } -} \ No newline at end of file +} diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/FileSink.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/FileSink.java index fc3df248fa..e0f2a94b0d 100644 --- a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/FileSink.java +++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/FileSink.java @@ -48,6 +48,7 @@ public class FileSink implements Sink> { * pluginSetting file. * * @param fileSinkConfig The file sink configuration + * @param sinkContext sink context */ @DataPrepperPluginConstructor public FileSink(final FileSinkConfig fileSinkConfig, final SinkContext sinkContext) { diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/StdOutSink.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/StdOutSink.java index e43c25f658..0396af3176 100644 --- a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/StdOutSink.java +++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/StdOutSink.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.sink; 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.event.EventHandle; @@ -27,7 +28,9 @@ public class StdOutSink implements Sink> { * pluginSetting file. * * @param pluginSetting instance with metadata information from pipeline pluginSetting file. + * @param sinkContext sink context */ + @DataPrepperPluginConstructor public StdOutSink(final PluginSetting pluginSetting, final SinkContext sinkContext) { this(Objects.nonNull(sinkContext) ? sinkContext.getTagsTargetKey() : null); } From 75fa735289ecf8d335d5681fa63a512e8a4ee03e Mon Sep 17 00:00:00 2001 From: Adi Suresh Date: Wed, 5 Jul 2023 16:20:12 -0500 Subject: [PATCH 10/10] Fix S3 errors around end of file behavior. (#2983) Signed-off-by: Adi Suresh --- .../plugins/source/S3InputStream.java | 57 ++++++++++++------- .../plugins/source/S3InputStreamTest.java | 41 +++++++++++++ 2 files changed, 77 insertions(+), 21 deletions(-) diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3InputStream.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3InputStream.java index 0629db6831..14f8a509ff 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3InputStream.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3InputStream.java @@ -129,11 +129,15 @@ public int read() throws IOException { Preconditions.checkState(!closed, "Cannot read: already closed"); positionStream(); - pos += 1; - next += 1; - bytesCounter.increment(); + final int byteRead = stream.read(); - return stream.read(); + if (byteRead != -1) { + pos += 1; + next += 1; + bytesCounter.increment(); + } + + return byteRead; } /** @@ -161,10 +165,13 @@ public int read(byte[] b, int off, int len) throws IOException { Preconditions.checkState(!closed, "Cannot read: already closed"); positionStream(); - int bytesRead = stream.read(b, off, len); - pos += bytesRead; - next += bytesRead; - bytesCounter.add(bytesRead); + final int bytesRead = stream.read(b, off, len); + + if (bytesRead > 0) { + pos += bytesRead; + next += bytesRead; + bytesCounter.add(bytesRead); + } return bytesRead; } @@ -203,9 +210,11 @@ public int readNBytes(byte[] b, int off, int len) throws IOException { final int bytesRead = stream.readNBytes(b, off, len); - pos += bytesRead; - next += bytesRead; - bytesCounter.add(bytesRead); + if (bytesRead > 0) { + pos += bytesRead; + next += bytesRead; + bytesCounter.add(bytesRead); + } return bytesRead; } @@ -325,9 +334,11 @@ public void readFully(byte[] bytes, int start, int len) throws IOException { int bytesRead = readFully(stream, bytes, start, len); - this.pos += bytesRead; - this.next += bytesRead; - this.bytesCounter.add(bytesRead); + if (bytesRead > 0) { + this.pos += bytesRead; + this.next += bytesRead; + this.bytesCounter.add(bytesRead); + } } /** @@ -354,9 +365,11 @@ public int read(ByteBuffer buf) throws IOException { bytesRead = readDirectBuffer(stream, buf, temp); } - this.pos += bytesRead; - this.next += bytesRead; - this.bytesCounter.add(bytesRead); + if (bytesRead > 0) { + this.pos += bytesRead; + this.next += bytesRead; + this.bytesCounter.add(bytesRead); + } return bytesRead; } @@ -385,9 +398,11 @@ public void readFully(ByteBuffer buf) throws IOException { bytesRead = readFullyDirectBuffer(stream, buf, temp); } - this.pos += bytesRead; - this.next += bytesRead; - this.bytesCounter.add(bytesRead); + if (bytesRead > 0) { + this.pos += bytesRead; + this.next += bytesRead; + this.bytesCounter.add(bytesRead); + } } /** @@ -478,7 +493,7 @@ private void closeStream() throws IOException { */ private void abortStream() { try { - if (stream instanceof Abortable && stream.read() != -1) { + if (stream instanceof Abortable) { ((Abortable) stream).abort(); } } catch (Exception e) { diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/S3InputStreamTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/S3InputStreamTest.java index 1ac31890ee..d75254189d 100644 --- a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/S3InputStreamTest.java +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/S3InputStreamTest.java @@ -15,6 +15,7 @@ import software.amazon.awssdk.services.s3.model.S3Exception; import java.io.ByteArrayInputStream; +import java.io.EOFException; import java.io.IOException; import java.io.InputStream; import java.nio.ByteBuffer; @@ -103,6 +104,19 @@ void testRead() throws IOException { verify(s3ObjectSizeProcessedSummary).record(1.0); } + @Test + void testReadEndOfFile() throws IOException { + InputStream inputStream = new ByteArrayInputStream("".getBytes()); + when(s3Client.getObject(any(GetObjectRequest.class), any(ResponseTransformer.class))).thenReturn(inputStream); + + int firstByte = s3InputStream.read(); + assertEquals(-1, firstByte); + + s3InputStream.close(); + + verify(s3ObjectSizeProcessedSummary).record(0.0); + } + @Test void testReadByteArray() throws IOException { InputStream inputStream = new ByteArrayInputStream("Test data".getBytes()); @@ -150,6 +164,20 @@ void testReadNBytes_intoArray() throws Exception { verify(s3ObjectSizeProcessedSummary).record(4.0); } + @Test + void testReadNBytes_endOfFile() throws Exception { + InputStream inputStream = new ByteArrayInputStream("".getBytes()); + when(s3Client.getObject(any(GetObjectRequest.class), any(ResponseTransformer.class))).thenReturn(inputStream); + + byte[] buffer = new byte[9]; + int bytesRead = s3InputStream.readNBytes(buffer, 0, 4); + + assertEquals(0, bytesRead); + + s3InputStream.close(); + verify(s3ObjectSizeProcessedSummary).record(0.0); + } + @Test void testReadNBytes_getArray() throws Exception { InputStream inputStream = new ByteArrayInputStream("Test data".getBytes()); @@ -242,6 +270,19 @@ void testReadFullyByteBuffer() throws IOException { verify(s3ObjectSizeProcessedSummary).record(4.0); } + @Test + void testReadFullyByteBuffer_endOfFile() throws IOException { + InputStream inputStream = new ByteArrayInputStream("".getBytes()); + when(s3Client.getObject(any(GetObjectRequest.class), any(ResponseTransformer.class))).thenReturn(inputStream); + s3InputStream.seek(0); // Force opening the stream + + ByteBuffer buffer = ByteBuffer.allocate(4); + assertThrows(EOFException.class, () -> s3InputStream.readFully(buffer)); + + s3InputStream.close(); + verify(s3ObjectSizeProcessedSummary).record(0.0); + } + @Test void testReadFullyHeapBuffer() throws IOException { InputStream inputStream = new ByteArrayInputStream("Test data".getBytes());