From 47a9bc0b99590d4b6554371a38860964947a06fd Mon Sep 17 00:00:00 2001 From: David Venable Date: Thu, 10 Aug 2023 10:21:05 -0500 Subject: [PATCH 01/28] Updates to the Avro codec README.md to include an example of using an Avro schema for VPC Flow Logs. Updates the YAML to make the string easier to handle. (#3111) Signed-off-by: David Venable --- data-prepper-plugins/avro-codecs/README.md | 33 ++++++++++++++++------ 1 file changed, 24 insertions(+), 9 deletions(-) diff --git a/data-prepper-plugins/avro-codecs/README.md b/data-prepper-plugins/avro-codecs/README.md index 9423b5380c..7ed68e95d6 100644 --- a/data-prepper-plugins/avro-codecs/README.md +++ b/data-prepper-plugins/avro-codecs/README.md @@ -1,6 +1,6 @@ # Avro Sink/Output Codec -This is an implementation of Avro Sink Codec that parses the Dataprepper Events into avro records and writes them into the underlying OutputStream. +This is an implementation of Avro Sink Codec that parses the Data Prepper Events into Avro records and writes them into the underlying OutputStream. ## Usages @@ -20,20 +20,35 @@ pipeline: max_retries: 5 bucket: bucket_name object_key: - path_prefix: my-elb/%{yyyy}/%{MM}/%{dd}/ + path_prefix: vpc-flow-logs/%{yyyy}/%{MM}/%{dd}/ threshold: event_count: 2000 maximum_size: 50mb event_collect_timeout: 15s codec: avro: - schema: "{\"namespace\": \"org.example.test\"," + - " \"type\": \"record\"," + - " \"name\": \"TestMessage\"," + - " \"fields\": [" + - " {\"name\": \"name\", \"type\": \"string\"}," + - " {\"name\": \"age\", \"type\": \"int\"}]" + - "}"; + schema: > + { + "type" : "record", + "namespace" : "org.opensearch.dataprepper.examples", + "name" : "VpcFlowLog", + "fields" : [ + { "name" : "version", "type" : ["null", "string"]}, + { "name" : "srcport", "type": ["null", "int"]}, + { "name" : "dstport", "type": ["null", "int"]}, + { "name" : "accountId", "type" : ["null", "string"]}, + { "name" : "interfaceId", "type" : ["null", "string"]}, + { "name" : "srcaddr", "type" : ["null", "string"]}, + { "name" : "dstaddr", "type" : ["null", "string"]}, + { "name" : "start", "type": ["null", "int"]}, + { "name" : "end", "type": ["null", "int"]}, + { "name" : "protocol", "type": ["null", "int"]}, + { "name" : "packets", "type": ["null", "int"]}, + { "name" : "bytes", "type": ["null", "int"]}, + { "name" : "action", "type": ["null", "string"]}, + { "name" : "logStatus", "type" : ["null", "string"]} + ] + } exclude_keys: - s3 buffer_type: in_memory From 244524b5ed74e12cdd075ee19261e0bc25931b31 Mon Sep 17 00:00:00 2001 From: David Venable Date: Thu, 10 Aug 2023 11:22:33 -0500 Subject: [PATCH 02/28] Improved Avro error reporting related to schemas (#3110) Check the Avro schema when starting Data Prepper and throw a clearer exception when an Avro field is missing from the schema. Some code clean-up. Signed-off-by: David Venable --- .../plugins/codec/avro/AvroOutputCodec.java | 35 ++++++++------- .../plugins/codec/avro/AvroSchemaParser.java | 19 ++++---- .../codec/avro/AvroSchemaParserFromS3.java | 13 +++--- .../codec/avro/AvroOutputCodecTest.java | 44 +++++++++++++++++-- 4 files changed, 76 insertions(+), 35 deletions(-) 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 0bdc919bd8..fc5e298be1 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 @@ -4,7 +4,6 @@ */ package org.opensearch.dataprepper.plugins.codec.avro; -import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.avro.Schema; import org.apache.avro.file.DataFileWriter; import org.apache.avro.generic.GenericData; @@ -34,9 +33,8 @@ @DataPrepperPlugin(name = "avro", pluginType = OutputCodec.class, pluginConfigurationType = AvroOutputCodecConfig.class) public class AvroOutputCodec implements OutputCodec { - private static final List nonComplexTypes = Arrays.asList("int", "long", "string", "float", "double", "bytes"); + private static final List PRIMITIVE_TYPES = Arrays.asList("int", "long", "string", "float", "double", "bytes"); private static final Logger LOG = LoggerFactory.getLogger(AvroOutputCodec.class); - private static final ObjectMapper objectMapper = new ObjectMapper(); private static final String AVRO = "avro"; private static final String BASE_SCHEMA_STRING = "{\"type\":\"record\",\"name\":\"AvroRecords\",\"fields\":["; private static final String END_SCHEMA_STRING = "]}"; @@ -52,13 +50,7 @@ public class AvroOutputCodec implements OutputCodec { public AvroOutputCodec(final AvroOutputCodecConfig config) { Objects.requireNonNull(config); this.config = config; - } - @Override - public void start(final OutputStream outputStream, final Event event, final OutputCodecContext codecContext) throws IOException { - Objects.requireNonNull(outputStream); - Objects.requireNonNull(codecContext); - this.codecContext = codecContext; if (config.getSchema() != null) { schema = parseSchema(config.getSchema()); } else if (config.getFileLocation() != null) { @@ -67,10 +59,18 @@ public void start(final OutputStream outputStream, final Event event, final Outp schema = parseSchema(AvroSchemaParserFromSchemaRegistry.getSchemaType(config.getSchemaRegistryUrl())); } else if (checkS3SchemaValidity()) { schema = AvroSchemaParserFromS3.parseSchema(config); - } else { + } + } + + @Override + public void start(final OutputStream outputStream, final Event event, final OutputCodecContext codecContext) throws IOException { + Objects.requireNonNull(outputStream); + Objects.requireNonNull(codecContext); + this.codecContext = codecContext; + if (schema == null) { schema = buildInlineSchemaFromEvent(event); } - final DatumWriter datumWriter = new GenericDatumWriter(schema); + final DatumWriter datumWriter = new GenericDatumWriter<>(schema); dataFileWriter = new DataFileWriter<>(datumWriter); dataFileWriter.create(schema, outputStream); } @@ -159,13 +159,13 @@ public String getExtension() { return AVRO; } - Schema parseSchema(final String schemaString) throws IOException { + Schema parseSchema(final String schemaString) { try { Objects.requireNonNull(schemaString); return new Schema.Parser().parse(schemaString); } catch (Exception e) { - LOG.error("Unable to parse Schema from Schema String provided."); - throw new IOException("Can't proceed without schema."); + LOG.error("Unable to parse Schema from Schema String provided.", e); + throw new RuntimeException("There is an error in the schema: " + e.getMessage()); } } @@ -177,6 +177,9 @@ private GenericRecord buildAvroRecord(final Schema schema, final Map jsonMap; try { jsonMap = mapper.readValue(Paths.get(location).toFile(), Map.class); - } catch (FileNotFoundException e) { + } catch (IOException e) { LOG.error("Schema file not found, Error: {}", e.getMessage()); - throw new IOException("Can't proceed without schema."); + throw new RuntimeException("Can't proceed without schema."); } - final Map schemaMap = new HashMap(); + final Map schemaMap = new HashMap(); for (Map.Entry entry : jsonMap.entrySet()) { schemaMap.put(entry.getKey(), entry.getValue()); } - try{ + try { return new Schema.Parser().parse(mapper.writeValueAsString(schemaMap)); - }catch(Exception e) { + } catch (Exception e) { LOG.error("Unable to parse schema from the provided schema file, Error: {}", e.getMessage()); - throw new IOException("Can't proceed without schema."); + throw new RuntimeException("Can't proceed without schema."); } } } diff --git a/data-prepper-plugins/avro-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/avro/AvroSchemaParserFromS3.java b/data-prepper-plugins/avro-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/avro/AvroSchemaParserFromS3.java index 906aa5cecf..28eacc21b0 100644 --- a/data-prepper-plugins/avro-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/avro/AvroSchemaParserFromS3.java +++ b/data-prepper-plugins/avro-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/avro/AvroSchemaParserFromS3.java @@ -28,12 +28,12 @@ public class AvroSchemaParserFromS3 { private static final ApacheHttpClient.Builder apacheHttpClientBuilder = ApacheHttpClient.builder(); private static final Logger LOG = LoggerFactory.getLogger(AvroSchemaParserFromS3.class); - public static Schema parseSchema(AvroOutputCodecConfig config) throws IOException { - try{ + public static Schema parseSchema(AvroOutputCodecConfig config) { + try { return new Schema.Parser().parse(getS3SchemaObject(config)); - }catch (Exception e){ - LOG.error("Unable to retrieve schema from S3. Error: "+e.getMessage()); - throw new IOException("Can't proceed without schema."); + } catch (Exception e) { + LOG.error("Unable to retrieve schema from S3.", e); + throw new RuntimeException("There is an error in the schema: " + e.getMessage()); } } @@ -44,7 +44,8 @@ private static String getS3SchemaObject(AvroOutputCodecConfig config) throws IOE .key(config.getFileKey()) .build(); ResponseInputStream s3Object = s3Client.getObject(getObjectRequest); - final Map stringObjectMap = objectMapper.readValue(s3Object, new TypeReference<>() {}); + final Map stringObjectMap = objectMapper.readValue(s3Object, new TypeReference<>() { + }); return objectMapper.writeValueAsString(stringObjectMap); } diff --git a/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/avro/AvroOutputCodecTest.java b/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/avro/AvroOutputCodecTest.java index e9010f44de..f632bb88dc 100644 --- a/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/avro/AvroOutputCodecTest.java +++ b/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/avro/AvroOutputCodecTest.java @@ -11,6 +11,7 @@ import org.apache.avro.generic.GenericRecord; import org.apache.avro.util.Utf8; import org.hamcrest.Matchers; +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; @@ -25,15 +26,21 @@ import java.io.UnsupportedEncodingException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.UUID; +import static org.hamcrest.CoreMatchers.containsString; 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.mock; +import static org.mockito.Mockito.when; public class AvroOutputCodecTest { - private static final String expectedSchemaString = "{\"type\":\"record\",\"name\":\"AvroRecords\",\"fields\":[{\"name\"" + + private static final String EXPECTED_SCHEMA_STRING = "{\"type\":\"record\",\"name\":\"AvroRecords\",\"fields\":[{\"name\"" + ":\"name\",\"type\":\"string\"},{\"name\":\"nestedRecord\",\"type\":{\"type\":\"record\",\"name\":" + "\"NestedRecord1\",\"fields\":[{\"name\":\"secondFieldInNestedRecord\",\"type\":\"int\"},{\"name\":\"" + "firstFieldInNestedRecord\",\"type\":\"string\"}]}},{\"name\":\"age\",\"type\":\"int\"}]}"; @@ -43,12 +50,27 @@ public class AvroOutputCodecTest { private static int numberOfRecords; - private AvroOutputCodec createObjectUnderTest() { + @BeforeEach + void setUp() { config = new AvroOutputCodecConfig(); config.setSchema(parseSchema().toString()); + } + + private AvroOutputCodec createObjectUnderTest() { return new AvroOutputCodec(config); } + @Test + void constructor_throws_if_schema_is_invalid() { + String invalidSchema = parseSchema().toString().replaceAll(",", ";"); + config.setSchema(invalidSchema); + + RuntimeException actualException = assertThrows(RuntimeException.class, this::createObjectUnderTest); + + assertThat(actualException.getMessage(), notNullValue()); + assertThat(actualException.getMessage(), containsString(invalidSchema)); + assertThat(actualException.getMessage(), containsString("was expecting comma")); + } @ParameterizedTest @ValueSource(ints = {1, 2, 10, 100}) @@ -86,9 +108,25 @@ void test_happy_case(final int numberOfRecords) throws Exception { } } + @Test + void writeEvent_throws_exception_when_field_does_not_exist() throws IOException { + final Event eventWithInvalidField = mock(Event.class); + final String invalidFieldName = UUID.randomUUID().toString(); + when(eventWithInvalidField.toMap()).thenReturn(Collections.singletonMap(invalidFieldName, UUID.randomUUID().toString())); + final AvroOutputCodec objectUnderTest = createObjectUnderTest(); + + outputStream = new ByteArrayOutputStream(); + objectUnderTest.start(outputStream, null, new OutputCodecContext()); + + final RuntimeException actualException = assertThrows(RuntimeException.class, () -> objectUnderTest.writeEvent(eventWithInvalidField, outputStream)); + + assertThat(actualException.getMessage(), notNullValue()); + assertThat(actualException.getMessage(), containsString(invalidFieldName)); + } + @Test public void testInlineSchemaBuilder() throws IOException { - Schema expectedSchema = new Schema.Parser().parse(expectedSchemaString); + Schema expectedSchema = new Schema.Parser().parse(EXPECTED_SCHEMA_STRING); AvroOutputCodec avroOutputCodec = createObjectUnderTest(); numberOfRecords = 1; Event event = getRecord(0).getData(); From 37df5bc00b7b080d912e8d79d722e68a860d5d67 Mon Sep 17 00:00:00 2001 From: David Venable Date: Thu, 10 Aug 2023 11:51:16 -0500 Subject: [PATCH 03/28] Update the developer_guide.md with updated instructions for user documentation. Updated the PR template to include documentation, and updated the Issues Resolved section to encourage use of "Resolves #". (#2772) Signed-off-by: David Venable --- .github/pull_request_template.md | 4 ++-- docs/developer_guide.md | 19 ++++++++++++++++--- 2 files changed, 18 insertions(+), 5 deletions(-) diff --git a/.github/pull_request_template.md b/.github/pull_request_template.md index ba11315f02..3d0dd79177 100644 --- a/.github/pull_request_template.md +++ b/.github/pull_request_template.md @@ -2,11 +2,11 @@ [Describe what this change achieves] ### Issues Resolved -[List any issues this PR will resolve] +Resolves #[Issue number to be closed when this PR is merged] ### Check List - [ ] New functionality includes testing. -- [ ] New functionality has been documented. +- [ ] New functionality has a documentation issue. Please link to it in this PR. - [ ] New functionality has javadoc added - [ ] Commits are signed with a real name per the DCO diff --git a/docs/developer_guide.md b/docs/developer_guide.md index 9d892d3e2f..1a6587d5b3 100644 --- a/docs/developer_guide.md +++ b/docs/developer_guide.md @@ -157,9 +157,22 @@ issue to request that a previous change be backported. Documentation is very important for users of Data Prepper and contributors. We are using the following conventions for documentation. -1. Document features in markdown. Plugins should have detailed documentation in a `README.md` file in the plugin project directory. Documentation for all of Data Prepper should be included in the [Data Prepper documentation](https://opensearch.org/docs/latest/data-prepper/index/). -2. Provide Javadocs for all public classes, methods, and fields. Plugins need not follow this guidance since their classes are generally not exposed. -3. Avoid commenting within code, unless it is required to understand that code. +1. Document features in the OpenSearch [documentation-website](https://github.com/opensearch-project/documentation-website). This makes the documentation available at https://opensearch.org/docs/latest/data-prepper. +2. Document any development guidance in this repository in markdown. In particular, for plugins, use the `README.md` file within the plugin project to note anything developers should keep in mind. +3. Provide Javadocs for all public classes, methods, and fields. Plugins need not follow this guidance since their classes are generally not exposed. +4. Avoid commenting within code, unless it is required to understand that code. + +#### Documentation Process + +When you submit a feature PR, please be sure to also submit a new "Documentation issue" +[issue in the documentation-website](https://github.com/opensearch-project/documentation-website/issues/new/choose) project. + +Please include in this feature a link to the GitHub issue which has information on the feature. +This GitHub issue will often have sample configurations and explanations of the options available to users. +Please also provide any additional guidance for the team doing the documentation. +Please include a link to that documentation issue in the PR you created for Data Prepper. + +You are also welcome to submit a PR directly in the [documentation-website](https://github.com/opensearch-project/documentation-website). ### Code From 44e2eaf78291170e7f281ed60323b754a52ce5cc Mon Sep 17 00:00:00 2001 From: kkondaka <41027584+kkondaka@users.noreply.github.com> Date: Thu, 10 Aug 2023 11:38:43 -0700 Subject: [PATCH 04/28] Add metrics to Kafka Source (#3118) * Add metrics to Kafka Source Signed-off-by: Krishna Kondaka * Removed debug print statement Signed-off-by: Krishna Kondaka * Fixed failing test case Signed-off-by: Krishna Kondaka * Added total committed metric and fixed tests Signed-off-by: Krishna Kondaka * Addressed review comments Signed-off-by: Krishna Kondaka * Fixed number of committed records stat. Also fixed bug when acknowledgements enabled Signed-off-by: Krishna Kondaka * Addressed review comments. Fixed acknowledgements related bug Signed-off-by: Krishna Kondaka * Fixed to use counters for records/bytes consumed metrics Signed-off-by: Krishna Kondaka * Removed unused code Signed-off-by: Krishna Kondaka * Added a metric for keeping track of number of consumers without any partitions assigned Signed-off-by: Krishna Kondaka * Added unit test for KafkaTopicMetrics Signed-off-by: Krishna Kondaka --------- Signed-off-by: Krishna Kondaka Co-authored-by: Krishna Kondaka --- .../kafka/source/KafkaSourceJsonTypeIT.java | 1 + .../source/KafkaSourceMultipleAuthTypeIT.java | 1 + .../consumer/KafkaSourceCustomConsumer.java | 120 +++++---- .../consumer/TopicPartitionCommitTracker.java | 2 +- .../plugins/kafka/source/KafkaSource.java | 6 +- .../util/KafkaSourceSecurityConfigurer.java | 2 +- .../plugins/kafka/util/KafkaTopicMetrics.java | 215 +++++++++++++++ .../KafkaSourceCustomConsumerTest.java | 16 +- .../TopicPartitionCommitTrackerTest.java | 2 +- .../kafka/util/KafkaTopicMetricsTests.java | 250 ++++++++++++++++++ 10 files changed, 557 insertions(+), 58 deletions(-) create mode 100644 data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaTopicMetrics.java create mode 100644 data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaTopicMetricsTests.java diff --git a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceJsonTypeIT.java b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceJsonTypeIT.java index 584ee24eb5..dff3d2b943 100644 --- a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceJsonTypeIT.java +++ b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceJsonTypeIT.java @@ -119,6 +119,7 @@ public void setup() { when(jsonTopic.getName()).thenReturn(testTopic); when(jsonTopic.getGroupId()).thenReturn(testGroup); when(jsonTopic.getWorkers()).thenReturn(1); + when(jsonTopic.getMaxPollInterval()).thenReturn(Duration.ofSeconds(5)); when(jsonTopic.getSessionTimeOut()).thenReturn(Duration.ofSeconds(15)); when(jsonTopic.getHeartBeatInterval()).thenReturn(Duration.ofSeconds(3)); when(jsonTopic.getAutoCommit()).thenReturn(false); diff --git a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceMultipleAuthTypeIT.java b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceMultipleAuthTypeIT.java index 6d52bba0ea..6179ba4f57 100644 --- a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceMultipleAuthTypeIT.java +++ b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceMultipleAuthTypeIT.java @@ -130,6 +130,7 @@ public void setup() { when(plainTextTopic.getName()).thenReturn(testTopic); when(plainTextTopic.getGroupId()).thenReturn(testGroup); when(plainTextTopic.getWorkers()).thenReturn(1); + when(plainTextTopic.getMaxPollInterval()).thenReturn(Duration.ofSeconds(5)); when(plainTextTopic.getSessionTimeOut()).thenReturn(Duration.ofSeconds(15)); when(plainTextTopic.getHeartBeatInterval()).thenReturn(Duration.ofSeconds(3)); when(plainTextTopic.getAutoCommit()).thenReturn(false); 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 e1a071ffcb..d083b4e98b 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,7 +7,6 @@ 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,11 +18,11 @@ import org.apache.kafka.common.TopicPartition; import org.apache.avro.generic.GenericRecord; import org.opensearch.dataprepper.model.log.JacksonLog; -import org.opensearch.dataprepper.metrics.PluginMetrics; 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.event.EventMetadata; +import org.opensearch.dataprepper.model.buffer.SizeOverflowException; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; @@ -34,17 +33,20 @@ import org.slf4j.LoggerFactory; import java.time.Duration; -import java.util.Map; -import java.util.HashMap; +import java.time.Instant; import java.util.Arrays; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Set; -import java.util.HashSet; import java.util.concurrent.atomic.AtomicBoolean; import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat; +import org.opensearch.dataprepper.plugins.kafka.util.KafkaTopicMetrics; import com.amazonaws.services.schemaregistry.serializers.json.JsonDataWithSchema; import org.apache.commons.lang3.Range; @@ -55,8 +57,6 @@ 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; @@ -64,7 +64,6 @@ public class KafkaSourceCustomConsumer implements Runnable, ConsumerRebalanceLis private AtomicBoolean shutdownInProgress; private final String topicName; private final TopicConfig topicConfig; - private PluginMetrics pluginMetrics= null; private MessageFormat schema; private final BufferAccumulator> bufferAccumulator; private final Buffer> buffer; @@ -74,10 +73,11 @@ public class KafkaSourceCustomConsumer implements Runnable, ConsumerRebalanceLis private Set partitionsToReset; private final AcknowledgementSetManager acknowledgementSetManager; private final Map partitionCommitTrackerMap; - private final Counter positiveAcknowledgementSetCounter; - private final Counter negativeAcknowledgementSetCounter; + private List>> acknowledgedOffsets; private final boolean acknowledgementsEnabled; private final Duration acknowledgementsTimeout; + private final KafkaTopicMetrics topicMetrics; + private long metricsUpdatedTime; public KafkaSourceCustomConsumer(final KafkaConsumer consumer, final AtomicBoolean shutdownInProgress, @@ -86,29 +86,33 @@ public KafkaSourceCustomConsumer(final KafkaConsumer consumer, final TopicConfig topicConfig, final String schemaType, final AcknowledgementSetManager acknowledgementSetManager, - final PluginMetrics pluginMetrics) { + KafkaTopicMetrics topicMetrics) { this.topicName = topicConfig.getName(); this.topicConfig = topicConfig; this.shutdownInProgress = shutdownInProgress; this.consumer = consumer; this.buffer = buffer; + this.topicMetrics = topicMetrics; + this.topicMetrics.register(consumer); this.offsetsToCommit = new HashMap<>(); + this.metricsUpdatedTime = Instant.now().getEpochSecond(); + this.acknowledgedOffsets = new ArrayList<>(); 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; this.partitionCommitTrackerMap = new HashMap<>(); - this.partitionsToReset = new HashSet<>(); + this.partitionsToReset = Collections.synchronizedSet(new HashSet<>()); this.schema = MessageFormat.getByMessageFormatByName(schemaType); Duration bufferTimeout = Duration.ofSeconds(1); 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) { + public void updateOffsetsToCommit(final TopicPartition partition, final OffsetAndMetadata offsetAndMetadata, Range offsetRange) { + long min = offsetRange.getMinimum(); + long max = offsetRange.getMaximum(); + topicMetrics.getNumberOfRecordsCommitted().increment(max - min + 1); if (Objects.isNull(offsetAndMetadata)) { return; } @@ -121,37 +125,20 @@ private AcknowledgementSet createAcknowledgementSet(Map { if (result == true) { - positiveAcknowledgementSetCounter.increment(); - offsets.forEach((partition, offsetRange) -> { - try { - 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); - } catch (Exception e) { - LOG.error("Failed to seek to last committed offset upon positive acknowledgement {}", partition, e); - } - }); + topicMetrics.getNumberOfPositiveAcknowledgements().increment(); + synchronized(acknowledgedOffsets) { + acknowledgedOffsets.add(offsets); + } } else { - negativeAcknowledgementSetCounter.increment(); + topicMetrics.getNumberOfNegativeAcknowledgements().increment(); offsets.forEach((partition, offsetRange) -> { - synchronized(partitionsToReset) { - partitionsToReset.add(partition); - } + partitionsToReset.add(partition); }); } }, acknowledgementsTimeout); return acknowledgementSet; } - double getPositiveAcknowledgementsCount() { - return positiveAcknowledgementSetCounter.count(); - } - public void consumeRecords() throws Exception { try { ConsumerRecords records = consumer.poll(topicConfig.getThreadWaitingTime().toMillis()/2); @@ -164,23 +151,25 @@ public void consumeRecords() throws Exception { iterateRecordPartitions(records, acknowledgementSet, offsets); if (!acknowledgementsEnabled) { offsets.forEach((partition, offsetRange) -> - updateOffsetsToCommit(partition, new OffsetAndMetadata(offsetRange.getMaximum() + 1))); + updateOffsetsToCommit(partition, new OffsetAndMetadata(offsetRange.getMaximum() + 1), offsetRange)); } else { acknowledgementSet.complete(); } } } catch (AuthenticationException e) { - LOG.warn("Access Denied while doing poll(). Will retry after 10 seconds", e); + LOG.warn("Authentication error while doing poll(). Will retry after 10 seconds", e); + topicMetrics.getNumberOfPollAuthErrors().increment(); Thread.sleep(10000); } catch (RecordDeserializationException e) { - LOG.warn("Serialization error - topic {} partition {} offset {}, seeking past the error record", + LOG.warn("Deserialization error - topic {} partition {} offset {}, seeking past the error record", e.topicPartition().topic(), e.topicPartition().partition(), e.offset()); + topicMetrics.getNumberOfDeserializationErrors().increment(); consumer.seek(e.topicPartition(), e.offset()+1); } } - private void resetOrCommitOffsets() { - synchronized(partitionsToReset) { + private void resetOffsets() { + if (partitionsToReset.size() > 0) { partitionsToReset.forEach(partition -> { try { final OffsetAndMetadata offsetAndMetadata = consumer.committed(partition); @@ -191,9 +180,35 @@ private void resetOrCommitOffsets() { }); partitionsToReset.clear(); } + } + + void processAcknowledgedOffsets() { + synchronized(acknowledgedOffsets) { + acknowledgedOffsets.forEach(offsets -> { + offsets.forEach((partition, offsetRange) -> { + try { + 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, offsetRange); + } catch (Exception e) { + LOG.error("Failed committed offsets upon positive acknowledgement {}", partition, e); + } + }); + }); + acknowledgedOffsets.clear(); + } + } + + private void commitOffsets() { if (topicConfig.getAutoCommit()) { return; } + processAcknowledgedOffsets(); long currentTimeMillis = System.currentTimeMillis(); if ((currentTimeMillis - lastCommitTime) < topicConfig.getCommitInterval().toMillis()) { return; @@ -221,10 +236,12 @@ public void run() { consumer.subscribe(Arrays.asList(topicName)); while (!shutdownInProgress.get()) { try { - resetOrCommitOffsets(); + resetOffsets(); + commitOffsets(); consumeRecords(); + topicMetrics.update(consumer); } catch (Exception exp) { - LOG.error("Error while reading the records from the topic...", exp); + LOG.error("Error while reading the records from the topic {}", topicName, exp); } } } @@ -251,6 +268,7 @@ private Record getRecord(ConsumerRecord consumerRecord, in } } catch (Exception e){ LOG.error("Failed to parse JSON or AVRO record", e); + topicMetrics.getNumberOfRecordsFailedToParse().increment(); } if (!plainTextMode) { if (!(value instanceof Map)) { @@ -293,7 +311,15 @@ private void iterateRecordPartitions(ConsumerRecords records, fin if (acknowledgementSet != null) { acknowledgementSet.add(record.getData()); } - bufferAccumulator.add(record); + while (true) { + try { + bufferAccumulator.add(record); + break; + } catch (SizeOverflowException e) { + topicMetrics.getNumberOfBufferSizeOverflows().increment(); + Thread.sleep(100); + } + } } } long lastOffset = partitionRecords.get(partitionRecords.size() - 1).offset(); 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 index 9d10b46611..7799327481 100644 --- 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 @@ -21,7 +21,7 @@ public class TopicPartitionCommitTracker { public TopicPartitionCommitTracker(final TopicPartition topicPartition, Long committedOffset) { this.topicPartition = topicPartition; - this.committedOffset = Objects.nonNull(committedOffset) ? committedOffset : -1L; + this.committedOffset = Objects.nonNull(committedOffset) ? committedOffset-1 : -1L; this.offsetMaxMap = new HashMap<>(); this.offsetMinMap = new HashMap<>(); this.offsetMaxMap.put(this.committedOffset, Range.between(this.committedOffset, this.committedOffset)); 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 21af54160d..92cf2527f8 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 @@ -45,6 +45,7 @@ import org.opensearch.dataprepper.plugins.kafka.util.KafkaSourceJsonDeserializer; import org.opensearch.dataprepper.plugins.kafka.util.KafkaSourceSecurityConfigurer; import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat; +import org.opensearch.dataprepper.plugins.kafka.util.KafkaTopicMetrics; import com.amazonaws.services.schemaregistry.deserializers.GlueSchemaRegistryKafkaDeserializer; import com.amazonaws.services.schemaregistry.utils.AWSSchemaRegistryConstants; @@ -118,12 +119,13 @@ public void start(Buffer> buffer) { KafkaSourceSecurityConfigurer.setAuthProperties(authProperties, sourceConfig, LOG); sourceConfig.getTopics().forEach(topic -> { consumerGroupID = topic.getGroupId(); + KafkaTopicMetrics topicMetrics = new KafkaTopicMetrics(topic.getName(), pluginMetrics); Properties consumerProperties = getConsumerProperties(topic, authProperties); MessageFormat schema = MessageFormat.getByMessageFormatByName(schemaType); try { int numWorkers = topic.getWorkers(); executorService = Executors.newFixedThreadPool(numWorkers); - IntStream.range(0, numWorkers + 1).forEach(index -> { + IntStream.range(0, numWorkers).forEach(index -> { switch (schema) { case JSON: kafkaConsumer = new KafkaConsumer(consumerProperties); @@ -136,7 +138,7 @@ public void start(Buffer> buffer) { kafkaConsumer = new KafkaConsumer(consumerProperties); break; } - consumer = new KafkaSourceCustomConsumer(kafkaConsumer, shutdownInProgress, buffer, sourceConfig, topic, schemaType, acknowledgementSetManager, pluginMetrics); + consumer = new KafkaSourceCustomConsumer(kafkaConsumer, shutdownInProgress, buffer, sourceConfig, topic, schemaType, acknowledgementSetManager, topicMetrics); executorService.submit(consumer); }); } catch (Exception e) { diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaSourceSecurityConfigurer.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaSourceSecurityConfigurer.java index 245056e06e..4a6aaf30da 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaSourceSecurityConfigurer.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaSourceSecurityConfigurer.java @@ -212,7 +212,7 @@ public static String getBootStrapServersForMsk(final AwsIamAuthConfig awsIamAuth try { result = kafkaClient.getBootstrapBrokers(request); } catch (KafkaException | StsException e) { - LOG.debug("Failed to get bootstrap server information from MSK. Retrying...", e); + LOG.info("Failed to get bootstrap server information from MSK. Will try every 10 seconds for {} seconds", 10*MAX_KAFKA_CLIENT_RETRIES, e); try { Thread.sleep(10000); } catch (InterruptedException exp) {} diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaTopicMetrics.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaTopicMetrics.java new file mode 100644 index 0000000000..df4b22a61f --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaTopicMetrics.java @@ -0,0 +1,215 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.util; + +import io.micrometer.core.instrument.Counter; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.apache.kafka.common.Metric; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.clients.consumer.KafkaConsumer; + +import java.time.Instant; +import java.util.Objects; +import java.util.Map; +import java.util.HashMap; + +public class KafkaTopicMetrics { + static final String NUMBER_OF_POSITIVE_ACKNOWLEDGEMENTS = "numberOfPositiveAcknowledgements"; + static final String NUMBER_OF_NEGATIVE_ACKNOWLEDGEMENTS = "numberOfNegativeAcknowledgements"; + static final String NUMBER_OF_RECORDS_FAILED_TO_PARSE = "numberOfRecordsFailedToParse"; + static final String NUMBER_OF_DESERIALIZATION_ERRORS = "numberOfDeserializationErrors"; + static final String NUMBER_OF_BUFFER_SIZE_OVERFLOWS = "numberOfBufferSizeOverflows"; + static final String NUMBER_OF_POLL_AUTH_ERRORS = "numberOfPollAuthErrors"; + static final String NUMBER_OF_RECORDS_COMMITTED = "numberOfRecordsCommitted"; + static final String NUMBER_OF_RECORDS_CONSUMED = "numberOfRecordsConsumed"; + static final String NUMBER_OF_BYTES_CONSUMED = "numberOfBytesConsumed"; + + private final String topicName; + private long updateTime; + private Map metricsNameMap; + private Map> metricValues; + private final PluginMetrics pluginMetrics; + private final Counter numberOfPositiveAcknowledgements; + private final Counter numberOfNegativeAcknowledgements; + private final Counter numberOfRecordsFailedToParse; + private final Counter numberOfDeserializationErrors; + private final Counter numberOfBufferSizeOverflows; + private final Counter numberOfPollAuthErrors; + private final Counter numberOfRecordsCommitted; + private final Counter numberOfRecordsConsumed; + private final Counter numberOfBytesConsumed; + + public KafkaTopicMetrics(final String topicName, final PluginMetrics pluginMetrics) { + this.pluginMetrics = pluginMetrics; + this.topicName = topicName; + this.updateTime = Instant.now().getEpochSecond(); + this.metricValues = new HashMap<>(); + initializeMetricNamesMap(); + this.numberOfRecordsConsumed = pluginMetrics.counter(getTopicMetricName(NUMBER_OF_RECORDS_CONSUMED)); + this.numberOfBytesConsumed = pluginMetrics.counter(getTopicMetricName(NUMBER_OF_BYTES_CONSUMED)); + this.numberOfRecordsCommitted = pluginMetrics.counter(getTopicMetricName(NUMBER_OF_RECORDS_COMMITTED)); + this.numberOfRecordsFailedToParse = pluginMetrics.counter(getTopicMetricName(NUMBER_OF_RECORDS_FAILED_TO_PARSE)); + this.numberOfDeserializationErrors = pluginMetrics.counter(getTopicMetricName(NUMBER_OF_DESERIALIZATION_ERRORS)); + this.numberOfBufferSizeOverflows = pluginMetrics.counter(getTopicMetricName(NUMBER_OF_BUFFER_SIZE_OVERFLOWS)); + this.numberOfPollAuthErrors = pluginMetrics.counter(getTopicMetricName(NUMBER_OF_POLL_AUTH_ERRORS)); + this.numberOfPositiveAcknowledgements = pluginMetrics.counter(getTopicMetricName(NUMBER_OF_POSITIVE_ACKNOWLEDGEMENTS)); + this.numberOfNegativeAcknowledgements = pluginMetrics.counter(getTopicMetricName(NUMBER_OF_NEGATIVE_ACKNOWLEDGEMENTS)); + } + + private void initializeMetricNamesMap() { + this.metricsNameMap = new HashMap<>(); + metricsNameMap.put("bytes-consumed-total", "bytesConsumedTotal"); + metricsNameMap.put("records-consumed-total", "recordsConsumedTotal"); + metricsNameMap.put("bytes-consumed-rate", "bytesConsumedRate"); + metricsNameMap.put("records-consumed-rate", "recordsConsumedRate"); + metricsNameMap.put("records-lag-max", "recordsLagMax"); + metricsNameMap.put("records-lead-min", "recordsLeadMin"); + metricsNameMap.put("commit-rate", "commitRate"); + metricsNameMap.put("join-rate", "joinRate"); + metricsNameMap.put("incoming-byte-rate", "incomingByteRate"); + metricsNameMap.put("outgoing-byte-rate", "outgoingByteRate"); + metricsNameMap.put("assigned-partitions", "numberOfNonConsumers"); + metricsNameMap.forEach((metricName, camelCaseName) -> { + if (metricName.equals("records-lag-max")) { + pluginMetrics.gauge(getTopicMetricName(camelCaseName), metricValues, metricValues -> { + double max = 0.0; + for (Map.Entry> entry : metricValues.entrySet()) { + Map consumerMetrics = entry.getValue(); + synchronized(consumerMetrics) { + max = Math.max(max, consumerMetrics.get(metricName)); + } + } + return max; + }); + } else if (metricName.equals("records-lead-min")) { + pluginMetrics.gauge(getTopicMetricName(camelCaseName), metricValues, metricValues -> { + double min = Double.MAX_VALUE; + for (Map.Entry> entry : metricValues.entrySet()) { + Map consumerMetrics = entry.getValue(); + synchronized(consumerMetrics) { + min = Math.min(min, consumerMetrics.get(metricName)); + } + } + return min; + }); + } else if (!metricName.contains("-total")) { + pluginMetrics.gauge(getTopicMetricName(camelCaseName), metricValues, metricValues -> { + double sum = 0; + for (Map.Entry> entry : metricValues.entrySet()) { + Map consumerMetrics = entry.getValue(); + synchronized(consumerMetrics) { + sum += consumerMetrics.get(metricName); + } + } + return sum; + }); + } + }); + } + + public void register(final KafkaConsumer consumer) { + metricValues.put(consumer, new HashMap<>()); + final Map consumerMetrics = metricValues.get(consumer); + metricsNameMap.forEach((k, name) -> { + consumerMetrics.put(k, 0.0); + }); + } + + Counter getNumberOfRecordsConsumed() { + return numberOfRecordsConsumed; + } + + Counter getNumberOfBytesConsumed() { + return numberOfBytesConsumed; + } + + public Counter getNumberOfRecordsCommitted() { + return numberOfRecordsCommitted; + } + + public Counter getNumberOfPollAuthErrors() { + return numberOfPollAuthErrors; + } + + public Counter getNumberOfBufferSizeOverflows() { + return numberOfBufferSizeOverflows; + } + + public Counter getNumberOfDeserializationErrors() { + return numberOfDeserializationErrors; + } + + public Counter getNumberOfRecordsFailedToParse() { + return numberOfRecordsFailedToParse; + } + + public Counter getNumberOfNegativeAcknowledgements() { + return numberOfNegativeAcknowledgements; + } + + public Counter getNumberOfPositiveAcknowledgements() { + return numberOfPositiveAcknowledgements; + } + + private String getTopicMetricName(final String metricName) { + return "topic."+topicName+"."+metricName; + } + + private String getCamelCaseName(final String name) { + String camelCaseName = metricsNameMap.get(name); + if (Objects.isNull(camelCaseName)) { + return name; + } + return camelCaseName; + } + + Map> getMetricValues() { + return metricValues; + } + + public void update(final KafkaConsumer consumer) { + Map consumerMetrics = metricValues.get(consumer); + + Map metrics = consumer.metrics(); + for (Map.Entry entry : metrics.entrySet()) { + MetricName metric = entry.getKey(); + Metric value = entry.getValue(); + String metricName = metric.name(); + if (Objects.nonNull(metricsNameMap.get(metricName))) { + if (metric.tags().containsKey("partition") && + (metricName.equals("records-lag-max") || metricName.equals("records-lead-min"))) { + continue; + } + + if (metricName.contains("consumed-total") && !metric.tags().containsKey("topic")) { + continue; + } + if (metricName.contains("byte-rate") && metric.tags().containsKey("node-id")) { + continue; + } + double newValue = (Double)value.metricValue(); + if (metricName.equals("records-consumed-total")) { + synchronized(consumerMetrics) { + double prevValue = consumerMetrics.get(metricName); + numberOfRecordsConsumed.increment(newValue - prevValue); + } + } else if (metricName.equals("bytes-consumed-total")) { + synchronized(consumerMetrics) { + double prevValue = consumerMetrics.get(metricName); + numberOfBytesConsumed.increment(newValue - prevValue); + } + } + // Keep the count of number of consumers without any assigned partitions. This value can go up or down. So, it is made as Guage metric + if (metricName.equals("assigned-partitions")) { + newValue = (newValue == 0.0) ? 1.0 : 0.0; + } + synchronized(consumerMetrics) { + consumerMetrics.put(metricName, newValue); + } + } + } + } +} 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 da84536a82..47080515d3 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 @@ -12,7 +12,6 @@ import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; -import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.CheckpointState; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.configuration.PluginSetting; @@ -23,6 +22,7 @@ import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaKeyMode; import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat; +import org.opensearch.dataprepper.plugins.kafka.util.KafkaTopicMetrics; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.acknowledgements.DefaultAcknowledgementSetManager; import io.micrometer.core.instrument.Counter; @@ -32,7 +32,6 @@ 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; @@ -77,7 +76,7 @@ public class KafkaSourceCustomConsumerTest { private TopicConfig topicConfig; @Mock - private PluginMetrics pluginMetrics; + private KafkaTopicMetrics topicMetrics; private KafkaSourceCustomConsumer consumer; @@ -100,15 +99,18 @@ public class KafkaSourceCustomConsumerTest { @BeforeEach public void setUp() { kafkaConsumer = mock(KafkaConsumer.class); - pluginMetrics = mock(PluginMetrics.class); + topicMetrics = mock(KafkaTopicMetrics.class); counter = mock(Counter.class); topicConfig = mock(TopicConfig.class); + when(topicMetrics.getNumberOfPositiveAcknowledgements()).thenReturn(counter); + when(topicMetrics.getNumberOfNegativeAcknowledgements()).thenReturn(counter); + when(topicMetrics.getNumberOfNegativeAcknowledgements()).thenReturn(counter); + when(topicMetrics.getNumberOfRecordsCommitted()).thenReturn(counter); when(topicConfig.getThreadWaitingTime()).thenReturn(Duration.ofSeconds(1)); when(topicConfig.getSerdeFormat()).thenReturn(MessageFormat.PLAINTEXT); 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)); @@ -122,7 +124,7 @@ public void setUp() { 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); + return new KafkaSourceCustomConsumer(kafkaConsumer, shutdownInProgress, buffer, sourceConfig, topicConfig, schemaType, acknowledgementSetManager, topicMetrics); } private BlockingBuffer> getBuffer() { @@ -203,6 +205,7 @@ public void testPlainTextConsumeRecordsWithAcknowledgements() throws Interrupted Thread.sleep(10000); } catch (Exception e){} + consumer.processAcknowledgedOffsets(); offsetsToCommit = consumer.getOffsetsToCommit(); Assertions.assertEquals(offsetsToCommit.size(), 1); offsetsToCommit.forEach((topicPartition, offsetAndMetadata) -> { @@ -246,6 +249,7 @@ public void testPlainTextConsumeRecordsWithNegativeAcknowledgements() throws Int Thread.sleep(10000); } catch (Exception e){} + consumer.processAcknowledgedOffsets(); offsetsToCommit = consumer.getOffsetsToCommit(); Assertions.assertEquals(offsetsToCommit.size(), 0); } 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 index 387ffb909e..1f2e2ae243 100644 --- 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 @@ -37,7 +37,7 @@ public TopicPartitionCommitTracker createObjectUnderTest(String topic, int parti @ParameterizedTest @MethodSource("getInputOrder") public void test(List order) { - topicPartitionCommitTracker = createObjectUnderTest(testTopic, testPartition, -1L); + topicPartitionCommitTracker = createObjectUnderTest(testTopic, testPartition, 0L); List> ranges = new ArrayList<>(); for (int i = 0; i < 10; i++) { ranges.add(Range.between(i*10L, i*10L+9L)); diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaTopicMetricsTests.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaTopicMetricsTests.java new file mode 100644 index 0000000000..ea31c216db --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaTopicMetricsTests.java @@ -0,0 +1,250 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.util; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import org.junit.jupiter.api.extension.ExtendWith; +import static org.junit.jupiter.api.Assertions.assertEquals; + +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.doAnswer; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.apache.kafka.common.Metric; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.Metric; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.commons.lang3.RandomStringUtils; + +import io.micrometer.core.instrument.Counter; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.CoreMatchers.equalTo; + +import java.util.Map; +import java.util.HashMap; +import java.util.Random; +import java.util.function.ToDoubleFunction; + +@ExtendWith(MockitoExtension.class) +public class KafkaTopicMetricsTests { + public final class KafkaTestMetric implements Metric { + private final Double value; + private final MetricName name; + + public KafkaTestMetric(final double value, final MetricName name) { + this.value = value; + this.name = name; + } + + @Override + public MetricName metricName() { + return name; + } + + @Override + public Object metricValue() { + return value; + } + } + + private String topicName; + + @Mock + private PluginMetrics pluginMetrics; + + private Map pluginMetricsMap; + + private Random random; + + private KafkaTopicMetrics topicMetrics; + + private double bytesConsumed; + private double recordsConsumed; + private double bytesConsumedRate; + private double recordsConsumedRate; + private double recordsLagMax; + private double recordsLeadMin; + private double commitRate; + private double joinRate; + private double incomingByteRate; + private double outgoingByteRate; + + @Mock + private Counter bytesConsumedCounter; + + @Mock + private Counter recordsConsumedCounter; + private double bytesConsumedCount; + private double recordsConsumedCount; + + @BeforeEach + void setUp() { + topicName = RandomStringUtils.randomAlphabetic(8); + bytesConsumed = 0.0; + recordsConsumed = 0.0; + bytesConsumedRate = 0.0; + recordsConsumedRate = 0.0; + recordsLagMax = 0.0; + recordsLeadMin = Double.MAX_VALUE; + commitRate = 0.0; + joinRate = 0.0; + incomingByteRate = 0.0; + outgoingByteRate = 0.0; + + bytesConsumedCount = 0.0; + recordsConsumedCount = 0.0; + + random = new Random(); + pluginMetrics = mock(PluginMetrics.class); + pluginMetricsMap = new HashMap<>(); + doAnswer((i) -> { + ToDoubleFunction f = (ToDoubleFunction)i.getArgument(2); + Object arg = (Object)i.getArgument(1); + String name = (String)i.getArgument(0); + pluginMetricsMap.put(name, f); + return f.applyAsDouble(arg); + }).when(pluginMetrics).gauge(any(String.class), any(Object.class), any()); + bytesConsumedCounter = mock(Counter.class); + recordsConsumedCounter = mock(Counter.class); + + doAnswer((i) -> { + String arg = (String)i.getArgument(0); + if (arg.contains("Bytes")) { + return bytesConsumedCounter; + } else { + return recordsConsumedCounter; + } + }).when(pluginMetrics).counter(any(String.class)); + doAnswer((i) -> { + bytesConsumedCount += (double)i.getArgument(0); + return null; + }).when(bytesConsumedCounter).increment(any(Double.class)); + doAnswer((i) -> { + recordsConsumedCount += (double)i.getArgument(0); + return null; + }).when(recordsConsumedCounter).increment(any(Double.class)); + } + + public KafkaTopicMetrics createObjectUnderTest() { + return new KafkaTopicMetrics(topicName, pluginMetrics); + } + + private KafkaTestMetric getMetric(final String name, final double value, Map tags) { + MetricName metricName = new MetricName(name, "group", "metric", tags); + return new KafkaTestMetric(value, metricName); + } + + + private void populateKafkaMetrics(Map metrics, double numAssignedPartitions) { + Integer tmpBytesConsumed = random.nextInt() % 100 + 1; + if (tmpBytesConsumed < 0) { + tmpBytesConsumed = -tmpBytesConsumed; + } + bytesConsumed += tmpBytesConsumed; + Integer tmpRecordsConsumed = random.nextInt() % 10 + 1; + if (tmpRecordsConsumed < 0) { + tmpRecordsConsumed = -tmpRecordsConsumed; + } + recordsConsumed += tmpRecordsConsumed; + + double tmpBytesConsumedRate = random.nextDouble()*100; + bytesConsumedRate += tmpBytesConsumedRate; + + double tmpRecordsConsumedRate = random.nextDouble()*10; + recordsConsumedRate += tmpRecordsConsumedRate; + + double tmpRecordsLagMax = random.nextDouble()*2; + recordsLagMax = Math.max(recordsLagMax, tmpRecordsLagMax); + + double tmpRecordsLeadMin = random.nextDouble()*3; + recordsLeadMin = Math.min(recordsLeadMin, tmpRecordsLeadMin); + + double tmpCommitRate = random.nextDouble(); + commitRate += tmpCommitRate; + + double tmpJoinRate = random.nextDouble(); + joinRate += tmpJoinRate; + + double tmpIncomingByteRate = random.nextDouble(); + incomingByteRate += tmpIncomingByteRate; + + double tmpOutgoingByteRate = random.nextDouble(); + outgoingByteRate += tmpOutgoingByteRate; + + Map metricsMap = new HashMap<>(); + metricsMap.put("bytes-consumed-total", (double)tmpBytesConsumed); + metricsMap.put("records-consumed-total", (double)tmpRecordsConsumed); + metricsMap.put("bytes-consumed-rate", tmpBytesConsumedRate); + metricsMap.put("records-consumed-rate", tmpRecordsConsumedRate); + metricsMap.put("records-lag-max", tmpRecordsLagMax); + metricsMap.put("records-lead-min", tmpRecordsLeadMin); + metricsMap.put("commit-rate", tmpCommitRate); + metricsMap.put("join-rate", tmpJoinRate); + metricsMap.put("incoming-byte-rate", tmpIncomingByteRate); + metricsMap.put("outgoing-byte-rate", tmpOutgoingByteRate); + metricsMap.put("assigned-partitions", numAssignedPartitions); + + metricsMap.forEach((name, value) -> { + Map tags = new HashMap<>(); + if (name.contains("-total")) { + tags.put("topic", topicName); + } + KafkaTestMetric metric = getMetric(name, value, tags); + metrics.put(metric.metricName(), metric); + }); + } + + @ParameterizedTest + @ValueSource(ints = {1, 5, 10}) + //@ValueSource(ints = {2}) + public void KafkaTopicMetricTest_checkMetricUpdates(int numConsumers) { + topicMetrics = createObjectUnderTest(); + for (int i = 0; i < numConsumers; i++) { + KafkaConsumer kafkaConsumer = mock(KafkaConsumer.class); + topicMetrics.register(kafkaConsumer); + Map metrics = new HashMap<>(); + when(kafkaConsumer.metrics()).thenReturn(metrics); + populateKafkaMetrics(metrics, (i %2 == 1) ? 0.0 : 1.0); + topicMetrics.update(kafkaConsumer); + } + when(recordsConsumedCounter.count()).thenReturn(recordsConsumedCount); + when(bytesConsumedCounter.count()).thenReturn(bytesConsumedCount); + assertThat(topicMetrics.getNumberOfRecordsConsumed().count(), equalTo(recordsConsumed)); + assertThat(topicMetrics.getNumberOfBytesConsumed().count(), equalTo(bytesConsumed)); + pluginMetricsMap.forEach((k, v) -> { + double result = v.applyAsDouble(topicMetrics.getMetricValues()); + if (k.contains("bytesConsumedRate")) { + assertEquals(result, bytesConsumedRate, 0.01d); + } else if (k.contains("recordsConsumedRate")) { + assertEquals(result, recordsConsumedRate, 0.01d); + } else if (k.contains("recordsLagMax")) { + assertEquals(result, recordsLagMax, 0.01d); + } else if (k.contains("recordsLeadMin")) { + assertEquals(result, recordsLeadMin, 0.01d); + } else if (k.contains("commitRate")) { + assertEquals(result, commitRate, 0.01d); + } else if (k.contains("joinRate")) { + assertEquals(result, joinRate, 0.01d); + } else if (k.contains("incomingByteRate")) { + assertEquals(result, incomingByteRate, 0.01d); + } else if (k.contains("outgoingByteRate")) { + assertEquals(result, outgoingByteRate, 0.01d); + } else if (k.contains("numberOfNonConsumers")) { + int expectedValue = numConsumers/2; + assertThat(result, equalTo((double)expectedValue)); + } else { + assertThat(result, equalTo(k+": Unknown Metric")); + } + }); + + } + +} From 7295b72da7288efdf8aa600a0bf6762d8a7ed6d7 Mon Sep 17 00:00:00 2001 From: David Venable Date: Thu, 10 Aug 2023 15:13:27 -0500 Subject: [PATCH 05/28] Correct the behavior of the JSON output codec to write a JSON object first. Adds a configurable keyName for the array. (#3132) Signed-off-by: David Venable --- .../plugins/codec/json/JsonOutputCodec.java | 4 +++- .../plugins/codec/json/JsonOutputCodecConfig.java | 13 ++++++++++++- .../plugins/codec/json/JsonCodecsIT.java | 9 +++++++++ .../plugins/codec/json/JsonOutputCodecTest.java | 9 +++++++++ 4 files changed, 33 insertions(+), 2 deletions(-) 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 b2c68224ce..30cc3a053b 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 @@ -42,12 +42,15 @@ public void start(final OutputStream outputStream, Event event, final OutputCode Objects.requireNonNull(codecContext); this.codecContext = codecContext; generator = factory.createGenerator(outputStream, JsonEncoding.UTF8); + generator.writeStartObject(); + generator.writeFieldName(config.getKeyName()); generator.writeStartArray(); } @Override public void complete(final OutputStream outputStream) throws IOException { generator.writeEndArray(); + generator.writeEndObject(); generator.close(); outputStream.flush(); outputStream.close(); @@ -78,7 +81,6 @@ public synchronized void writeEvent(final Event event, final OutputStream output public String getExtension() { return JSON; } - } diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/codec/json/JsonOutputCodecConfig.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/codec/json/JsonOutputCodecConfig.java index 642717ab71..ea99a23f4a 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/codec/json/JsonOutputCodecConfig.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/codec/json/JsonOutputCodecConfig.java @@ -4,6 +4,17 @@ */ package org.opensearch.dataprepper.plugins.codec.json; +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.Size; + public class JsonOutputCodecConfig { - + static final String DEFAULT_KEY_NAME = "events"; + + @JsonProperty("key_name") + @Size(min = 1, max = 2048) + private String keyName = DEFAULT_KEY_NAME; + + public String getKeyName() { + return keyName; + } } diff --git a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/codec/json/JsonCodecsIT.java b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/codec/json/JsonCodecsIT.java index 1ea165938c..cbdedb3f1f 100644 --- a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/codec/json/JsonCodecsIT.java +++ b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/codec/json/JsonCodecsIT.java @@ -7,6 +7,7 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.JsonNodeType; import org.hamcrest.Matchers; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.params.ParameterizedTest; @@ -30,6 +31,7 @@ import java.util.function.Consumer; import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.MatcherAssert.assertThat; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; @@ -83,6 +85,13 @@ void parse_with_InputStream_calls_Consumer_with_Event(final int numberOfObjects) int index = 0; ObjectMapper mapper = new ObjectMapper(); JsonNode jsonNode = mapper.readTree(outputStream.toByteArray()); + assertThat(jsonNode.getNodeType(), equalTo(JsonNodeType.OBJECT)); + Map.Entry nextField = jsonNode.fields().next(); + assertThat(nextField, notNullValue()); + assertThat(nextField.getKey(), equalTo(JsonOutputCodecConfig.DEFAULT_KEY_NAME)); + jsonNode = nextField.getValue(); + assertThat(jsonNode, notNullValue()); + assertThat(jsonNode.getNodeType(), equalTo(JsonNodeType.ARRAY)); for (JsonNode element : jsonNode) { Set keys = initialRecords.get(index).keySet(); Map actualMap = new HashMap<>(); diff --git a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/codec/json/JsonOutputCodecTest.java b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/codec/json/JsonOutputCodecTest.java index 3409b3a593..073a768cfc 100644 --- a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/codec/json/JsonOutputCodecTest.java +++ b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/codec/json/JsonOutputCodecTest.java @@ -7,6 +7,7 @@ import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.JsonNodeType; import org.hamcrest.Matchers; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -25,6 +26,7 @@ import java.util.Set; import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.MatcherAssert.assertThat; class JsonOutputCodecTest { @@ -75,6 +77,13 @@ void test_happy_case(final int numberOfRecords) throws IOException { int index = 0; ObjectMapper mapper = new ObjectMapper(); JsonNode jsonNode = mapper.readTree(outputStream.toByteArray()); + assertThat(jsonNode.getNodeType(), equalTo(JsonNodeType.OBJECT)); + Map.Entry nextField = jsonNode.fields().next(); + assertThat(nextField, notNullValue()); + assertThat(nextField.getKey(), equalTo(JsonOutputCodecConfig.DEFAULT_KEY_NAME)); + jsonNode = nextField.getValue(); + assertThat(jsonNode, notNullValue()); + assertThat(jsonNode.getNodeType(), equalTo(JsonNodeType.ARRAY)); for (JsonNode element : jsonNode) { Set keys = expectedRecords.get(index).keySet(); Map actualMap = new HashMap<>(); From 000c39de63e03252e3472d9507570a67d8bcd0f2 Mon Sep 17 00:00:00 2001 From: Asif Sohail Mohammed Date: Thu, 10 Aug 2023 15:30:28 -0500 Subject: [PATCH 06/28] S3 single scan improvements (#3124) * S3 single scan improvements Signed-off-by: Asif Sohail Mohammed --------- Signed-off-by: Asif Sohail Mohammed --- data-prepper-plugins/s3-source/README.md | 35 +++++----- .../plugins/source/S3ScanObjectWorkerIT.java | 2 +- .../CustomLocalDateTimeDeserializer.java | 45 +++++++++++++ .../S3ScanPartitionCreationSupplier.java | 6 +- .../configuration/S3ScanBucketOption.java | 20 ++---- .../configuration/S3ScanKeyPathOption.java | 8 +-- .../configuration/S3ScanScanOptions.java | 13 +--- .../S3ScanSchedulingOptions.java | 17 +++-- .../CustomLocalDateTimeDeserializerTest.java | 64 +++++++++++++++++++ .../S3ScanPartitionCreationSupplierTest.java | 8 +-- .../plugins/source/S3ScanServiceTest.java | 18 +++--- .../configuration/S3ScanBucketOptionTest.java | 14 ++-- .../S3ScanBucketOptionsTest.java | 12 ++-- .../configuration/S3ScanScanOptionsTest.java | 12 ++-- .../S3ScanSchedulingOptionsTest.java | 4 +- 15 files changed, 189 insertions(+), 89 deletions(-) create mode 100644 data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/CustomLocalDateTimeDeserializer.java create mode 100644 data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/CustomLocalDateTimeDeserializerTest.java diff --git a/data-prepper-plugins/s3-source/README.md b/data-prepper-plugins/s3-source/README.md index 63ea62bd69..08f5437841 100644 --- a/data-prepper-plugins/s3-source/README.md +++ b/data-prepper-plugins/s3-source/README.md @@ -72,13 +72,13 @@ source-pipeline: region: "us-east-1" sts_role_arn: "arn:aws:iam::123456789012:role/Data-Prepper" scan: - start_time: 2023-01-21T18:00:00 - end_time: 2023-04-21T18:00:00 + start_time: now + end_time: 2023-12-31T11:59:59 buckets: - bucket: name: my-bucket-1 - key_prefix: - include: + filter: + include_prefix: - bucket2/ exclude_suffix: - .jpeg @@ -117,7 +117,7 @@ All Duration values are a string that represents a duration. They support ISO_86 * `disable_bucket_ownership_validation` (Optional) : Boolean - If set to true, then the S3 Source will not attempt to validate that the bucket is owned by the expected account. The only expected account is the same account which owns the SQS queue. Defaults to `false`. -* `delete_on_read` (Optional) : Boolean - If set to true, then the S3 Source will attempt to delete S3 objects after all the events from the S3 object are successfully acknowledged by all sinks. `acknowledgments` should be enabled for deleting S3 objects. Defaults to `false`. +* `delete_s3_objects_on_read` (Optional) : Boolean - If set to true, then the S3 Scan will attempt to delete S3 objects after all the events from the S3 object are successfully acknowledged by all sinks. `acknowledgments` should be enabled for deleting S3 objects. Defaults to `false`. ### S3 Select Configuration @@ -150,15 +150,15 @@ All Duration values are a string that represents a duration. They support ISO_86 * `poll_delay` (Optional) : Duration - A delay to place between reading and processing a batch of SQS messages and making a subsequent request. Defaults to 0 seconds. ### S3 Scan Configuration -* `start_time` (Optional) : Provide the start time to scan objects from all the buckets. This parameter defines a time range together with either end_time or range. Example: `2023-01-23T10:00:00`. -* `end_time` (Optional) : Provide the end time to scan objects from all the buckets. This parameter defines a time range together with either start_time or range. Example: `2023-01-23T10:00:00`. -* `range` (Optional) : Provide the duration to scan objects from all the buckets. This parameter defines a time range together with either start_time or end_time. +* `start_time` (Optional) : Provide the start time to scan objects from all the buckets. This should follow [ISO LocalDateTime](https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html#ISO_LOCAL_DATE_TIME) format, or it can be configured to `now` keyword which represents current LocalDateTime. This parameter defines a time range together with either end_time or range. Examples: `2023-01-23T10:00:00`, `now`. +* `end_time` (Optional) : Provide the end time to scan objects from all the buckets. This should follow [ISO LocalDateTime](https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html#ISO_LOCAL_DATE_TIME) format, or it can be configured to `now` keyword which represents current LocalDateTime. This parameter defines a time range together with either start_time or range. Examples: `2023-01-23T10:00:00`, `now`. +* `range` (Optional) : Provide the duration to scan objects from all the buckets. This parameter defines a time range together with either `start_time` or `end_time`. * `scheduling` (Optional): See [Scheduling Configuration](#scheduling_configuration) for details * `bucket`: Provide S3 bucket information * `name` (Required if bucket block is used): Provide S3 bucket name. - * `key_prefix` (Optional) : Provide include and exclude the list items. - * `include` (Optional) : Provide the list of include key path prefix. - * `exclude_suffix` (Optional) : Provide the list of suffix to exclude items. + * `filter` (Optional) : Provide include and exclude list items to filter objects in bucket. + * `include_prefix` (Optional) : Provide the list of include key path prefix. For example `dlq/` + * `exclude_suffix` (Optional) : Provide the list of suffix to exclude items. For example `.csv`. * `start_time` (Optional) : Provide the start time to scan objects from the current bucket. This parameter defines a time range together with either end_time or range. Example: `2023-01-23T10:00:00`. * `end_time` (Optional) : Provide the end time to scan objects from the current bucket. This parameter defines a time range together with either start_time or range. Example: `2023-01-23T10:00:00`. * `range` (Optional) : Provide the duration to scan objects from the current bucket. This parameter defines a time range together with either start_time or end_time. @@ -167,14 +167,13 @@ All Duration values are a string that represents a duration. They support ISO_86 ### Scheduling Configuration -Schedule frequency and amount of times an object should be processed when using S3 Scan. For example, -a `rate` of `PT1H` and a `job_count` of 3 would result in each object getting processed 3 times, starting after source is ready -and then every hour after the first time the object is processed. - -* `rate` (Optional) : A String that indicates the rate to process an S3 object based on the `job_count`. +Schedule interval and amount of times a S3 bucket should be scanned when using S3 Scan. For example, +a `interval` of `PT1H` and a `count` of `3` would result in each bucket being scanned 3 times with 1 hour interval in between each scan, starting after source is ready +and then every hour after the first scan. +* `interval` (Optional) : A String that indicates the minimum interval between each scan. If objects from fist scan are not proceed within configured interval, scan will be done whenever there are no pending objects to process from previous scan. Supports ISO_8601 notation Strings ("PT20.345S", "PT15M", etc.) as well as simple notation Strings for seconds ("60s") and milliseconds ("1500ms"). - Defaults to 8 hours, and is only applicable when `job_count` is greater than 1. -* `job_count` (Optional) : An Integer that specifies how many times each S3 object should be processed. Defaults to 1. + Defaults to 8 hours, and is only applicable when `count` is greater than 1. +* `count` (Optional) : An Integer that specifies how many times bucket will be scanned. Defaults to 1. ### AWS Configuration diff --git a/data-prepper-plugins/s3-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/S3ScanObjectWorkerIT.java b/data-prepper-plugins/s3-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/S3ScanObjectWorkerIT.java index c75dfa14c5..9703943111 100644 --- a/data-prepper-plugins/s3-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/S3ScanObjectWorkerIT.java +++ b/data-prepper-plugins/s3-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/S3ScanObjectWorkerIT.java @@ -206,7 +206,7 @@ private ScanObjectWorker createObjectUnderTest(final RecordsGenerator recordsGen when(s3SourceConfig.getS3ScanScanOptions()).thenReturn(s3ScanScanOptions); when(s3ScanScanOptions.getSchedulingOptions()).thenReturn(s3ScanSchedulingOptions); - lenient().when(s3ScanSchedulingOptions.getRate()).thenReturn(Duration.ofHours(1)); + lenient().when(s3ScanSchedulingOptions.getInterval()).thenReturn(Duration.ofHours(1)); lenient().when(s3ScanSchedulingOptions.getCount()).thenReturn(1); ExecutorService executor = Executors.newFixedThreadPool(2); diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/CustomLocalDateTimeDeserializer.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/CustomLocalDateTimeDeserializer.java new file mode 100644 index 0000000000..c118e95cbc --- /dev/null +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/CustomLocalDateTimeDeserializer.java @@ -0,0 +1,45 @@ +/* + * Copyright OpenSearch Contributors + * * SPDX-License-Identifier: Apache-2.0 + * + */ +package org.opensearch.dataprepper.plugins.source; + +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.deser.std.StdDeserializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.time.LocalDateTime; +import java.time.format.DateTimeParseException; + +public class CustomLocalDateTimeDeserializer extends StdDeserializer { + private static final Logger LOG = LoggerFactory.getLogger(CustomLocalDateTimeDeserializer.class); + static final String CURRENT_LOCAL_DATE_TIME_STRING = "now"; + + public CustomLocalDateTimeDeserializer() { + this(null); + } + + public CustomLocalDateTimeDeserializer(Class vc) { + super(vc); + } + + @Override + public LocalDateTime deserialize(JsonParser parser, DeserializationContext context) throws IOException { + final String valueAsString = parser.getValueAsString(); + + if (valueAsString.equals(CURRENT_LOCAL_DATE_TIME_STRING)) { + return LocalDateTime.now(); + } else { + try { + return LocalDateTime.parse(valueAsString); + } catch (final DateTimeParseException e) { + LOG.error("Unable to parse {} to LocalDateTime.", valueAsString, e); + throw new IllegalArgumentException("Unable to obtain instance of LocalDateTime from " + valueAsString, e); + } + } + } +} diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplier.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplier.java index 9df1b5ab9d..cadcaf71e8 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplier.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplier.java @@ -47,7 +47,7 @@ public List apply(final Map globalStateMap) for (final ScanOptions scanOptions : scanOptionsList) { final List excludeItems = new ArrayList<>(); - final S3ScanKeyPathOption s3ScanKeyPathOption = scanOptions.getBucketOption().getkeyPrefix(); + final S3ScanKeyPathOption s3ScanKeyPathOption = scanOptions.getBucketOption().getS3ScanFilter(); final ListObjectsV2Request.Builder listObjectsV2Request = ListObjectsV2Request.builder() .bucket(scanOptions.getBucketOption().getName()); bucketOwnerProvider.getBucketOwner(scanOptions.getBucketOption().getName()) @@ -56,8 +56,8 @@ public List apply(final Map globalStateMap) if (Objects.nonNull(s3ScanKeyPathOption) && Objects.nonNull(s3ScanKeyPathOption.getS3ScanExcludeSuffixOptions())) excludeItems.addAll(s3ScanKeyPathOption.getS3ScanExcludeSuffixOptions()); - if (Objects.nonNull(s3ScanKeyPathOption) && Objects.nonNull(s3ScanKeyPathOption.getS3scanIncludeOptions())) - s3ScanKeyPathOption.getS3scanIncludeOptions().forEach(includePath -> { + if (Objects.nonNull(s3ScanKeyPathOption) && Objects.nonNull(s3ScanKeyPathOption.getS3scanIncludePrefixOptions())) + s3ScanKeyPathOption.getS3scanIncludePrefixOptions().forEach(includePath -> { listObjectsV2Request.prefix(includePath); objectsToProcess.addAll(listFilteredS3ObjectsForBucket(excludeItems, listObjectsV2Request, scanOptions.getBucketOption().getName(), scanOptions.getUseStartDateTime(), scanOptions.getUseEndDateTime())); diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOption.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOption.java index 2f16c86706..c840b82907 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOption.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOption.java @@ -6,12 +6,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.annotation.JsonDeserialize; -import com.fasterxml.jackson.databind.annotation.JsonSerialize; import com.fasterxml.jackson.datatype.jsr310.deser.DurationDeserializer; -import com.fasterxml.jackson.datatype.jsr310.deser.LocalDateTimeDeserializer; -import com.fasterxml.jackson.datatype.jsr310.ser.DurationSerializer; -import com.fasterxml.jackson.datatype.jsr310.ser.LocalDateTimeSerializer; import jakarta.validation.constraints.AssertTrue; +import org.opensearch.dataprepper.plugins.source.CustomLocalDateTimeDeserializer; import java.time.Duration; import java.time.LocalDateTime; @@ -25,23 +22,20 @@ public class S3ScanBucketOption { @JsonProperty("name") private String name; - @JsonSerialize(using = LocalDateTimeSerializer.class) - @JsonDeserialize(using = LocalDateTimeDeserializer.class) + @JsonDeserialize(using = CustomLocalDateTimeDeserializer.class) @JsonProperty("start_time") private LocalDateTime startTime; - @JsonSerialize(using = LocalDateTimeSerializer.class) - @JsonDeserialize(using = LocalDateTimeDeserializer.class) + @JsonDeserialize(using = CustomLocalDateTimeDeserializer.class) @JsonProperty("end_time") private LocalDateTime endTime; - @JsonSerialize(using = DurationSerializer.class) @JsonDeserialize(using = DurationDeserializer.class) @JsonProperty("range") private Duration range; - @JsonProperty("key_prefix") - private S3ScanKeyPathOption keyPrefix; + @JsonProperty("filter") + private S3ScanKeyPathOption s3ScanFilter; @AssertTrue(message = "At most two options from start_time, end_time and range can be specified at the same time") public boolean hasValidTimeOptions() { @@ -64,7 +58,7 @@ public Duration getRange() { return range; } - public S3ScanKeyPathOption getkeyPrefix() { - return keyPrefix; + public S3ScanKeyPathOption getS3ScanFilter() { + return s3ScanFilter; } } \ No newline at end of file diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanKeyPathOption.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanKeyPathOption.java index d038888e07..451eb26718 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanKeyPathOption.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanKeyPathOption.java @@ -12,13 +12,13 @@ * Class consists the scan include and exclude keys properties. */ public class S3ScanKeyPathOption { - @JsonProperty("include") - private List s3scanIncludeOptions; + @JsonProperty("include_prefix") + private List s3scanIncludePrefixOptions; @JsonProperty("exclude_suffix") private List s3ScanExcludeSuffixOptions; - public List getS3scanIncludeOptions() { - return s3scanIncludeOptions; + public List getS3scanIncludePrefixOptions() { + return s3scanIncludePrefixOptions; } public List getS3ScanExcludeSuffixOptions() { diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanScanOptions.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanScanOptions.java index 824e2d61bb..6948597119 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanScanOptions.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanScanOptions.java @@ -6,12 +6,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.annotation.JsonDeserialize; -import com.fasterxml.jackson.databind.annotation.JsonSerialize; import com.fasterxml.jackson.datatype.jsr310.deser.DurationDeserializer; -import com.fasterxml.jackson.datatype.jsr310.deser.LocalDateTimeDeserializer; -import com.fasterxml.jackson.datatype.jsr310.ser.DurationSerializer; -import com.fasterxml.jackson.datatype.jsr310.ser.LocalDateTimeSerializer; import jakarta.validation.constraints.AssertTrue; +import org.opensearch.dataprepper.plugins.source.CustomLocalDateTimeDeserializer; import java.time.Duration; import java.time.LocalDateTime; @@ -23,19 +20,15 @@ * Class consists the scan options list bucket configuration properties. */ public class S3ScanScanOptions { - - @JsonSerialize(using = DurationSerializer.class) @JsonDeserialize(using = DurationDeserializer.class) @JsonProperty("range") private Duration range; - @JsonSerialize(using = LocalDateTimeSerializer.class) - @JsonDeserialize(using = LocalDateTimeDeserializer.class) + @JsonDeserialize(using = CustomLocalDateTimeDeserializer.class) @JsonProperty("start_time") private LocalDateTime startTime; - @JsonSerialize(using = LocalDateTimeSerializer.class) - @JsonDeserialize(using = LocalDateTimeDeserializer.class) + @JsonDeserialize(using = CustomLocalDateTimeDeserializer.class) @JsonProperty("end_time") private LocalDateTime endTime; diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanSchedulingOptions.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanSchedulingOptions.java index 9dbe4b8840..c474675818 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanSchedulingOptions.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanSchedulingOptions.java @@ -1,3 +1,8 @@ +/* + * * Copyright OpenSearch Contributors + * * SPDX-License-Identifier: Apache-2.0 + * + */ package org.opensearch.dataprepper.plugins.source.configuration; import com.fasterxml.jackson.annotation.JsonProperty; @@ -6,19 +11,19 @@ import java.time.Duration; public class S3ScanSchedulingOptions { - @JsonProperty("rate") - private Duration rate = Duration.ofHours(8); + @JsonProperty("interval") + private Duration interval = Duration.ofHours(8); @Min(1) @JsonProperty("count") - private int Count = 1; + private int count = 1; - public Duration getRate() { - return rate; + public Duration getInterval() { + return interval; } public int getCount() { - return Count; + return count; } } diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/CustomLocalDateTimeDeserializerTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/CustomLocalDateTimeDeserializerTest.java new file mode 100644 index 0000000000..e814da31f1 --- /dev/null +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/CustomLocalDateTimeDeserializerTest.java @@ -0,0 +1,64 @@ +/* + * Copyright OpenSearch Contributors + * * SPDX-License-Identifier: Apache-2.0 + * + */ + +package org.opensearch.dataprepper.plugins.source; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.module.SimpleModule; +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 java.time.Duration; +import java.time.LocalDateTime; +import java.time.temporal.ChronoUnit; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.lessThan; +import static org.junit.Assert.assertThrows; +import static org.opensearch.dataprepper.plugins.source.CustomLocalDateTimeDeserializer.CURRENT_LOCAL_DATE_TIME_STRING; + +class CustomLocalDateTimeDeserializerTest { + private ObjectMapper objectMapper; + + @BeforeEach + void setUp() { + objectMapper = new ObjectMapper(); + + final SimpleModule simpleModule = new SimpleModule(); + simpleModule.addDeserializer(LocalDateTime.class, new CustomLocalDateTimeDeserializer()); + objectMapper.registerModule(simpleModule); + } + + @ParameterizedTest + @ValueSource(strings = {"2023-01-2118:00:00", "2023-01-21T8:00:00"}) + void deserialize_with_invalid_values_throws(final String invalidDateTimeString) { + assertThrows(IllegalArgumentException.class, () -> objectMapper.convertValue(invalidDateTimeString, LocalDateTime.class)); + } + + @Test + void deserialize_with_predefined_custom_value_returns_current_local_datetime() { + final LocalDateTime expectedDateTime = objectMapper.convertValue(CURRENT_LOCAL_DATE_TIME_STRING, LocalDateTime.class); + assertThat(expectedDateTime, lessThan(LocalDateTime.now())); + assertThat(expectedDateTime, greaterThan(LocalDateTime.now().minus(Duration.of(5, ChronoUnit.SECONDS)))); + } + + @Test + void deserialize_with_iso_local_date_time_string_returns_correct_local_datetime() { + final String testLocalDateTimeString = "2023-01-21T18:30:45"; + final LocalDateTime expectedDateTime = objectMapper.convertValue(testLocalDateTimeString, LocalDateTime.class); + assertThat(expectedDateTime, equalTo(LocalDateTime.of(2023, 1, 21, 18, 30, 45))); + assertThat(expectedDateTime.getYear(), equalTo(2023)); + assertThat(expectedDateTime.getMonthValue(), equalTo(1)); + assertThat(expectedDateTime.getDayOfMonth(), equalTo(21)); + assertThat(expectedDateTime.getHour(), equalTo(18)); + assertThat(expectedDateTime.getMinute(), equalTo(30)); + assertThat(expectedDateTime.getSecond(), equalTo(45)); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplierTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplierTest.java index 02a38074ae..d3c557aa06 100644 --- a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplierTest.java +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplierTest.java @@ -78,8 +78,8 @@ void getNextPartition_supplier_returns_expected_PartitionIdentifiers() { given(firstBucketScanOptions.getUseStartDateTime()).willReturn(LocalDateTime.ofInstant(startTime, ZoneId.systemDefault())); given(firstBucketScanOptions.getUseEndDateTime()).willReturn(LocalDateTime.ofInstant(endTime, ZoneId.systemDefault())); final S3ScanKeyPathOption firstBucketScanKeyPath = mock(S3ScanKeyPathOption.class); - given(firstBucketScanBucketOption.getkeyPrefix()).willReturn(firstBucketScanKeyPath); - given(firstBucketScanKeyPath.getS3scanIncludeOptions()).willReturn(List.of(UUID.randomUUID().toString())); + given(firstBucketScanBucketOption.getS3ScanFilter()).willReturn(firstBucketScanKeyPath); + given(firstBucketScanKeyPath.getS3scanIncludePrefixOptions()).willReturn(List.of(UUID.randomUUID().toString())); given(firstBucketScanKeyPath.getS3ScanExcludeSuffixOptions()).willReturn(List.of(".invalid")); scanOptionsList.add(firstBucketScanOptions); @@ -90,8 +90,8 @@ void getNextPartition_supplier_returns_expected_PartitionIdentifiers() { given(secondBucketScanOptions.getUseStartDateTime()).willReturn(LocalDateTime.ofInstant(startTime, ZoneId.systemDefault())); given(secondBucketScanOptions.getUseEndDateTime()).willReturn(LocalDateTime.ofInstant(endTime, ZoneId.systemDefault())); final S3ScanKeyPathOption secondBucketScanKeyPath = mock(S3ScanKeyPathOption.class); - given(secondBucketScanBucketOption.getkeyPrefix()).willReturn(secondBucketScanKeyPath); - given(secondBucketScanKeyPath.getS3scanIncludeOptions()).willReturn(null); + given(secondBucketScanBucketOption.getS3ScanFilter()).willReturn(secondBucketScanKeyPath); + given(secondBucketScanKeyPath.getS3scanIncludePrefixOptions()).willReturn(null); given(secondBucketScanKeyPath.getS3ScanExcludeSuffixOptions()).willReturn(null); scanOptionsList.add(secondBucketScanOptions); diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/S3ScanServiceTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/S3ScanServiceTest.java index 9e5dc128a5..09807e24b8 100644 --- a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/S3ScanServiceTest.java +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/S3ScanServiceTest.java @@ -67,15 +67,15 @@ void scan_service_with_valid_s3_scan_configuration_test_and_verify() { final S3ScanBucketOption s3ScanBucketOption = mock(S3ScanBucketOption.class); when(s3ScanBucketOption.getName()).thenReturn(bucketName); S3ScanKeyPathOption s3ScanKeyPathOption = mock(S3ScanKeyPathOption.class); - when(s3ScanKeyPathOption.getS3scanIncludeOptions()).thenReturn(includeKeyPathList); + when(s3ScanKeyPathOption.getS3scanIncludePrefixOptions()).thenReturn(includeKeyPathList); when(s3ScanBucketOption.getRange()).thenReturn(null); - when(s3ScanBucketOption.getkeyPrefix()).thenReturn(s3ScanKeyPathOption); + when(s3ScanBucketOption.getS3ScanFilter()).thenReturn(s3ScanKeyPathOption); when(bucket.getS3ScanBucketOption()).thenReturn(s3ScanBucketOption); when(s3ScanScanOptions.getBuckets()).thenReturn(List.of(bucket)); when(s3SourceConfig.getS3ScanScanOptions()).thenReturn(s3ScanScanOptions); S3ScanService service = new S3ScanService(s3SourceConfig, s3ClientBuilderFactory, s3ObjectHandler, bucketOwnerProvider, sourceCoordinator, acknowledgementSetManager, s3ObjectDeleteWorker, pluginMetrics); final List scanOptionsBuilder = service.getScanOptions(); - assertThat(scanOptionsBuilder.get(0).getBucketOption().getkeyPrefix().getS3scanIncludeOptions(),sameInstance(includeKeyPathList)); + assertThat(scanOptionsBuilder.get(0).getBucketOption().getS3ScanFilter().getS3scanIncludePrefixOptions(),sameInstance(includeKeyPathList)); assertThat(scanOptionsBuilder.get(0).getBucketOption().getName(),sameInstance(bucketName)); assertThat(scanOptionsBuilder.get(0).getUseStartDateTime(),equalTo(startDateTime)); assertThat(scanOptionsBuilder.get(0).getUseEndDateTime(),equalTo(startDateTime.plus(range))); @@ -93,16 +93,16 @@ void scan_service_with_valid_bucket_time_range_configuration_test_and_verify() { final S3ScanBucketOption s3ScanBucketOption = mock(S3ScanBucketOption.class); when(s3ScanBucketOption.getName()).thenReturn(bucketName); S3ScanKeyPathOption s3ScanKeyPathOption = mock(S3ScanKeyPathOption.class); - when(s3ScanKeyPathOption.getS3scanIncludeOptions()).thenReturn(includeKeyPathList); + when(s3ScanKeyPathOption.getS3scanIncludePrefixOptions()).thenReturn(includeKeyPathList); when(s3ScanBucketOption.getStartTime()).thenReturn(startDateTime); when(s3ScanBucketOption.getRange()).thenReturn(range); - when(s3ScanBucketOption.getkeyPrefix()).thenReturn(s3ScanKeyPathOption); + when(s3ScanBucketOption.getS3ScanFilter()).thenReturn(s3ScanKeyPathOption); when(bucket.getS3ScanBucketOption()).thenReturn(s3ScanBucketOption); when(s3ScanScanOptions.getBuckets()).thenReturn(List.of(bucket)); when(s3SourceConfig.getS3ScanScanOptions()).thenReturn(s3ScanScanOptions); S3ScanService service = new S3ScanService(s3SourceConfig, s3ClientBuilderFactory, s3ObjectHandler, bucketOwnerProvider, sourceCoordinator, acknowledgementSetManager, s3ObjectDeleteWorker, pluginMetrics); final List scanOptionsBuilder = service.getScanOptions(); - assertThat(scanOptionsBuilder.get(0).getBucketOption().getkeyPrefix().getS3scanIncludeOptions(),sameInstance(includeKeyPathList)); + assertThat(scanOptionsBuilder.get(0).getBucketOption().getS3ScanFilter().getS3scanIncludePrefixOptions(),sameInstance(includeKeyPathList)); assertThat(scanOptionsBuilder.get(0).getBucketOption().getName(),sameInstance(bucketName)); assertThat(scanOptionsBuilder.get(0).getUseStartDateTime(),equalTo(startDateTime)); assertThat(scanOptionsBuilder.get(0).getUseEndDateTime(),equalTo(startDateTime.plus(range))); @@ -120,14 +120,14 @@ void scan_service_with_no_time_range_configuration_test_and_verify() { when(s3ScanBucketOption.getName()).thenReturn(bucketName); when(s3ScanBucketOption.getRange()).thenReturn(null); S3ScanKeyPathOption s3ScanKeyPathOption = mock(S3ScanKeyPathOption.class); - when(s3ScanKeyPathOption.getS3scanIncludeOptions()).thenReturn(includeKeyPathList); - when(s3ScanBucketOption.getkeyPrefix()).thenReturn(s3ScanKeyPathOption); + when(s3ScanKeyPathOption.getS3scanIncludePrefixOptions()).thenReturn(includeKeyPathList); + when(s3ScanBucketOption.getS3ScanFilter()).thenReturn(s3ScanKeyPathOption); when(bucket.getS3ScanBucketOption()).thenReturn(s3ScanBucketOption); when(s3ScanScanOptions.getBuckets()).thenReturn(List.of(bucket)); when(s3SourceConfig.getS3ScanScanOptions()).thenReturn(s3ScanScanOptions); S3ScanService service = new S3ScanService(s3SourceConfig, s3ClientBuilderFactory, s3ObjectHandler, bucketOwnerProvider, sourceCoordinator, acknowledgementSetManager, s3ObjectDeleteWorker, pluginMetrics); final List scanOptionsBuilder = service.getScanOptions(); - assertThat(scanOptionsBuilder.get(0).getBucketOption().getkeyPrefix().getS3scanIncludeOptions(),sameInstance(includeKeyPathList)); + assertThat(scanOptionsBuilder.get(0).getBucketOption().getS3ScanFilter().getS3scanIncludePrefixOptions(),sameInstance(includeKeyPathList)); assertThat(scanOptionsBuilder.get(0).getBucketOption().getName(),sameInstance(bucketName)); assertThat(scanOptionsBuilder.get(0).getUseStartDateTime(),equalTo(null)); assertThat(scanOptionsBuilder.get(0).getUseEndDateTime(),equalTo(null)); diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOptionTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOptionTest.java index 276eceffe1..69661e6efd 100644 --- a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOptionTest.java +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOptionTest.java @@ -22,8 +22,8 @@ public class S3ScanBucketOptionTest { @Test public void s3scan_bucket_options_with_scan_buckets_yaml_configuration_test() throws JsonProcessingException { final String bucketOptionsYaml = " name: test-s3-source-test-output\n" + - " key_prefix:\n" + - " include:\n" + + " filter:\n" + + " include_prefix:\n" + " - bucket2\n" + " exclude_suffix:\n" + " - .jpeg\n" + @@ -35,10 +35,10 @@ public void s3scan_bucket_options_with_scan_buckets_yaml_configuration_test() th " - .gzip"; final S3ScanBucketOption s3ScanBucketOption = objectMapper.readValue(bucketOptionsYaml, S3ScanBucketOption.class); assertThat(s3ScanBucketOption.getName(), equalTo("test-s3-source-test-output")); - assertThat(s3ScanBucketOption.getkeyPrefix(), instanceOf(S3ScanKeyPathOption.class)); - assertThat(s3ScanBucketOption.getkeyPrefix().getS3scanIncludeOptions(),instanceOf(List.class)); - assertThat(s3ScanBucketOption.getkeyPrefix().getS3scanIncludeOptions().get(0),equalTo("bucket2")); - assertThat(s3ScanBucketOption.getkeyPrefix().getS3ScanExcludeSuffixOptions(),instanceOf(List.class)); - assertThat(s3ScanBucketOption.getkeyPrefix().getS3ScanExcludeSuffixOptions().get(1),equalTo(".png")); + assertThat(s3ScanBucketOption.getS3ScanFilter(), instanceOf(S3ScanKeyPathOption.class)); + assertThat(s3ScanBucketOption.getS3ScanFilter().getS3scanIncludePrefixOptions(),instanceOf(List.class)); + assertThat(s3ScanBucketOption.getS3ScanFilter().getS3scanIncludePrefixOptions().get(0),equalTo("bucket2")); + assertThat(s3ScanBucketOption.getS3ScanFilter().getS3ScanExcludeSuffixOptions(),instanceOf(List.class)); + assertThat(s3ScanBucketOption.getS3ScanFilter().getS3ScanExcludeSuffixOptions().get(1),equalTo(".png")); } } diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOptionsTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOptionsTest.java index 2235abd2e0..3c6b4112f1 100644 --- a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOptionsTest.java +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOptionsTest.java @@ -26,17 +26,17 @@ public void s3_scan_bucket_option_yaml_configuration_test() throws JsonProcessin final String bucketOptionsYaml = " bucket:\n" + " name: test-s3-source-test-output\n" + - " key_prefix:\n" + - " include:\n" + + " filter:\n" + + " include_prefix:\n" + " - bucket2\n" + " exclude_suffix:\n" + " - .jpeg"; final S3ScanBucketOptions s3ScanBucketOptions = objectMapper.readValue(bucketOptionsYaml, S3ScanBucketOptions.class); assertThat(s3ScanBucketOptions.getS3ScanBucketOption(),instanceOf(S3ScanBucketOption.class)); assertThat(s3ScanBucketOptions.getS3ScanBucketOption().getName(),equalTo("test-s3-source-test-output")); - assertThat(s3ScanBucketOptions.getS3ScanBucketOption().getkeyPrefix().getS3scanIncludeOptions(),instanceOf(List.class)); - assertThat(s3ScanBucketOptions.getS3ScanBucketOption().getkeyPrefix().getS3scanIncludeOptions().get(0), Matchers.equalTo("bucket2")); - assertThat(s3ScanBucketOptions.getS3ScanBucketOption().getkeyPrefix().getS3ScanExcludeSuffixOptions(),instanceOf(List.class)); - assertThat(s3ScanBucketOptions.getS3ScanBucketOption().getkeyPrefix().getS3ScanExcludeSuffixOptions().get(0), Matchers.equalTo(".jpeg")); + assertThat(s3ScanBucketOptions.getS3ScanBucketOption().getS3ScanFilter().getS3scanIncludePrefixOptions(),instanceOf(List.class)); + assertThat(s3ScanBucketOptions.getS3ScanBucketOption().getS3ScanFilter().getS3scanIncludePrefixOptions().get(0), Matchers.equalTo("bucket2")); + assertThat(s3ScanBucketOptions.getS3ScanBucketOption().getS3ScanFilter().getS3ScanExcludeSuffixOptions(),instanceOf(List.class)); + assertThat(s3ScanBucketOptions.getS3ScanBucketOption().getS3ScanFilter().getS3ScanExcludeSuffixOptions().get(0), Matchers.equalTo(".jpeg")); } } diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanScanOptionsTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanScanOptionsTest.java index 74954947cd..e0c7890520 100644 --- a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanScanOptionsTest.java +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanScanOptionsTest.java @@ -30,8 +30,8 @@ public void s3scan_options_test_with_scan_yaml_configuration_test() throws JsonP " buckets:\n" + " - bucket:\n" + " name: test-s3-source-test-output\n" + - " key_prefix:\n" + - " include:\n" + + " filter:\n" + + " include_prefix:\n" + " - bucket2\n" + " exclude_suffix:\n" + " - .jpeg"; @@ -41,11 +41,11 @@ public void s3scan_options_test_with_scan_yaml_configuration_test() throws JsonP assertThat(s3ScanScanOptions.getRange(),equalTo(Duration.parse("P90DT3H4M"))); assertThat(s3ScanScanOptions.getBuckets(),instanceOf(List.class)); assertThat(s3ScanScanOptions.getBuckets().get(0).getS3ScanBucketOption().getName(),equalTo("test-s3-source-test-output")); - assertThat(s3ScanScanOptions.getBuckets().get(0).getS3ScanBucketOption().getkeyPrefix().getS3ScanExcludeSuffixOptions(),instanceOf(List.class)); - assertThat(s3ScanScanOptions.getBuckets().get(0).getS3ScanBucketOption().getkeyPrefix().getS3scanIncludeOptions(),instanceOf(List.class)); - assertThat(s3ScanScanOptions.getBuckets().get(0).getS3ScanBucketOption().getkeyPrefix().getS3scanIncludeOptions().get(0), + assertThat(s3ScanScanOptions.getBuckets().get(0).getS3ScanBucketOption().getS3ScanFilter().getS3ScanExcludeSuffixOptions(),instanceOf(List.class)); + assertThat(s3ScanScanOptions.getBuckets().get(0).getS3ScanBucketOption().getS3ScanFilter().getS3scanIncludePrefixOptions(),instanceOf(List.class)); + assertThat(s3ScanScanOptions.getBuckets().get(0).getS3ScanBucketOption().getS3ScanFilter().getS3scanIncludePrefixOptions().get(0), equalTo("bucket2")); - assertThat(s3ScanScanOptions.getBuckets().get(0).getS3ScanBucketOption().getkeyPrefix().getS3ScanExcludeSuffixOptions().get(0), + assertThat(s3ScanScanOptions.getBuckets().get(0).getS3ScanBucketOption().getS3ScanFilter().getS3ScanExcludeSuffixOptions().get(0), equalTo(".jpeg")); } } diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanSchedulingOptionsTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanSchedulingOptionsTest.java index 9d707a2606..edd8b92f62 100644 --- a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanSchedulingOptionsTest.java +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanSchedulingOptionsTest.java @@ -19,9 +19,9 @@ public class S3ScanSchedulingOptionsTest { @Test public void s3_scan_bucket_option_yaml_configuration_test() throws JsonProcessingException { - final String schedulingOptionsYaml = "rate: \"PT1H\" \ncount: 2 \n"; + final String schedulingOptionsYaml = "interval: \"PT1H\" \ncount: 2 \n"; final S3ScanSchedulingOptions s3ScanSchedulingOptions = objectMapper.readValue(schedulingOptionsYaml, S3ScanSchedulingOptions.class); assertThat(s3ScanSchedulingOptions.getCount(), equalTo(2)); - assertThat(s3ScanSchedulingOptions.getRate(), equalTo(Duration.ofHours(1))); + assertThat(s3ScanSchedulingOptions.getInterval(), equalTo(Duration.ofHours(1))); } } \ No newline at end of file From e54c838d9e0b89548e61f3ad23e5645084676062 Mon Sep 17 00:00:00 2001 From: Jonah Calvo Date: Thu, 10 Aug 2023 15:33:20 -0500 Subject: [PATCH 07/28] Adding cardinality key support for AD processor (#3073) * Adding cardinality key support for AD processor Signed-off-by: Jonah Calvo * Refactor hash function to common package. Add metrics for RCF instances. Implement optional verbose mode for RCF Signed-off-by: Jonah Calvo --------- Signed-off-by: Jonah Calvo --- .../AggregateActionSynchronizer.java | 5 +- .../aggregate/AggregateGroupManager.java | 15 +- .../aggregate/AggregateProcessor.java | 15 +- .../AggregateActionSynchronizerTest.java | 3 +- .../aggregate/AggregateGroupManagerTest.java | 17 +- .../aggregate/AggregateProcessorTest.java | 17 +- .../anomaly-detector-processor/build.gradle | 9 +- .../anomalydetector/AnomalyDetectorMode.java | 4 +- .../AnomalyDetectorProcessor.java | 40 ++++- .../AnomalyDetectorProcessorConfig.java | 14 ++ .../modes/RandomCutForestMode.java | 5 +- .../AnomalyDetectorProcessorTests.java | 159 +++++++++++------- .../modes/RandomCutForestModeTests.java | 47 +++++- .../hasher/IdentificationKeysHasher.java} | 12 +- .../hasher/IdentificationKeysHasherTest.java} | 34 ++-- 15 files changed, 261 insertions(+), 135 deletions(-) rename data-prepper-plugins/{aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateIdentificationKeysHasher.java => common/src/main/java/org/opensearch/dataprepper/plugins/hasher/IdentificationKeysHasher.java} (76%) rename data-prepper-plugins/{aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateIdentificationKeysHasherTest.java => common/src/test/java/org/opensearch/dataprepper/plugins/hasher/IdentificationKeysHasherTest.java} (65%) diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateActionSynchronizer.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateActionSynchronizer.java index 14eeae80f2..c60e1da6e7 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateActionSynchronizer.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateActionSynchronizer.java @@ -8,6 +8,7 @@ import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.event.Event; import io.micrometer.core.instrument.Counter; +import org.opensearch.dataprepper.plugins.hasher.IdentificationKeysHasher; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,7 +50,7 @@ private AggregateActionSynchronizer(final AggregateAction aggregateAction, final this.actionConcludeGroupEventsProcessingErrors = pluginMetrics.counter(ACTION_CONCLUDE_GROUP_EVENTS_PROCESSING_ERRORS); } - AggregateActionOutput concludeGroup(final AggregateIdentificationKeysHasher.IdentificationKeysMap hash, final AggregateGroup aggregateGroup, final boolean forceConclude) { + AggregateActionOutput concludeGroup(final IdentificationKeysHasher.IdentificationKeysMap hash, final AggregateGroup aggregateGroup, final boolean forceConclude) { final Lock concludeGroupLock = aggregateGroup.getConcludeGroupLock(); final Lock handleEventForGroupLock = aggregateGroup.getHandleEventForGroupLock(); @@ -74,7 +75,7 @@ AggregateActionOutput concludeGroup(final AggregateIdentificationKeysHasher.Iden return actionOutput; } - AggregateActionResponse handleEventForGroup(final Event event, final AggregateIdentificationKeysHasher.IdentificationKeysMap hash, final AggregateGroup aggregateGroup) { + AggregateActionResponse handleEventForGroup(final Event event, final IdentificationKeysHasher.IdentificationKeysMap hash, final AggregateGroup aggregateGroup) { final Lock concludeGroupLock = aggregateGroup.getConcludeGroupLock(); final Lock handleEventForGroupLock = aggregateGroup.getHandleEventForGroupLock(); diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateGroupManager.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateGroupManager.java index 19431e131d..9d271aa40b 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateGroupManager.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateGroupManager.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.processor.aggregate; import com.google.common.collect.Maps; +import org.opensearch.dataprepper.plugins.hasher.IdentificationKeysHasher; import java.time.Duration; import java.util.ArrayList; @@ -14,20 +15,20 @@ class AggregateGroupManager { - private final Map allGroups = Maps.newConcurrentMap(); + private final Map allGroups = Maps.newConcurrentMap(); private final Duration groupDuration; AggregateGroupManager(final Duration groupDuration) { this.groupDuration = groupDuration; } - AggregateGroup getAggregateGroup(final AggregateIdentificationKeysHasher.IdentificationKeysMap identificationKeysMap) { + AggregateGroup getAggregateGroup(final IdentificationKeysHasher.IdentificationKeysMap identificationKeysMap) { return allGroups.computeIfAbsent(identificationKeysMap, (hash) -> new AggregateGroup(identificationKeysMap.getKeyMap())); } - List> getGroupsToConclude(final boolean forceConclude) { - final List> groupsToConclude = new ArrayList<>(); - for (final Map.Entry groupEntry : allGroups.entrySet()) { + List> getGroupsToConclude(final boolean forceConclude) { + final List> groupsToConclude = new ArrayList<>(); + for (final Map.Entry groupEntry : allGroups.entrySet()) { if (groupEntry.getValue().shouldConcludeGroup(groupDuration) || forceConclude) { groupsToConclude.add(groupEntry); } @@ -35,12 +36,12 @@ List, Record< private final AggregateProcessorConfig aggregateProcessorConfig; private final AggregateGroupManager aggregateGroupManager; private final AggregateActionSynchronizer aggregateActionSynchronizer; - private final AggregateIdentificationKeysHasher aggregateIdentificationKeysHasher; + private final IdentificationKeysHasher identificationKeysHasher; private final AggregateAction aggregateAction; private boolean forceConclude = false; @@ -51,15 +52,15 @@ public class AggregateProcessor extends AbstractProcessor, Record< @DataPrepperPluginConstructor public AggregateProcessor(final AggregateProcessorConfig aggregateProcessorConfig, final PluginMetrics pluginMetrics, final PluginFactory pluginFactory, final ExpressionEvaluator expressionEvaluator) { this(aggregateProcessorConfig, pluginMetrics, pluginFactory, new AggregateGroupManager(aggregateProcessorConfig.getGroupDuration()), - new AggregateIdentificationKeysHasher(aggregateProcessorConfig.getIdentificationKeys()), new AggregateActionSynchronizer.AggregateActionSynchronizerProvider(), expressionEvaluator); + new IdentificationKeysHasher(aggregateProcessorConfig.getIdentificationKeys()), new AggregateActionSynchronizer.AggregateActionSynchronizerProvider(), expressionEvaluator); } public AggregateProcessor(final AggregateProcessorConfig aggregateProcessorConfig, final PluginMetrics pluginMetrics, final PluginFactory pluginFactory, final AggregateGroupManager aggregateGroupManager, - final AggregateIdentificationKeysHasher aggregateIdentificationKeysHasher, final AggregateActionSynchronizer.AggregateActionSynchronizerProvider aggregateActionSynchronizerProvider, final ExpressionEvaluator expressionEvaluator) { + final IdentificationKeysHasher identificationKeysHasher, final AggregateActionSynchronizer.AggregateActionSynchronizerProvider aggregateActionSynchronizerProvider, final ExpressionEvaluator expressionEvaluator) { super(pluginMetrics); this.aggregateProcessorConfig = aggregateProcessorConfig; this.aggregateGroupManager = aggregateGroupManager; this.expressionEvaluator = expressionEvaluator; - this.aggregateIdentificationKeysHasher = aggregateIdentificationKeysHasher; + this.identificationKeysHasher = identificationKeysHasher; this.aggregateAction = loadAggregateAction(pluginFactory); this.aggregateActionSynchronizer = aggregateActionSynchronizerProvider.provide(aggregateAction, aggregateGroupManager, pluginMetrics); @@ -82,8 +83,8 @@ private AggregateAction loadAggregateAction(final PluginFactory pluginFactory) { public Collection> doExecute(Collection> records) { final List> recordsOut = new LinkedList<>(); - final List> groupsToConclude = aggregateGroupManager.getGroupsToConclude(forceConclude); - for (final Map.Entry groupEntry : groupsToConclude) { + final List> groupsToConclude = aggregateGroupManager.getGroupsToConclude(forceConclude); + for (final Map.Entry groupEntry : groupsToConclude) { final AggregateActionOutput actionOutput = aggregateActionSynchronizer.concludeGroup(groupEntry.getKey(), groupEntry.getValue(), forceConclude); final List concludeGroupEvents = actionOutput != null ? actionOutput.getEvents() : null; @@ -105,7 +106,7 @@ public Collection> doExecute(Collection> records) { handleEventsDropped++; continue; } - final AggregateIdentificationKeysHasher.IdentificationKeysMap identificationKeysMap = aggregateIdentificationKeysHasher.createIdentificationKeysMapFromEvent(event); + final IdentificationKeysHasher.IdentificationKeysMap identificationKeysMap = identificationKeysHasher.createIdentificationKeysMapFromEvent(event); final AggregateGroup aggregateGroupForEvent = aggregateGroupManager.getAggregateGroup(identificationKeysMap); final AggregateActionResponse handleEventResponse = aggregateActionSynchronizer.handleEventForGroup(event, identificationKeysMap, aggregateGroupForEvent); diff --git a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateActionSynchronizerTest.java b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateActionSynchronizerTest.java index 4b6e71611a..ed8195d0fb 100644 --- a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateActionSynchronizerTest.java +++ b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateActionSynchronizerTest.java @@ -17,6 +17,7 @@ import org.mockito.junit.jupiter.MockitoExtension; import org.mockito.junit.jupiter.MockitoSettings; import org.mockito.quality.Strictness; +import org.opensearch.dataprepper.plugins.hasher.IdentificationKeysHasher; import java.time.Duration; import java.util.List; @@ -47,7 +48,7 @@ public class AggregateActionSynchronizerTest { private AggregateGroup aggregateGroup; @Mock - private AggregateIdentificationKeysHasher.IdentificationKeysMap identificationKeysMap; + private IdentificationKeysHasher.IdentificationKeysMap identificationKeysMap; @Mock private AggregateActionResponse aggregateActionResponse; diff --git a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateGroupManagerTest.java b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateGroupManagerTest.java index 3ddca61aa9..03d3c634b6 100644 --- a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateGroupManagerTest.java +++ b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateGroupManagerTest.java @@ -7,6 +7,7 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.plugins.hasher.IdentificationKeysHasher; import java.time.Duration; import java.util.Collections; @@ -28,7 +29,7 @@ public class AggregateGroupManagerTest { private AggregateGroupManager aggregateGroupManager; - private AggregateIdentificationKeysHasher.IdentificationKeysMap identificationKeysMap; + private IdentificationKeysHasher.IdentificationKeysMap identificationKeysMap; private static final Duration TEST_GROUP_DURATION = Duration.ofSeconds(new Random().nextInt(10) + 10); @@ -37,7 +38,7 @@ void setup() { final Map identificationKeysHash = new HashMap<>(); identificationKeysHash.put(UUID.randomUUID().toString(), UUID.randomUUID().toString()); - identificationKeysMap = new AggregateIdentificationKeysHasher.IdentificationKeysMap(identificationKeysHash); + identificationKeysMap = new IdentificationKeysHasher.IdentificationKeysMap(identificationKeysHash); } private AggregateGroupManager createObjectUnderTest() { @@ -92,16 +93,16 @@ void getGroupsToConclude_returns_correct_group() { final AggregateGroup groupToConclude = mock(AggregateGroup.class); when(groupToConclude.shouldConcludeGroup(TEST_GROUP_DURATION)).thenReturn(true); - final AggregateIdentificationKeysHasher.IdentificationKeysMap hashForGroupToConclude = mock(AggregateIdentificationKeysHasher.IdentificationKeysMap.class); + final IdentificationKeysHasher.IdentificationKeysMap hashForGroupToConclude = mock(IdentificationKeysHasher.IdentificationKeysMap.class); final AggregateGroup groupToNotConclude = mock(AggregateGroup.class); when(groupToNotConclude.shouldConcludeGroup(TEST_GROUP_DURATION)).thenReturn(false); - final AggregateIdentificationKeysHasher.IdentificationKeysMap hashForGroupToNotConclude = mock(AggregateIdentificationKeysHasher.IdentificationKeysMap.class); + final IdentificationKeysHasher.IdentificationKeysMap hashForGroupToNotConclude = mock(IdentificationKeysHasher.IdentificationKeysMap.class); aggregateGroupManager.putGroupWithHash(hashForGroupToConclude, groupToConclude); aggregateGroupManager.putGroupWithHash(hashForGroupToNotConclude, groupToNotConclude); - final List> groupsToConclude = aggregateGroupManager.getGroupsToConclude(false); + final List> groupsToConclude = aggregateGroupManager.getGroupsToConclude(false); assertThat(groupsToConclude.size(), equalTo(1)); assertThat(groupsToConclude.get(0), notNullValue()); @@ -114,15 +115,15 @@ void getGroupsToConclude_with_force_conclude_return_all() { aggregateGroupManager = createObjectUnderTest(); final AggregateGroup groupToConclude1 = mock(AggregateGroup.class); - final AggregateIdentificationKeysHasher.IdentificationKeysMap hashForGroupToConclude1 = mock(AggregateIdentificationKeysHasher.IdentificationKeysMap.class); + final IdentificationKeysHasher.IdentificationKeysMap hashForGroupToConclude1 = mock(IdentificationKeysHasher.IdentificationKeysMap.class); final AggregateGroup groupToConclude2 = mock(AggregateGroup.class); - final AggregateIdentificationKeysHasher.IdentificationKeysMap hashForGroupToConclude2 = mock(AggregateIdentificationKeysHasher.IdentificationKeysMap.class); + final IdentificationKeysHasher.IdentificationKeysMap hashForGroupToConclude2 = mock(IdentificationKeysHasher.IdentificationKeysMap.class); aggregateGroupManager.putGroupWithHash(hashForGroupToConclude1, groupToConclude1); aggregateGroupManager.putGroupWithHash(hashForGroupToConclude2, groupToConclude2); - final List> groupsToConclude = aggregateGroupManager.getGroupsToConclude(true); + final List> groupsToConclude = aggregateGroupManager.getGroupsToConclude(true); assertThat(groupsToConclude.size(), equalTo(2)); assertThat(groupsToConclude.get(0), notNullValue()); diff --git a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorTest.java b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorTest.java index 90d912cbd8..ad0d763078 100644 --- a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorTest.java +++ b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorTest.java @@ -25,6 +25,7 @@ import org.junit.jupiter.params.provider.MethodSource; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.plugins.hasher.IdentificationKeysHasher; import java.util.AbstractMap; import java.util.Collection; @@ -53,10 +54,10 @@ public class AggregateProcessorTest { private PluginFactory pluginFactory; @Mock - private AggregateIdentificationKeysHasher aggregateIdentificationKeysHasher; + private IdentificationKeysHasher identificationKeysHasher; @Mock - private AggregateIdentificationKeysHasher.IdentificationKeysMap identificationKeysMap; + private IdentificationKeysHasher.IdentificationKeysMap identificationKeysMap; @Mock private AggregateProcessorConfig aggregateProcessorConfig; @@ -115,7 +116,7 @@ public class AggregateProcessorTest { private Event event; private AggregateProcessor createObjectUnderTest() { - return new AggregateProcessor(aggregateProcessorConfig, pluginMetrics, pluginFactory, aggregateGroupManager, aggregateIdentificationKeysHasher, aggregateActionSynchronizerProvider, expressionEvaluator); + return new AggregateProcessor(aggregateProcessorConfig, pluginMetrics, pluginFactory, aggregateGroupManager, identificationKeysHasher, aggregateActionSynchronizerProvider, expressionEvaluator); } @BeforeEach @@ -160,7 +161,7 @@ void getIdentificationKeys_should_return_configured_identification_keys() { class TestDoExecute { @BeforeEach void setup() { - when(aggregateIdentificationKeysHasher.createIdentificationKeysMapFromEvent(event)) + when(identificationKeysHasher.createIdentificationKeysMapFromEvent(event)) .thenReturn(identificationKeysMap); when(aggregateGroupManager.getAggregateGroup(identificationKeysMap)).thenReturn(aggregateGroup); when(aggregateActionSynchronizer.handleEventForGroup(event, identificationKeysMap, aggregateGroup)).thenReturn(aggregateActionResponse); @@ -209,7 +210,7 @@ void handleEvent_returning_with_condition_eliminates_one_record() { .build(); - when(aggregateIdentificationKeysHasher.createIdentificationKeysMapFromEvent(firstEvent)) + when(identificationKeysHasher.createIdentificationKeysMapFromEvent(firstEvent)) .thenReturn(identificationKeysMap); when(aggregateActionSynchronizer.handleEventForGroup(firstEvent, identificationKeysMap, aggregateGroup)).thenReturn(firstAggregateActionResponse); when(expressionEvaluator.evaluateConditional(condition, event)).thenReturn(true); @@ -267,7 +268,7 @@ void handleEvent_returning_with_event_adds_event_to_records_out() { void concludeGroup_returning_with_no_event_does_not_add_event_to_records_out() { final AggregateProcessor objectUnderTest = createObjectUnderTest(); - final Map.Entry groupEntry = new AbstractMap.SimpleEntry(identificationKeysMap, aggregateGroup); + final Map.Entry groupEntry = new AbstractMap.SimpleEntry(identificationKeysMap, aggregateGroup); when(aggregateGroupManager.getGroupsToConclude(eq(false))).thenReturn(Collections.singletonList(groupEntry)); when(aggregateActionResponse.getEvent()).thenReturn(null); when(aggregateActionSynchronizer.concludeGroup(identificationKeysMap, aggregateGroup, false)).thenReturn(new AggregateActionOutput(List.of())); @@ -289,7 +290,7 @@ void concludeGroup_returning_with_no_event_does_not_add_event_to_records_out() { void concludeGroup_returning_with_event_adds_event_to_records_out() { final AggregateProcessor objectUnderTest = createObjectUnderTest(); - final Map.Entry groupEntry = new AbstractMap.SimpleEntry(identificationKeysMap, aggregateGroup); + final Map.Entry groupEntry = new AbstractMap.SimpleEntry(identificationKeysMap, aggregateGroup); when(aggregateGroupManager.getGroupsToConclude(eq(false))).thenReturn(Collections.singletonList(groupEntry)); when(aggregateActionResponse.getEvent()).thenReturn(null); when(aggregateActionSynchronizer.concludeGroup(identificationKeysMap, aggregateGroup, false)).thenReturn(new AggregateActionOutput(List.of(event))); @@ -314,7 +315,7 @@ void concludeGroup_after_prepare_for_shutdown() { final AggregateProcessor objectUnderTest = createObjectUnderTest(); objectUnderTest.prepareForShutdown(); - final Map.Entry groupEntry = new AbstractMap.SimpleEntry(identificationKeysMap, aggregateGroup); + final Map.Entry groupEntry = new AbstractMap.SimpleEntry(identificationKeysMap, aggregateGroup); when(aggregateGroupManager.getGroupsToConclude(eq(true))).thenReturn(Collections.singletonList(groupEntry)); when(aggregateActionResponse.getEvent()).thenReturn(null); when(aggregateActionSynchronizer.concludeGroup(identificationKeysMap, aggregateGroup, true)).thenReturn(new AggregateActionOutput(List.of(event))); diff --git a/data-prepper-plugins/anomaly-detector-processor/build.gradle b/data-prepper-plugins/anomaly-detector-processor/build.gradle index e76b92a960..76f5ae9513 100644 --- a/data-prepper-plugins/anomaly-detector-processor/build.gradle +++ b/data-prepper-plugins/anomaly-detector-processor/build.gradle @@ -10,12 +10,13 @@ plugins { dependencies { implementation project(':data-prepper-api') implementation project(':data-prepper-test-common') + implementation project(':data-prepper-plugins:common') implementation 'com.fasterxml.jackson.core:jackson-databind' implementation 'io.micrometer:micrometer-core' - implementation 'software.amazon.randomcutforest:randomcutforest-testutils:3.7.0' - implementation 'software.amazon.randomcutforest:randomcutforest-core:3.7.0' - implementation 'software.amazon.randomcutforest:randomcutforest-examples:3.7.0' - implementation 'software.amazon.randomcutforest:randomcutforest-parkservices:3.7.0' + implementation 'software.amazon.randomcutforest:randomcutforest-testutils:3.8.0' + implementation 'software.amazon.randomcutforest:randomcutforest-core:3.8.0' + implementation 'software.amazon.randomcutforest:randomcutforest-examples:3.8.0' + implementation 'software.amazon.randomcutforest:randomcutforest-parkservices:3.8.0' implementation 'software.amazon.randomcutforest:randomcutforest-serialization-json:1.0' testImplementation libs.commons.lang3 } diff --git a/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorMode.java b/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorMode.java index a2fc9e6773..911b587c58 100644 --- a/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorMode.java +++ b/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorMode.java @@ -20,8 +20,10 @@ public interface AnomalyDetectorMode { * * @param keys List of keys which are used as dimensions in the anomaly detector * @since 2.1 + * + * @param verbose Optional, when true, RCF will turn off Auto-Adjust, and anomalies will be continually detected after a level shift */ - void initialize(List keys); + void initialize(List keys, boolean verbose); /** * handles a collection of records diff --git a/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessor.java b/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessor.java index 74e7ba8501..57872c7ecd 100644 --- a/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessor.java +++ b/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessor.java @@ -5,6 +5,7 @@ package org.opensearch.dataprepper.plugins.processor.anomalydetector; +import io.micrometer.core.instrument.Counter; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; @@ -15,26 +16,38 @@ import org.opensearch.dataprepper.model.processor.AbstractProcessor; import org.opensearch.dataprepper.model.processor.Processor; import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.plugins.hasher.IdentificationKeysHasher; import java.util.Collection; +import java.util.HashMap; +import java.util.LinkedList; import java.util.List; +import java.util.Objects; @DataPrepperPlugin(name = "anomaly_detector", pluginType = Processor.class, pluginConfigurationType = AnomalyDetectorProcessorConfig.class) public class AnomalyDetectorProcessor extends AbstractProcessor, Record> { public static final String DEVIATION_KEY = "deviation_from_expected"; public static final String GRADE_KEY = "grade"; + static final String NUMBER_RCF_INSTANCES = "numberRCFInstances"; + private final Boolean verbose; + private final IdentificationKeysHasher identificationKeysHasher; + private final Counter numberRCFInstances; private final List keys; - private final AnomalyDetectorMode mode; + private final PluginFactory pluginFactory; + private final HashMap forestMap; private final AnomalyDetectorProcessorConfig anomalyDetectorProcessorConfig; @DataPrepperPluginConstructor public AnomalyDetectorProcessor(final AnomalyDetectorProcessorConfig anomalyDetectorProcessorConfig, final PluginMetrics pluginMetrics, final PluginFactory pluginFactory) { super(pluginMetrics); + this.identificationKeysHasher = new IdentificationKeysHasher(anomalyDetectorProcessorConfig.getIdentificationKeys()); this.anomalyDetectorProcessorConfig = anomalyDetectorProcessorConfig; - keys = anomalyDetectorProcessorConfig.getKeys(); - mode = loadAnomalyDetectorMode(pluginFactory); - mode.initialize(keys); + this.pluginFactory = pluginFactory; + this.numberRCFInstances = pluginMetrics.counter(NUMBER_RCF_INSTANCES); + this.keys = anomalyDetectorProcessorConfig.getKeys(); + this.verbose = anomalyDetectorProcessorConfig.getVerbose(); + forestMap = new HashMap<>(); } private AnomalyDetectorMode loadAnomalyDetectorMode(final PluginFactory pluginFactory) { @@ -45,7 +58,24 @@ private AnomalyDetectorMode loadAnomalyDetectorMode(final PluginFactory pluginFa @Override public Collection> doExecute(Collection> records) { - return mode.handleEvents(records); + final List> recordsOut = new LinkedList<>(); + + for (final Record record : records) { + final Event event = record.getData(); + // If user has not configured IdentificationKeys, the empty set will always hash to "31", + // so the same forest will be used, and we don't need to write a special case. + final IdentificationKeysHasher.IdentificationKeysMap identificationKeysMap = identificationKeysHasher.createIdentificationKeysMapFromEvent(event); + AnomalyDetectorMode forest = forestMap.get(identificationKeysMap.hashCode()); + + if (Objects.isNull(forest)) { + forest = loadAnomalyDetectorMode(pluginFactory); + forest.initialize(keys, verbose); + forestMap.put(identificationKeysMap.hashCode(), forest); + this.numberRCFInstances.increment(); + } + recordsOut.addAll(forestMap.get(identificationKeysMap.hashCode()).handleEvents(List.of(record))); + } + return recordsOut; } diff --git a/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorConfig.java b/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorConfig.java index 08bf2455c1..c92fdb9000 100644 --- a/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorConfig.java +++ b/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorConfig.java @@ -21,6 +21,12 @@ public class AnomalyDetectorProcessorConfig { @NotEmpty private List keys; + @JsonProperty("identification_keys") + private List identificationKeys; + + @JsonProperty("verbose") + private Boolean verbose = false; + public PluginModel getDetectorMode() { return detectorMode; } @@ -34,4 +40,12 @@ public List getKeys() { return keys; } + public List getIdentificationKeys() { + return identificationKeys; + } + public boolean getVerbose() { + return verbose; + } + + } diff --git a/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/modes/RandomCutForestMode.java b/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/modes/RandomCutForestMode.java index 9a8f6c0b8f..c3be98d136 100644 --- a/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/modes/RandomCutForestMode.java +++ b/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/modes/RandomCutForestMode.java @@ -58,7 +58,7 @@ public RandomCutForestMode(final RandomCutForestModeConfig randomCutForestModeCo } @Override - public void initialize(List keys) { + public void initialize(List keys, boolean verbose) { this.keys = keys; baseDimensions = keys.size(); Precision precision = Precision.FLOAT_32; @@ -79,7 +79,8 @@ public void initialize(List keys) { .transformMethod(transformMethod) .outputAfter(outputAfter) .timeDecay(timeDecay / sampleSize) - .initialAcceptFraction(INITIAL_ACCEPT_FRACTION).build(); + .initialAcceptFraction(INITIAL_ACCEPT_FRACTION) + .autoAdjust(!verbose).build(); forest.setLowerThreshold(LOWER_THRESHOLD); forest.setHorizon(HORIZON_VALUE); } diff --git a/data-prepper-plugins/anomaly-detector-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorTests.java b/data-prepper-plugins/anomaly-detector-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorTests.java index 90336ee09d..65c39518d2 100644 --- a/data-prepper-plugins/anomaly-detector-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorTests.java +++ b/data-prepper-plugins/anomaly-detector-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorTests.java @@ -5,6 +5,10 @@ package org.opensearch.dataprepper.plugins.processor.anomalydetector; +import io.micrometer.core.instrument.Counter; +import io.micrometer.core.instrument.Timer; +import org.junit.jupiter.api.BeforeEach; +import org.opensearch.dataprepper.metrics.MetricNames; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.configuration.PluginModel; import org.opensearch.dataprepper.model.configuration.PluginSetting; @@ -43,6 +47,16 @@ public class AnomalyDetectorProcessorTests { @Mock private PluginMetrics pluginMetrics; + @Mock + private Counter numberRCFInstances; + @Mock + private Counter recordsIn; + + @Mock + private Counter recordsOut; + + @Mock + private Timer timeElapsed; @Mock private AnomalyDetectorProcessorConfig mockConfig; @@ -55,48 +69,29 @@ public class AnomalyDetectorProcessorTests { private AnomalyDetectorProcessor anomalyDetectorProcessor; - static Record buildRecordWithEvent(final Map data) { - return new Record<>(JacksonEvent.builder() - .withData(data) - .withEventType("event") - .build()); - } - private Record getBytesStringMessage(String message, String bytes) { - final Map testData = new HashMap(); - testData.put("message", message); - testData.put("bytes", bytes); - return buildRecordWithEvent(testData); - } + @BeforeEach + void setUp() { + when(mockConfig.getKeys()).thenReturn(new ArrayList(Collections.singleton("latency"))); + RandomCutForestModeConfig randomCutForestModeConfig = new RandomCutForestModeConfig(); - private Record getLatencyMessage(String message, Object latency) { - final Map testData = new HashMap(); - testData.put("message", message); - testData.put("latency", latency); - return buildRecordWithEvent(testData); - } + when(mockConfig.getDetectorMode()).thenReturn(modeConfiguration); + when(modeConfiguration.getPluginName()).thenReturn(UUID.randomUUID().toString()); + when(modeConfiguration.getPluginSettings()).thenReturn(Collections.emptyMap()); + when(pluginFactory.loadPlugin(eq(AnomalyDetectorMode.class), any(PluginSetting.class))) + .thenAnswer(invocation -> new RandomCutForestMode(randomCutForestModeConfig)); + + when(pluginMetrics.counter(AnomalyDetectorProcessor.NUMBER_RCF_INSTANCES)).thenReturn(numberRCFInstances); + when(pluginMetrics.counter(MetricNames.RECORDS_IN)).thenReturn(recordsIn); + when(pluginMetrics.counter(MetricNames.RECORDS_OUT)).thenReturn(recordsOut); + when(pluginMetrics.timer(MetricNames.TIME_ELAPSED)).thenReturn(timeElapsed); - private Record getLatencyBytesMessage(String message, double latency, long bytes) { - final Map testData = new HashMap(); - testData.put("message", message); - testData.put("latency", latency); - testData.put("bytes", bytes); - return buildRecordWithEvent(testData); } @ParameterizedTest @ValueSource(ints = {1, 2, 3, 4, 5, 6}) void testAnomalyDetectorProcessor(int type) { - when(mockConfig.getKeys()).thenReturn(new ArrayList(Collections.singleton("latency"))); - RandomCutForestModeConfig randomCutForestModeConfig = new RandomCutForestModeConfig(); - AnomalyDetectorMode anomalyDetectorMode = new RandomCutForestMode(randomCutForestModeConfig); - when(mockConfig.getDetectorMode()).thenReturn(modeConfiguration); - when(modeConfiguration.getPluginName()).thenReturn(UUID.randomUUID().toString()); - when(modeConfiguration.getPluginSettings()).thenReturn(Collections.emptyMap()); - - when(pluginFactory.loadPlugin(eq(AnomalyDetectorMode.class), any(PluginSetting.class))) - .thenReturn(anomalyDetectorMode); anomalyDetectorProcessor = new AnomalyDetectorProcessor(mockConfig, pluginMetrics, pluginFactory); final int numSamples = 1024; final List> records = new ArrayList>(); @@ -146,19 +141,7 @@ void testAnomalyDetectorProcessor(int type) { @Test void testAnomalyDetectorProcessorTwoKeys() { - List keyList = new ArrayList(); - keyList.add("latency"); - keyList.add("bytes"); - when(mockConfig.getKeys()).thenReturn(keyList); - RandomCutForestModeConfig randomCutForestModeConfig = new RandomCutForestModeConfig(); - AnomalyDetectorMode anomalyDetectorMode = new RandomCutForestMode(randomCutForestModeConfig); - when(mockConfig.getDetectorMode()).thenReturn(modeConfiguration); - when(modeConfiguration.getPluginName()).thenReturn(UUID.randomUUID().toString()); - when(modeConfiguration.getPluginSettings()).thenReturn(Collections.emptyMap()); - - when(pluginFactory.loadPlugin(eq(AnomalyDetectorMode.class), any(PluginSetting.class))) - .thenReturn(anomalyDetectorMode); anomalyDetectorProcessor = new AnomalyDetectorProcessor(mockConfig, pluginMetrics, pluginFactory); final int numSamples = 1024; final List> records = new ArrayList>(); @@ -170,7 +153,7 @@ void testAnomalyDetectorProcessorTwoKeys() { assertThat(recordsWithAnomaly.size(), equalTo(1)); Event event = recordsWithAnomaly.get(0).getData(); List deviation = event.get(AnomalyDetectorProcessor.DEVIATION_KEY, List.class); - for (int i = 0; i < keyList.size(); i++) { + for (int i = 0; i < mockConfig.getKeys().size(); i++) { assertThat((double)deviation.get(i), greaterThan(9.0)); } double grade = (double)event.get(AnomalyDetectorProcessor.GRADE_KEY, Double.class); @@ -180,15 +163,6 @@ void testAnomalyDetectorProcessorTwoKeys() { @Test void testAnomalyDetectorProcessorNoMatchingKeys() { when(mockConfig.getKeys()).thenReturn(new ArrayList(Collections.singleton("bytes"))); - RandomCutForestModeConfig randomCutForestModeConfig = new RandomCutForestModeConfig(); - AnomalyDetectorMode anomalyDetectorMode = new RandomCutForestMode(randomCutForestModeConfig); - - when(mockConfig.getDetectorMode()).thenReturn(modeConfiguration); - when(modeConfiguration.getPluginName()).thenReturn(UUID.randomUUID().toString()); - when(modeConfiguration.getPluginSettings()).thenReturn(Collections.emptyMap()); - - when(pluginFactory.loadPlugin(eq(AnomalyDetectorMode.class), any(PluginSetting.class))) - .thenReturn(anomalyDetectorMode); anomalyDetectorProcessor = new AnomalyDetectorProcessor(mockConfig, pluginMetrics, pluginFactory); final int numSamples = 1024; final List> records = new ArrayList>(); @@ -204,15 +178,6 @@ void testAnomalyDetectorProcessorNoMatchingKeys() { @Test void testAnomalyDetectorProcessorInvalidTypeKeys() { when(mockConfig.getKeys()).thenReturn(new ArrayList(Collections.singleton("bytes"))); - RandomCutForestModeConfig randomCutForestModeConfig = new RandomCutForestModeConfig(); - AnomalyDetectorMode anomalyDetectorMode = new RandomCutForestMode(randomCutForestModeConfig); - - when(mockConfig.getDetectorMode()).thenReturn(modeConfiguration); - when(modeConfiguration.getPluginName()).thenReturn(UUID.randomUUID().toString()); - when(modeConfiguration.getPluginSettings()).thenReturn(Collections.emptyMap()); - - when(pluginFactory.loadPlugin(eq(AnomalyDetectorMode.class), any(PluginSetting.class))) - .thenReturn(anomalyDetectorMode); anomalyDetectorProcessor = new AnomalyDetectorProcessor(mockConfig, pluginMetrics, pluginFactory); final int numSamples = 1024; final List> records = new ArrayList>(); @@ -221,4 +186,70 @@ void testAnomalyDetectorProcessorInvalidTypeKeys() { } assertThrows(RuntimeException.class, () -> anomalyDetectorProcessor.doExecute(records)); } + + @Test + void testAnomalyDetectorCardinality() { + List identificationKeyList = new ArrayList(); + identificationKeyList.add("ip"); + when(mockConfig.getIdentificationKeys()).thenReturn(identificationKeyList); + + anomalyDetectorProcessor = new AnomalyDetectorProcessor(mockConfig, pluginMetrics, pluginFactory); + final int numSamples = 1024; + final List> records = new ArrayList>(); + for (int i = 0; i < numSamples; i++) { + if (i % 2 == 0) { + records.add(getLatencyBytesMessageWithIp(UUID.randomUUID().toString(), ThreadLocalRandom.current().nextDouble(0.5, 0.6), ThreadLocalRandom.current().nextLong(100, 110), "1.1.1.1")); + } else { + records.add(getLatencyBytesMessageWithIp(UUID.randomUUID().toString(), ThreadLocalRandom.current().nextDouble(15.5, 15.6), ThreadLocalRandom.current().nextLong(1000, 1110), "255.255.255.255")); + } + } + + anomalyDetectorProcessor.doExecute(records); + + final List> slowRecordFromFastIp = (List>) anomalyDetectorProcessor.doExecute(Collections.singletonList(getLatencyBytesMessageWithIp(UUID.randomUUID().toString(), ThreadLocalRandom.current().nextDouble(15.5, 15.8), ThreadLocalRandom.current().nextLong(1000, 1110), "1.1.1.1"))); + assertThat(slowRecordFromFastIp.size(), equalTo(1)); + + final List> slowRecordFromSlowIp = (List>) anomalyDetectorProcessor.doExecute(Collections.singletonList(getLatencyBytesMessageWithIp(UUID.randomUUID().toString(), ThreadLocalRandom.current().nextDouble(15.5, 15.6), ThreadLocalRandom.current().nextLong(1000, 1110), "255.255.255.255"))); + assertThat(slowRecordFromSlowIp.size(), equalTo(0)); + + } + + static Record buildRecordWithEvent(final Map data) { + return new Record<>(JacksonEvent.builder() + .withData(data) + .withEventType("event") + .build()); + } + + private Record getBytesStringMessage(String message, String bytes) { + final Map testData = new HashMap(); + testData.put("message", message); + testData.put("bytes", bytes); + return buildRecordWithEvent(testData); + } + + private Record getLatencyMessage(String message, Object latency) { + final Map testData = new HashMap(); + testData.put("message", message); + testData.put("latency", latency); + return buildRecordWithEvent(testData); + } + + private Record getLatencyBytesMessage(String message, double latency, long bytes) { + final Map testData = new HashMap(); + testData.put("message", message); + testData.put("latency", latency); + testData.put("bytes", bytes); + return buildRecordWithEvent(testData); + } + + private Record getLatencyBytesMessageWithIp(String message, double latency, long bytes, String ip) { + final Map testData = new HashMap(); + testData.put("message", message); + testData.put("latency", latency); + testData.put("bytes", bytes); + testData.put("ip", ip); + + return buildRecordWithEvent(testData); + } } diff --git a/data-prepper-plugins/anomaly-detector-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/modes/RandomCutForestModeTests.java b/data-prepper-plugins/anomaly-detector-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/modes/RandomCutForestModeTests.java index be3afe1fad..c52788433f 100644 --- a/data-prepper-plugins/anomaly-detector-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/modes/RandomCutForestModeTests.java +++ b/data-prepper-plugins/anomaly-detector-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/modes/RandomCutForestModeTests.java @@ -27,6 +27,7 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.junit.jupiter.MockitoExtension; + import static org.mockito.Mockito.mock; import org.mockito.Mock; import static org.mockito.Mockito.when; @@ -92,7 +93,7 @@ private RandomCutForestMode createObjectUnderTest() { void testRandomCutForestMode() { randomCutForestMode = createObjectUnderTest(); List keys = new ArrayList(Collections.singleton("latency")); - randomCutForestMode.initialize(keys); + randomCutForestMode.initialize(keys, false); final int numSamples = 1024; List> records = new ArrayList>(); for (int i = 0; i < numSamples; i++) { @@ -118,7 +119,7 @@ void testRandomCutForestModeMultipleKeys() { String longFieldName = "bytes"; keyList.add(floatFieldName); keyList.add(longFieldName); - randomCutForestMode.initialize(keyList); + randomCutForestMode.initialize(keyList, false); final int numSamples = 1024; List> records = new ArrayList>(); for (int i = 0; i < numSamples; i++) { @@ -151,7 +152,7 @@ void testRandomCutForestModeWithOutputAfter() { String longFieldName = "hour"; keyList.add(floatFieldName); keyList.add(longFieldName); - randomCutForestMode.initialize(keyList); + randomCutForestMode.initialize(keyList, false); final int numSamples = (365+200)*24+4; // number of samples more than a year List> records = new ArrayList>(); long hour = 0; @@ -166,4 +167,44 @@ void testRandomCutForestModeWithOutputAfter() { final List> anomalyRecords = randomCutForestMode.handleEvents(recordsWithAnomaly).stream().collect(toList());; assertThat(anomalyRecords.size(), equalTo(1)); } + + @Test + void testRandomCutForestModeVerboseTrue() { + randomCutForestMode = createObjectUnderTest(); + List keys = new ArrayList(Collections.singleton("latency")); + randomCutForestMode.initialize(keys, true); + final int numSamples = 1024; + List> records = new ArrayList>(); + for (int i = 0; i < numSamples; i++) { + records.add(getLatencyMessage(UUID.randomUUID().toString(), ThreadLocalRandom.current().nextDouble(0.5, 0.6))); + } + randomCutForestMode.handleEvents(records); + final List> recordsWithAnomaly = new ArrayList>(); + for (int i = 0; i < numSamples; i++) { + recordsWithAnomaly.add(getLatencyMessage(UUID.randomUUID().toString(), ThreadLocalRandom.current().nextDouble(1, 1.1))); + } + + final List> anomalyRecords = randomCutForestMode.handleEvents(recordsWithAnomaly).stream().collect(toList());; + assertThat(anomalyRecords.size(), greaterThan(5)); + } + + @Test + void testRandomCutForestModeVerboseFalse() { + randomCutForestMode = createObjectUnderTest(); + List keys = new ArrayList(Collections.singleton("latency")); + randomCutForestMode.initialize(keys, false); + final int numSamples = 1024; + List> records = new ArrayList>(); + for (int i = 0; i < numSamples; i++) { + records.add(getLatencyMessage(UUID.randomUUID().toString(), ThreadLocalRandom.current().nextDouble(0.5, 0.6))); + } + randomCutForestMode.handleEvents(records); + final List> recordsWithAnomaly = new ArrayList>(); + for (int i = 0; i < numSamples; i++) { + recordsWithAnomaly.add(getLatencyMessage(UUID.randomUUID().toString(), ThreadLocalRandom.current().nextDouble(1, 1.1))); + } + + final List> anomalyRecords = randomCutForestMode.handleEvents(recordsWithAnomaly).stream().collect(toList());; + assertThat(anomalyRecords.size(), equalTo(1)); + } } diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateIdentificationKeysHasher.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/hasher/IdentificationKeysHasher.java similarity index 76% rename from data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateIdentificationKeysHasher.java rename to data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/hasher/IdentificationKeysHasher.java index cc2a7ca580..2dd86ae893 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateIdentificationKeysHasher.java +++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/hasher/IdentificationKeysHasher.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.processor.aggregate; +package org.opensearch.dataprepper.plugins.hasher; import org.opensearch.dataprepper.model.event.Event; @@ -12,13 +12,13 @@ import java.util.Map; import java.util.Objects; -class AggregateIdentificationKeysHasher { +public class IdentificationKeysHasher { private final List identificationKeys; - AggregateIdentificationKeysHasher(final List identificationKeys) { + public IdentificationKeysHasher(final List identificationKeys) { this.identificationKeys = identificationKeys; } - IdentificationKeysMap createIdentificationKeysMapFromEvent(final Event event) { + public IdentificationKeysMap createIdentificationKeysMapFromEvent(final Event event) { final Map identificationKeysMap = new HashMap<>(); for (final String identificationKey : identificationKeys) { identificationKeysMap.put(identificationKey, event.get(identificationKey, Object.class)); @@ -29,7 +29,7 @@ IdentificationKeysMap createIdentificationKeysMapFromEvent(final Event event) { public static class IdentificationKeysMap { private final Map keyMap; - IdentificationKeysMap(final Map keyMap) { + public IdentificationKeysMap(final Map keyMap) { this.keyMap = keyMap; } @@ -46,7 +46,7 @@ public int hashCode() { return Objects.hash(keyMap); } - Map getKeyMap() { + public Map getKeyMap() { return keyMap; } } diff --git a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateIdentificationKeysHasherTest.java b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/hasher/IdentificationKeysHasherTest.java similarity index 65% rename from data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateIdentificationKeysHasherTest.java rename to data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/hasher/IdentificationKeysHasherTest.java index 63221811f6..aac85d02b8 100644 --- a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateIdentificationKeysHasherTest.java +++ b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/hasher/IdentificationKeysHasherTest.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.processor.aggregate; +package org.opensearch.dataprepper.plugins.hasher; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.JacksonEvent; @@ -21,10 +21,10 @@ import static org.hamcrest.CoreMatchers.not; import static org.hamcrest.MatcherAssert.assertThat; -public class AggregateIdentificationKeysHasherTest { +public class IdentificationKeysHasherTest { private Event event; private List identificationKeys; - private AggregateIdentificationKeysHasher aggregateIdentificationKeysHasher; + private IdentificationKeysHasher identificationKeysHasher; @BeforeEach void setup() { @@ -33,18 +33,18 @@ void setup() { identificationKeys.add("secondIdentificationKey"); } - private AggregateIdentificationKeysHasher createObjectUnderTest() { - return new AggregateIdentificationKeysHasher(identificationKeys); + private IdentificationKeysHasher createObjectUnderTest() { + return new IdentificationKeysHasher(identificationKeys); } @Test void createIdentificationKeysMapFromEvent_returns_expected_IdentficationKeysMap() { - aggregateIdentificationKeysHasher = createObjectUnderTest(); + identificationKeysHasher = createObjectUnderTest(); final Map eventMap = new HashMap<>(); eventMap.put("firstIdentificationKey", UUID.randomUUID().toString()); eventMap.put("secondIdentificationKey", UUID.randomUUID().toString()); - final AggregateIdentificationKeysHasher.IdentificationKeysMap expectedResult = new AggregateIdentificationKeysHasher.IdentificationKeysMap(new HashMap<>(eventMap)); + final IdentificationKeysHasher.IdentificationKeysMap expectedResult = new IdentificationKeysHasher.IdentificationKeysMap(new HashMap<>(eventMap)); eventMap.put(UUID.randomUUID().toString(), UUID.randomUUID().toString()); @@ -53,20 +53,20 @@ void createIdentificationKeysMapFromEvent_returns_expected_IdentficationKeysMap( .withData(eventMap) .build(); - final AggregateIdentificationKeysHasher.IdentificationKeysMap result = aggregateIdentificationKeysHasher.createIdentificationKeysMapFromEvent(event); + final IdentificationKeysHasher.IdentificationKeysMap result = identificationKeysHasher.createIdentificationKeysMapFromEvent(event); assertThat(result, equalTo(expectedResult)); } @Test void createIdentificationKeysMapFromEvent_where_Event_does_not_contain_one_of_the_identification_keys_returns_expected_Map() { - aggregateIdentificationKeysHasher = createObjectUnderTest(); + identificationKeysHasher = createObjectUnderTest(); final Map eventMap = new HashMap<>(); eventMap.put("firstIdentificationKey", UUID.randomUUID().toString()); final Map mapForExpectedHash = new HashMap<>(eventMap); mapForExpectedHash.put("secondIdentificationKey", null); - final AggregateIdentificationKeysHasher.IdentificationKeysMap expectedResult = new AggregateIdentificationKeysHasher.IdentificationKeysMap(mapForExpectedHash); + final IdentificationKeysHasher.IdentificationKeysMap expectedResult = new IdentificationKeysHasher.IdentificationKeysMap(mapForExpectedHash); eventMap.put(UUID.randomUUID().toString(), UUID.randomUUID().toString()); @@ -75,13 +75,13 @@ void createIdentificationKeysMapFromEvent_where_Event_does_not_contain_one_of_th .withData(eventMap) .build(); - final AggregateIdentificationKeysHasher.IdentificationKeysMap result = aggregateIdentificationKeysHasher.createIdentificationKeysMapFromEvent(event); + final IdentificationKeysHasher.IdentificationKeysMap result = identificationKeysHasher.createIdentificationKeysMapFromEvent(event); assertThat(result, equalTo(expectedResult)); } @Test void identical_identification_hashes_but_different_objects_are_considered_equal() { - aggregateIdentificationKeysHasher = createObjectUnderTest(); + identificationKeysHasher = createObjectUnderTest(); final Map eventMap = new HashMap<>(); eventMap.put("firstIdentificationKey", UUID.randomUUID().toString()); eventMap.put("secondIdentificationKey", UUID.randomUUID().toString()); @@ -92,15 +92,15 @@ void identical_identification_hashes_but_different_objects_are_considered_equal( .withData(eventMap) .build(); - final AggregateIdentificationKeysHasher.IdentificationKeysMap result = aggregateIdentificationKeysHasher.createIdentificationKeysMapFromEvent(event); - final AggregateIdentificationKeysHasher.IdentificationKeysMap secondResult = aggregateIdentificationKeysHasher.createIdentificationKeysMapFromEvent(event); + final IdentificationKeysHasher.IdentificationKeysMap result = identificationKeysHasher.createIdentificationKeysMapFromEvent(event); + final IdentificationKeysHasher.IdentificationKeysMap secondResult = identificationKeysHasher.createIdentificationKeysMapFromEvent(event); assertThat(result, equalTo(secondResult)); } @Test void different_identification_hashes_are_not_considered_equal() { - aggregateIdentificationKeysHasher = createObjectUnderTest(); + identificationKeysHasher = createObjectUnderTest(); final Map eventMap = new HashMap<>(); eventMap.put("firstIdentificationKey", UUID.randomUUID().toString()); eventMap.put("secondIdentificationKey", UUID.randomUUID().toString()); @@ -119,8 +119,8 @@ void different_identification_hashes_are_not_considered_equal() { .withData(secondEventMap) .build(); - final AggregateIdentificationKeysHasher.IdentificationKeysMap result = aggregateIdentificationKeysHasher.createIdentificationKeysMapFromEvent(event); - final AggregateIdentificationKeysHasher.IdentificationKeysMap secondResult = aggregateIdentificationKeysHasher.createIdentificationKeysMapFromEvent(secondEvent); + final IdentificationKeysHasher.IdentificationKeysMap result = identificationKeysHasher.createIdentificationKeysMapFromEvent(event); + final IdentificationKeysHasher.IdentificationKeysMap secondResult = identificationKeysHasher.createIdentificationKeysMapFromEvent(secondEvent); assertThat(result, is(not(equalTo(secondResult)))); } From 439e4af90b19eb40e0ac75f5b233bbd0c7cf890c Mon Sep 17 00:00:00 2001 From: David Venable Date: Thu, 10 Aug 2023 20:02:37 -0500 Subject: [PATCH 08/28] The rss-source is defined twice in the settings.gradle and this removes the extra one. (#3134) Signed-off-by: David Venable --- settings.gradle | 1 - 1 file changed, 1 deletion(-) diff --git a/settings.gradle b/settings.gradle index ffe3ddea42..bbbcfb471c 100644 --- a/settings.gradle +++ b/settings.gradle @@ -112,7 +112,6 @@ include 'release:archives:linux' include 'release:docker' include 'release:maven' include 'e2e-test:peerforwarder' -include 'rss-source' include 'data-prepper-plugins:failures-common' include 'data-prepper-plugins:newline-codecs' include 'data-prepper-plugins:avro-codecs' From 4e2e911c3eef81ac20d1c6664b56d78806c0aab0 Mon Sep 17 00:00:00 2001 From: Taylor Gray Date: Fri, 11 Aug 2023 10:54:42 -0500 Subject: [PATCH 09/28] Add support for scheduled scan to s3 scan (#3140) Add support for scheduled scan to s3 scan Signed-off-by: Taylor Gray --- .../S3ScanPartitionCreationSupplier.java | 114 +++++++++++++- .../plugins/source/ScanObjectWorker.java | 2 +- .../configuration/S3ScanBucketOption.java | 2 - .../configuration/S3ScanScanOptions.java | 13 +- .../S3ScanSchedulingOptions.java | 13 +- .../S3ScanPartitionCreationSupplierTest.java | 142 +++++++++++++++++- .../configuration/S3ScanScanOptionsTest.java | 3 - 7 files changed, 266 insertions(+), 23 deletions(-) diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplier.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplier.java index cadcaf71e8..731803f7cf 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplier.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplier.java @@ -7,10 +7,14 @@ import org.opensearch.dataprepper.model.source.coordinator.PartitionIdentifier; import org.opensearch.dataprepper.plugins.source.configuration.S3ScanKeyPathOption; +import org.opensearch.dataprepper.plugins.source.configuration.S3ScanSchedulingOptions; import org.opensearch.dataprepper.plugins.source.ownership.BucketOwnerProvider; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; import software.amazon.awssdk.services.s3.model.ListObjectsV2Response; +import software.amazon.awssdk.services.s3.model.S3Object; import software.amazon.awssdk.utils.Pair; import java.time.Instant; @@ -18,6 +22,7 @@ import java.time.ZoneId; import java.time.ZonedDateTime; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Objects; @@ -26,23 +31,38 @@ public class S3ScanPartitionCreationSupplier implements Function, List> { + private static final Logger LOG = LoggerFactory.getLogger(S3ScanPartitionCreationSupplier.class); + private static final String BUCKET_OBJECT_PARTITION_KEY_FORMAT = "%s|%s"; + static final String SCAN_COUNT = "SCAN_COUNT"; + static final String LAST_SCAN_TIME = "LAST_SCAN_TIME"; private final S3Client s3Client; private final BucketOwnerProvider bucketOwnerProvider; private final List scanOptionsList; + private final S3ScanSchedulingOptions schedulingOptions; public S3ScanPartitionCreationSupplier(final S3Client s3Client, final BucketOwnerProvider bucketOwnerProvider, - final List scanOptionsList) { + final List scanOptionsList, + final S3ScanSchedulingOptions schedulingOptions) { this.s3Client = s3Client; this.bucketOwnerProvider = bucketOwnerProvider; this.scanOptionsList = scanOptionsList; + this.schedulingOptions = schedulingOptions; } @Override public List apply(final Map globalStateMap) { + if (globalStateMap.isEmpty()) { + initializeGlobalStateMap(globalStateMap); + } + + if (shouldScanBeSkipped(globalStateMap)) { + return Collections.emptyList(); + } + final List objectsToProcess = new ArrayList<>(); for (final ScanOptions scanOptions : scanOptionsList) { @@ -60,27 +80,33 @@ public List apply(final Map globalStateMap) s3ScanKeyPathOption.getS3scanIncludePrefixOptions().forEach(includePath -> { listObjectsV2Request.prefix(includePath); objectsToProcess.addAll(listFilteredS3ObjectsForBucket(excludeItems, listObjectsV2Request, - scanOptions.getBucketOption().getName(), scanOptions.getUseStartDateTime(), scanOptions.getUseEndDateTime())); + scanOptions.getBucketOption().getName(), scanOptions.getUseStartDateTime(), scanOptions.getUseEndDateTime(), globalStateMap)); }); else objectsToProcess.addAll(listFilteredS3ObjectsForBucket(excludeItems, listObjectsV2Request, - scanOptions.getBucketOption().getName(), scanOptions.getUseStartDateTime(), scanOptions.getUseEndDateTime())); + scanOptions.getBucketOption().getName(), scanOptions.getUseStartDateTime(), scanOptions.getUseEndDateTime(), globalStateMap)); } + globalStateMap.put(SCAN_COUNT, (Integer) globalStateMap.get(SCAN_COUNT) + 1); + globalStateMap.put(LAST_SCAN_TIME, Instant.now().toEpochMilli()); + return objectsToProcess; } private List listFilteredS3ObjectsForBucket(final List excludeKeyPaths, - final ListObjectsV2Request.Builder listObjectsV2Request, - final String bucket, - final LocalDateTime startDateTime, - final LocalDateTime endDateTime) { + final ListObjectsV2Request.Builder listObjectsV2Request, + final String bucket, + final LocalDateTime startDateTime, + final LocalDateTime endDateTime, + final Map globalStateMap) { + Instant mostRecentLastModifiedTimestamp = globalStateMap.containsKey(bucket) ? Instant.parse((String) globalStateMap.get(bucket)) : null; final List allPartitionIdentifiers = new ArrayList<>(); ListObjectsV2Response listObjectsV2Response = null; do { listObjectsV2Response = s3Client.listObjectsV2(listObjectsV2Request.fetchOwner(true).continuationToken(Objects.nonNull(listObjectsV2Response) ? listObjectsV2Response.nextContinuationToken() : null).build()); allPartitionIdentifiers.addAll(listObjectsV2Response.contents().stream() + .filter(s3Object -> isLastModifiedTimeAfterMostRecentScanForBucket(bucket, s3Object, globalStateMap)) .map(s3Object -> Pair.of(s3Object.key(), instantToLocalDateTime(s3Object.lastModified()))) .filter(keyTimestampPair -> !keyTimestampPair.left().endsWith("/")) .filter(keyTimestampPair -> excludeKeyPaths.stream() @@ -89,8 +115,11 @@ private List listFilteredS3ObjectsForBucket(final List PartitionIdentifier.builder().withPartitionKey(String.format(BUCKET_OBJECT_PARTITION_KEY_FORMAT, bucket, objectKey)).build()) .collect(Collectors.toList())); + + mostRecentLastModifiedTimestamp = getMostRecentLastModifiedTimestamp(listObjectsV2Response, mostRecentLastModifiedTimestamp); } while (listObjectsV2Response.isTruncated()); + globalStateMap.put(bucket, Objects.nonNull(mostRecentLastModifiedTimestamp) ? mostRecentLastModifiedTimestamp.toString() : null); return allPartitionIdentifiers; } @@ -106,9 +135,78 @@ private LocalDateTime instantToLocalDateTime(final Instant instant) { private boolean isKeyMatchedBetweenTimeRange(final LocalDateTime lastModifiedTime, final LocalDateTime startDateTime, final LocalDateTime endDateTime){ - if (Objects.isNull(startDateTime) || Objects.isNull(endDateTime)) { + if (Objects.isNull(startDateTime) || Objects.isNull(endDateTime) || Objects.nonNull(schedulingOptions)) { return true; } return lastModifiedTime.isAfter(startDateTime) && lastModifiedTime.isBefore(endDateTime); } + + private void initializeGlobalStateMap(final Map globalStateMap) { + globalStateMap.put(SCAN_COUNT, 0); + } + + private boolean isLastModifiedTimeAfterMostRecentScanForBucket(final String bucketName, + final S3Object s3Object, + final Map globalStateMap) { + if (!globalStateMap.containsKey(bucketName) || Objects.isNull(globalStateMap.get(bucketName))) { + return true; + } + + final Instant lastProcessedObjectTimestamp = Instant.parse((String) globalStateMap.get(bucketName)); + + return s3Object.lastModified().compareTo(lastProcessedObjectTimestamp) > 0; + } + + private Instant getMostRecentLastModifiedTimestamp(final ListObjectsV2Response listObjectsV2Response, + Instant mostRecentLastModifiedTimestamp) { + + if (Objects.isNull(schedulingOptions)) { + return null; + } + + for (final S3Object s3Object : listObjectsV2Response.contents()) { + if (Objects.isNull(mostRecentLastModifiedTimestamp) || s3Object.lastModified().isAfter(mostRecentLastModifiedTimestamp)) { + mostRecentLastModifiedTimestamp = s3Object.lastModified(); + } + } + + return mostRecentLastModifiedTimestamp; + } + + private boolean shouldScanBeSkipped(final Map globalStateMap) { + if (Objects.isNull(schedulingOptions) && hasAlreadyBeenScanned(globalStateMap)) { + LOG.info("Skipping scan because the buckets have already been scanned once"); + return true; + } + + if (Objects.nonNull(schedulingOptions) && + (hasReachedMaxScanCount(globalStateMap) || !hasReachedScheduledScanTime(globalStateMap))) { + + if (hasReachedMaxScanCount(globalStateMap)) { + LOG.info("Skipping scan as the max scan count {} has been reached", schedulingOptions.getCount()); + } else { + LOG.info("Skipping scan as the interval of {} seconds has not been reached yet", schedulingOptions.getInterval().toSeconds()); + } + + return true; + } + + return false; + } + + private boolean hasAlreadyBeenScanned(final Map globalStateMap) { + return (Integer) globalStateMap.get(SCAN_COUNT) > 0; + } + + private boolean hasReachedMaxScanCount(final Map globalStateMap) { + return (Integer) globalStateMap.get(SCAN_COUNT) >= schedulingOptions.getCount(); + } + + private boolean hasReachedScheduledScanTime(final Map globalStateMap) { + if (!globalStateMap.containsKey(LAST_SCAN_TIME)) { + return true; + } + + return Instant.now().minus(schedulingOptions.getInterval()).isAfter(Instant.ofEpochMilli((Long) globalStateMap.get(LAST_SCAN_TIME))); + } } diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/ScanObjectWorker.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/ScanObjectWorker.java index 2b1d8388aa..1257efeaf4 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/ScanObjectWorker.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/ScanObjectWorker.java @@ -93,7 +93,7 @@ public ScanObjectWorker(final S3Client s3Client, acknowledgementSetCallbackCounter = pluginMetrics.counter(ACKNOWLEDGEMENT_SET_CALLBACK_METRIC_NAME); this.sourceCoordinator.initialize(); - this.partitionCreationSupplier = new S3ScanPartitionCreationSupplier(s3Client, bucketOwnerProvider, scanOptionsBuilderList); + this.partitionCreationSupplier = new S3ScanPartitionCreationSupplier(s3Client, bucketOwnerProvider, scanOptionsBuilderList, s3ScanSchedulingOptions); } @Override diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOption.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOption.java index c840b82907..323b0480f4 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOption.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOption.java @@ -6,7 +6,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.annotation.JsonDeserialize; -import com.fasterxml.jackson.datatype.jsr310.deser.DurationDeserializer; import jakarta.validation.constraints.AssertTrue; import org.opensearch.dataprepper.plugins.source.CustomLocalDateTimeDeserializer; @@ -30,7 +29,6 @@ public class S3ScanBucketOption { @JsonProperty("end_time") private LocalDateTime endTime; - @JsonDeserialize(using = DurationDeserializer.class) @JsonProperty("range") private Duration range; diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanScanOptions.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanScanOptions.java index 6948597119..3db6abb179 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanScanOptions.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanScanOptions.java @@ -6,7 +6,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.annotation.JsonDeserialize; -import com.fasterxml.jackson.datatype.jsr310.deser.DurationDeserializer; +import jakarta.validation.Valid; import jakarta.validation.constraints.AssertTrue; import org.opensearch.dataprepper.plugins.source.CustomLocalDateTimeDeserializer; @@ -20,7 +20,7 @@ * Class consists the scan options list bucket configuration properties. */ public class S3ScanScanOptions { - @JsonDeserialize(using = DurationDeserializer.class) + @JsonProperty("range") private Duration range; @@ -33,16 +33,23 @@ public class S3ScanScanOptions { private LocalDateTime endTime; @JsonProperty("buckets") + @Valid private List buckets; @JsonProperty("scheduling") - private S3ScanSchedulingOptions schedulingOptions = new S3ScanSchedulingOptions(); + @Valid + private S3ScanSchedulingOptions schedulingOptions; @AssertTrue(message = "At most two options from start_time, end_time and range can be specified at the same time") public boolean hasValidTimeOptions() { return Stream.of(startTime, endTime, range).filter(Objects::nonNull).count() < 3; } + @AssertTrue(message = "start_time, end_time, and range are not valid options when using scheduling with s3 scan") + public boolean hasValidTimeOptionsWithScheduling() { + return !Objects.nonNull(schedulingOptions) || Stream.of(startTime, endTime, range).noneMatch(Objects::nonNull); + } + public Duration getRange() { return range; } diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanSchedulingOptions.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanSchedulingOptions.java index c474675818..604b8debc2 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanSchedulingOptions.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanSchedulingOptions.java @@ -7,16 +7,23 @@ import com.fasterxml.jackson.annotation.JsonProperty; import jakarta.validation.constraints.Min; +import jakarta.validation.constraints.NotNull; +import org.hibernate.validator.constraints.time.DurationMax; +import org.hibernate.validator.constraints.time.DurationMin; import java.time.Duration; public class S3ScanSchedulingOptions { + @JsonProperty("interval") - private Duration interval = Duration.ofHours(8); + @NotNull + @DurationMin(seconds = 30L, message = "S3 scan interval must be at least 30 seconds") + @DurationMax(days = 365L, message = "S3 scan interval must be less than or equal to 365 days") + private Duration interval; - @Min(1) + @Min(2) @JsonProperty("count") - private int count = 1; + private int count = Integer.MAX_VALUE; public Duration getInterval() { return interval; diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplierTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplierTest.java index d3c557aa06..40a9501766 100644 --- a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplierTest.java +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplierTest.java @@ -15,17 +15,20 @@ import org.opensearch.dataprepper.model.source.coordinator.PartitionIdentifier; import org.opensearch.dataprepper.plugins.source.configuration.S3ScanBucketOption; import org.opensearch.dataprepper.plugins.source.configuration.S3ScanKeyPathOption; +import org.opensearch.dataprepper.plugins.source.configuration.S3ScanSchedulingOptions; import org.opensearch.dataprepper.plugins.source.ownership.BucketOwnerProvider; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; import software.amazon.awssdk.services.s3.model.ListObjectsV2Response; import software.amazon.awssdk.services.s3.model.S3Object; +import java.time.Duration; import java.time.Instant; import java.time.LocalDateTime; import java.time.ZoneId; import java.time.temporal.ChronoUnit; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -39,6 +42,10 @@ import static org.hamcrest.Matchers.notNullValue; import static org.mockito.BDDMockito.given; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.opensearch.dataprepper.plugins.source.S3ScanPartitionCreationSupplier.LAST_SCAN_TIME; +import static org.opensearch.dataprepper.plugins.source.S3ScanPartitionCreationSupplier.SCAN_COUNT; @ExtendWith(MockitoExtension.class) public class S3ScanPartitionCreationSupplierTest { @@ -51,6 +58,8 @@ public class S3ScanPartitionCreationSupplierTest { private List scanOptionsList; + private S3ScanSchedulingOptions schedulingOptions; + @BeforeEach void setup() { scanOptionsList = new ArrayList<>(); @@ -58,11 +67,12 @@ void setup() { private Function, List> createObjectUnderTest() { - return new S3ScanPartitionCreationSupplier(s3Client, bucketOwnerProvider, scanOptionsList); + return new S3ScanPartitionCreationSupplier(s3Client, bucketOwnerProvider, scanOptionsList, schedulingOptions); } @Test - void getNextPartition_supplier_returns_expected_PartitionIdentifiers() { + void getNextPartition_supplier_without_scheduling_options_returns_expected_PartitionIdentifiers() { + schedulingOptions = null; final String firstBucket = UUID.randomUUID().toString(); final String secondBucket = UUID.randomUUID().toString(); @@ -130,11 +140,137 @@ void getNextPartition_supplier_returns_expected_PartitionIdentifiers() { final ArgumentCaptor listObjectsV2RequestArgumentCaptor = ArgumentCaptor.forClass(ListObjectsV2Request.class); given(s3Client.listObjectsV2(listObjectsV2RequestArgumentCaptor.capture())).willReturn(listObjectsResponse); - final List resultingPartitions = partitionCreationSupplier.apply(new HashMap<>()); + final Map globalStateMap = new HashMap<>(); + final List resultingPartitions = partitionCreationSupplier.apply(globalStateMap); + + assertThat(globalStateMap, notNullValue()); + assertThat(globalStateMap.containsKey(SCAN_COUNT), equalTo(true)); + assertThat(globalStateMap.get(SCAN_COUNT), equalTo(1)); + + assertThat(partitionCreationSupplier.apply(globalStateMap), equalTo(Collections.emptyList())); assertThat(resultingPartitions, notNullValue()); assertThat(resultingPartitions.size(), equalTo(expectedPartitionIdentifiers.size())); assertThat(resultingPartitions.stream().map(PartitionIdentifier::getPartitionKey).collect(Collectors.toList()), containsInAnyOrder(expectedPartitionIdentifiers.stream().map(PartitionIdentifier::getPartitionKey).map(Matchers::equalTo).collect(Collectors.toList()))); } + + @Test + void getNextPartition_supplier_with_scheduling_options_returns_expected_PartitionIdentifiers() { + schedulingOptions = mock(S3ScanSchedulingOptions.class); + given(schedulingOptions.getInterval()).willReturn(Duration.ofMillis(0)); + given(schedulingOptions.getCount()).willReturn(2); + + final String firstBucket = "bucket-one"; + final String secondBucket = "bucket-two"; + + final ScanOptions firstBucketScanOptions = mock(ScanOptions.class); + final S3ScanBucketOption firstBucketScanBucketOption = mock(S3ScanBucketOption.class); + given(firstBucketScanOptions.getBucketOption()).willReturn(firstBucketScanBucketOption); + given(firstBucketScanBucketOption.getName()).willReturn(firstBucket); + given(firstBucketScanOptions.getUseStartDateTime()).willReturn(null); + given(firstBucketScanOptions.getUseEndDateTime()).willReturn(null); + final S3ScanKeyPathOption firstBucketScanKeyPath = mock(S3ScanKeyPathOption.class); + given(firstBucketScanBucketOption.getS3ScanFilter()).willReturn(firstBucketScanKeyPath); + given(firstBucketScanKeyPath.getS3scanIncludePrefixOptions()).willReturn(List.of(UUID.randomUUID().toString())); + given(firstBucketScanKeyPath.getS3ScanExcludeSuffixOptions()).willReturn(List.of(".invalid")); + scanOptionsList.add(firstBucketScanOptions); + + final ScanOptions secondBucketScanOptions = mock(ScanOptions.class); + final S3ScanBucketOption secondBucketScanBucketOption = mock(S3ScanBucketOption.class); + given(secondBucketScanOptions.getBucketOption()).willReturn(secondBucketScanBucketOption); + given(secondBucketScanBucketOption.getName()).willReturn(secondBucket); + given(secondBucketScanOptions.getUseStartDateTime()).willReturn(null); + given(secondBucketScanOptions.getUseEndDateTime()).willReturn(null); + final S3ScanKeyPathOption secondBucketScanKeyPath = mock(S3ScanKeyPathOption.class); + given(secondBucketScanBucketOption.getS3ScanFilter()).willReturn(secondBucketScanKeyPath); + given(secondBucketScanKeyPath.getS3scanIncludePrefixOptions()).willReturn(null); + given(secondBucketScanKeyPath.getS3ScanExcludeSuffixOptions()).willReturn(null); + scanOptionsList.add(secondBucketScanOptions); + + final Function, List> partitionCreationSupplier = createObjectUnderTest(); + + final List expectedPartitionIdentifiers = new ArrayList<>(); + + final ListObjectsV2Response listObjectsResponse = mock(ListObjectsV2Response.class); + final List s3ObjectsList = new ArrayList<>(); + + final S3Object invalidFolderObject = mock(S3Object.class); + given(invalidFolderObject.key()).willReturn("folder-key/"); + given(invalidFolderObject.lastModified()).willReturn(Instant.now()); + s3ObjectsList.add(invalidFolderObject); + + final S3Object invalidForFirstBucketSuffixObject = mock(S3Object.class); + given(invalidForFirstBucketSuffixObject.key()).willReturn("test.invalid"); + given(invalidForFirstBucketSuffixObject.lastModified()).willReturn(Instant.now()); + s3ObjectsList.add(invalidForFirstBucketSuffixObject); + expectedPartitionIdentifiers.add(PartitionIdentifier.builder().withPartitionKey(secondBucket + "|" + invalidForFirstBucketSuffixObject.key()).build()); + + final Instant mostRecentFirstScan = Instant.now().plusSeconds(1); + final S3Object validObject = mock(S3Object.class); + given(validObject.key()).willReturn("valid"); + given(validObject.lastModified()).willReturn(mostRecentFirstScan); + s3ObjectsList.add(validObject); + expectedPartitionIdentifiers.add(PartitionIdentifier.builder().withPartitionKey(firstBucket + "|" + validObject.key()).build()); + expectedPartitionIdentifiers.add(PartitionIdentifier.builder().withPartitionKey(secondBucket + "|" + validObject.key()).build()); + + final S3Object secondScanObject = mock(S3Object.class); + final Instant mostRecentSecondScan = Instant.now().plusSeconds(10); + given(secondScanObject.key()).willReturn("second-scan"); + given(secondScanObject.lastModified()).willReturn(mostRecentSecondScan); + + final List expectedPartitionIdentifiersSecondScan = new ArrayList<>(); + expectedPartitionIdentifiersSecondScan.add(PartitionIdentifier.builder().withPartitionKey(firstBucket + "|" + secondScanObject.key()).build()); + expectedPartitionIdentifiersSecondScan.add(PartitionIdentifier.builder().withPartitionKey(secondBucket + "|" + secondScanObject.key()).build()); + + final List secondScanObjects = new ArrayList<>(s3ObjectsList); + secondScanObjects.add(secondScanObject); + given(listObjectsResponse.contents()) + .willReturn(s3ObjectsList) + .willReturn(s3ObjectsList) + .willReturn(s3ObjectsList) + .willReturn(s3ObjectsList) + .willReturn(secondScanObjects) + .willReturn(secondScanObjects) + .willReturn(secondScanObjects) + .willReturn(secondScanObjects); + + final ArgumentCaptor listObjectsV2RequestArgumentCaptor = ArgumentCaptor.forClass(ListObjectsV2Request.class); + given(s3Client.listObjectsV2(listObjectsV2RequestArgumentCaptor.capture())).willReturn(listObjectsResponse); + + final Map globalStateMap = new HashMap<>(); + final List resultingPartitions = partitionCreationSupplier.apply(globalStateMap); + + assertThat(resultingPartitions, notNullValue()); + assertThat(resultingPartitions.size(), equalTo(expectedPartitionIdentifiers.size())); + assertThat(resultingPartitions.stream().map(PartitionIdentifier::getPartitionKey).collect(Collectors.toList()), + containsInAnyOrder(expectedPartitionIdentifiers.stream().map(PartitionIdentifier::getPartitionKey) + .map(Matchers::equalTo).collect(Collectors.toList()))); + + assertThat(globalStateMap, notNullValue()); + assertThat(globalStateMap.containsKey(SCAN_COUNT), equalTo(true)); + assertThat(globalStateMap.get(SCAN_COUNT), equalTo(1)); + assertThat(globalStateMap.containsKey(firstBucket), equalTo(true)); + assertThat(globalStateMap.get(firstBucket), equalTo(mostRecentFirstScan.toString())); + assertThat(globalStateMap.containsKey(secondBucket), equalTo(true)); + assertThat(globalStateMap.get(secondBucket), equalTo(mostRecentFirstScan.toString())); + + final List secondScanPartitions = partitionCreationSupplier.apply(globalStateMap); + assertThat(secondScanPartitions.size(), equalTo(expectedPartitionIdentifiersSecondScan.size())); + assertThat(secondScanPartitions.stream().map(PartitionIdentifier::getPartitionKey).collect(Collectors.toList()), + containsInAnyOrder(expectedPartitionIdentifiersSecondScan.stream().map(PartitionIdentifier::getPartitionKey).map(Matchers::equalTo).collect(Collectors.toList()))); + + assertThat(globalStateMap, notNullValue()); + assertThat(globalStateMap.containsKey(SCAN_COUNT), equalTo(true)); + assertThat(globalStateMap.get(SCAN_COUNT), equalTo(2)); + assertThat(globalStateMap.containsKey(firstBucket), equalTo(true)); + assertThat(globalStateMap.get(firstBucket), equalTo(mostRecentSecondScan.toString())); + assertThat(globalStateMap.containsKey(secondBucket), equalTo(true)); + assertThat(globalStateMap.get(secondBucket), equalTo(mostRecentSecondScan.toString())); + assertThat(Instant.ofEpochMilli((Long) globalStateMap.get(LAST_SCAN_TIME)).isBefore(Instant.now()), equalTo(true)); + + assertThat(partitionCreationSupplier.apply(globalStateMap), equalTo(Collections.emptyList())); + + verify(listObjectsResponse, times(8)).contents(); + } } diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanScanOptionsTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanScanOptionsTest.java index e0c7890520..f0e31de164 100644 --- a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanScanOptionsTest.java +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanScanOptionsTest.java @@ -10,7 +10,6 @@ import com.fasterxml.jackson.dataformat.yaml.YAMLGenerator; import org.junit.Test; -import java.time.Duration; import java.time.LocalDateTime; import java.util.List; @@ -25,7 +24,6 @@ public class S3ScanScanOptionsTest { @Test public void s3scan_options_test_with_scan_yaml_configuration_test() throws JsonProcessingException { final String scanYaml = " start_time: 2023-01-21T18:00:00\n" + - " range: P90DT3H4M\n" + " end_time: 2023-04-21T18:00:00\n" + " buckets:\n" + " - bucket:\n" + @@ -38,7 +36,6 @@ public void s3scan_options_test_with_scan_yaml_configuration_test() throws JsonP final S3ScanScanOptions s3ScanScanOptions = objectMapper.readValue(scanYaml, S3ScanScanOptions.class); assertThat(s3ScanScanOptions.getStartTime(),equalTo(LocalDateTime.parse("2023-01-21T18:00:00"))); assertThat(s3ScanScanOptions.getEndTime(),equalTo(LocalDateTime.parse("2023-04-21T18:00:00"))); - assertThat(s3ScanScanOptions.getRange(),equalTo(Duration.parse("P90DT3H4M"))); assertThat(s3ScanScanOptions.getBuckets(),instanceOf(List.class)); assertThat(s3ScanScanOptions.getBuckets().get(0).getS3ScanBucketOption().getName(),equalTo("test-s3-source-test-output")); assertThat(s3ScanScanOptions.getBuckets().get(0).getS3ScanBucketOption().getS3ScanFilter().getS3ScanExcludeSuffixOptions(),instanceOf(List.class)); From f3f63b56c4843f3d7c373032457b41bc81317681 Mon Sep 17 00:00:00 2001 From: David Venable Date: Fri, 11 Aug 2023 10:58:22 -0500 Subject: [PATCH 10/28] Adds S3 sink compression. Resolves #3130. (#3138) Signed-off-by: David Venable --- data-prepper-plugins/s3-sink/build.gradle | 8 +- .../plugins/sink/s3/S3SinkServiceIT.java | 80 ++++++++-- .../dataprepper/plugins/sink/s3/S3Sink.java | 9 +- .../plugins/sink/s3/S3SinkConfig.java | 8 + .../plugins/sink/s3/accumulator/Buffer.java | 5 +- .../s3/accumulator/CompressionBuffer.java | 66 +++++++++ .../accumulator/CompressionBufferFactory.java | 25 ++++ .../sink/s3/accumulator/InMemoryBuffer.java | 22 --- .../sink/s3/accumulator/LocalFileBuffer.java | 20 --- .../s3/compression/CompressionEngine.java | 13 ++ .../s3/compression/CompressionOption.java | 41 +++++ .../s3/compression/GZipCompressionEngine.java | 18 +++ .../s3/compression/NoneCompressionEngine.java | 15 ++ .../plugins/sink/s3/S3SinkTest.java | 2 + .../CompressionBufferFactoryTest.java | 73 +++++++++ .../s3/accumulator/CompressionBufferTest.java | 140 ++++++++++++++++++ .../GZipCompressionEngineTest.java | 60 ++++++++ .../NoneCompressionEngineTest.java | 38 +++++ 18 files changed, 578 insertions(+), 65 deletions(-) create mode 100644 data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBuffer.java create mode 100644 data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBufferFactory.java create mode 100644 data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/CompressionEngine.java create mode 100644 data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/CompressionOption.java create mode 100644 data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/GZipCompressionEngine.java create mode 100644 data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/NoneCompressionEngine.java create mode 100644 data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBufferFactoryTest.java create mode 100644 data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBufferTest.java create mode 100644 data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/compression/GZipCompressionEngineTest.java create mode 100644 data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/compression/NoneCompressionEngineTest.java diff --git a/data-prepper-plugins/s3-sink/build.gradle b/data-prepper-plugins/s3-sink/build.gradle index 00bc6d0f11..5d74fd169d 100644 --- a/data-prepper-plugins/s3-sink/build.gradle +++ b/data-prepper-plugins/s3-sink/build.gradle @@ -23,8 +23,10 @@ dependencies { implementation 'org.jetbrains.kotlin:kotlin-stdlib-common:1.8.21' implementation libs.commons.lang3 testImplementation project(':data-prepper-test-common') - implementation project(':data-prepper-plugins:parquet-codecs') - implementation project(':data-prepper-plugins:parse-json-processor') + testImplementation project(':data-prepper-plugins:parquet-codecs') + testImplementation project(':data-prepper-plugins:parse-json-processor') + testImplementation project(':data-prepper-plugins:csv-processor') + testImplementation project(':data-prepper-plugins:avro-codecs') } test { @@ -55,7 +57,7 @@ task integrationTest(type: Test) { classpath = sourceSets.integrationTest.runtimeClasspath systemProperty 'tests.s3sink.bucket', System.getProperty('tests.s3sink.bucket') - systemProperty 'tests.s3ink.region', System.getProperty('tests.s3sink.region') + systemProperty 'tests.s3sink.region', System.getProperty('tests.s3sink.region') filter { includeTestsMatching '*IT' diff --git a/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkServiceIT.java b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkServiceIT.java index 7134dc47fc..d679663f11 100644 --- a/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkServiceIT.java +++ b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkServiceIT.java @@ -5,10 +5,13 @@ package org.opensearch.dataprepper.plugins.sink.s3; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; import io.micrometer.core.instrument.Counter; import io.micrometer.core.instrument.DistributionSummary; import org.apache.avro.Schema; import org.apache.avro.SchemaBuilder; +import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.parquet.ParquetReadOptions; @@ -24,9 +27,9 @@ import org.apache.parquet.io.RecordReader; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.Type; -import org.hamcrest.CoreMatchers; import org.hamcrest.Matchers; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; @@ -43,13 +46,16 @@ import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.sink.OutputCodecContext; import org.opensearch.dataprepper.model.types.ByteCount; +import org.opensearch.dataprepper.plugins.codec.GZipDecompressionEngine; import org.opensearch.dataprepper.plugins.codec.json.NdjsonOutputCodec; import org.opensearch.dataprepper.plugins.codec.json.NdjsonOutputConfig; import org.opensearch.dataprepper.plugins.codec.parquet.ParquetOutputCodec; import org.opensearch.dataprepper.plugins.codec.parquet.ParquetOutputCodecConfig; import org.opensearch.dataprepper.plugins.sink.s3.accumulator.BufferFactory; +import org.opensearch.dataprepper.plugins.sink.s3.accumulator.CompressionBufferFactory; import org.opensearch.dataprepper.plugins.sink.s3.accumulator.InMemoryBufferFactory; import org.opensearch.dataprepper.plugins.sink.s3.accumulator.ObjectKey; +import org.opensearch.dataprepper.plugins.sink.s3.compression.CompressionOption; import org.opensearch.dataprepper.plugins.sink.s3.configuration.AwsAuthenticationOptions; import org.opensearch.dataprepper.plugins.sink.s3.configuration.ObjectKeyOptions; import org.opensearch.dataprepper.plugins.sink.s3.configuration.ThresholdOptions; @@ -66,6 +72,7 @@ import java.io.File; import java.io.IOException; import java.io.InputStream; +import java.nio.charset.Charset; import java.nio.file.Files; import java.nio.file.StandardCopyOption; import java.time.Duration; @@ -129,7 +136,7 @@ class S3SinkServiceIT { @BeforeEach public void setUp() { - s3region = System.getProperty("tests.s3ink.region"); + s3region = System.getProperty("tests.s3sink.region"); s3Client = S3Client.builder().region(Region.of(s3region)).build(); bucketName = System.getProperty("tests.s3sink.bucket"); @@ -168,19 +175,65 @@ void configureNewLineCodec() { } @Test - void verify_flushed_records_into_s3_bucketNewLine() { + void verify_flushed_records_into_s3_bucketNewLine() throws JsonProcessingException { configureNewLineCodec(); S3SinkService s3SinkService = createObjectUnderTest(); Collection> recordsData = setEventQueue(); s3SinkService.output(recordsData); - String objectData = getS3Object(); + String objectData = new String(getS3Object()); + final ObjectMapper objectMapperForDeserialization = new ObjectMapper(); int count = 0; - String[] objectDataArr = objectData.split("\r\n"); + + String[] objectDataArr = objectData.split(System.lineSeparator()); + assertThat(objectDataArr.length, equalTo(recordsData.size())); + for (Record recordData : recordsData) { - String objectRecord = recordData.getData().toJsonString(); - assertThat(objectDataArr[count], CoreMatchers.containsString(objectRecord)); + final String actualLine = objectDataArr[count]; + final Map actualDeserializedJson = objectMapperForDeserialization.readValue(actualLine, Map.class); + + final Map expectedMap = new HashMap<>(recordData.getData().toMap()); + expectedMap.put("Tag", new ArrayList<>(recordData.getData().getMetadata().getTags())); + assertThat(actualDeserializedJson, equalTo(expectedMap)); + + final String expectedJsonString = recordData.getData().jsonBuilder().includeTags("Tag").toJsonString(); + assertThat(actualLine, equalTo(expectedJsonString)); + count++; + } + } + + @Test + void verify_flushed_records_into_s3_bucketNewLine_with_compression() throws IOException { + configureNewLineCodec(); + bufferFactory = new CompressionBufferFactory(bufferFactory, CompressionOption.GZIP.getCompressionEngine()); + S3SinkService s3SinkService = createObjectUnderTest(); + Collection> recordsData = setEventQueue(); + + s3SinkService.output(recordsData); + byte[] s3ObjectBytes = getS3Object(); + + ByteArrayInputStream s3ObjectInputStream = new ByteArrayInputStream(s3ObjectBytes); + InputStream decompressingInputStream = new GZipDecompressionEngine().createInputStream(s3ObjectInputStream); + + String objectData = IOUtils.toString(decompressingInputStream, Charset.defaultCharset()); + + final ObjectMapper objectMapperForDeserialization = new ObjectMapper(); + int count = 0; + + String[] objectDataArr = objectData.split(System.lineSeparator()); + assertThat(objectDataArr.length, equalTo(recordsData.size())); + + for (Record recordData : recordsData) { + final String actualLine = objectDataArr[count]; + final Map actualDeserializedJson = objectMapperForDeserialization.readValue(actualLine, Map.class); + + final Map expectedMap = new HashMap<>(recordData.getData().toMap()); + expectedMap.put("Tag", new ArrayList<>(recordData.getData().getMetadata().getTags())); + assertThat(actualDeserializedJson, equalTo(expectedMap)); + + final String expectedJsonString = recordData.getData().jsonBuilder().includeTags("Tag").toJsonString(); + assertThat(actualLine, equalTo(expectedJsonString)); count++; } } @@ -202,7 +255,7 @@ private int gets3ObjectCount() { return s3ObjectCount; } - private String getS3Object() { + private byte[] getS3Object() { ListObjectsRequest listObjects = ListObjectsRequest.builder() .bucket(bucketName) @@ -220,8 +273,7 @@ private String getS3Object() { .bucket(bucketName).build(); ResponseBytes objectBytes = s3Client.getObjectAsBytes(objectRequest); - byte[] data = objectBytes.asByteArray(); - return new String(data); + return objectBytes.asByteArray(); } private String getPathPrefix() { @@ -240,20 +292,19 @@ private static Record createRecord() { final EventMetadata defaultEventMetadata = DefaultEventMetadata.builder(). withEventType(EventType.LOG.toString()). withTags(testTags).build(); - Map json = generateJson(testTags); + Map json = generateJson(); final JacksonEvent event = JacksonLog.builder().withData(json).withEventMetadata(defaultEventMetadata).build(); event.setEventHandle(mock(EventHandle.class)); return new Record<>(event); } - private static Map generateJson(Set testTags) { + 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())); - jsonObject.put("Tag", testTags.toArray()); return jsonObject; } @@ -280,6 +331,7 @@ private static List generateRecords(int numberOfRecords) { } @Test + @Disabled void verify_flushed_records_into_s3_bucket_Parquet() throws IOException { configureParquetCodec(); S3SinkService s3SinkService = createObjectUnderTest(); @@ -287,7 +339,7 @@ void verify_flushed_records_into_s3_bucket_Parquet() throws IOException { s3SinkService.output(recordsData); - List> actualRecords = createParquetRecordsList(new ByteArrayInputStream(getS3Object().getBytes())); + List> actualRecords = createParquetRecordsList(new ByteArrayInputStream(getS3Object())); int index = 0; for (final HashMap actualMap : actualRecords) { assertThat(actualMap, notNullValue()); diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3Sink.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3Sink.java index 11aa67637d..c880a72464 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3Sink.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3Sink.java @@ -21,8 +21,10 @@ import org.opensearch.dataprepper.model.sink.SinkContext; import org.opensearch.dataprepper.plugins.sink.s3.accumulator.BufferFactory; import org.opensearch.dataprepper.plugins.sink.s3.accumulator.BufferTypeOptions; +import org.opensearch.dataprepper.plugins.sink.s3.accumulator.CompressionBufferFactory; import org.opensearch.dataprepper.plugins.sink.s3.accumulator.InMemoryBufferFactory; import org.opensearch.dataprepper.plugins.sink.s3.accumulator.LocalFileBufferFactory; +import org.opensearch.dataprepper.plugins.sink.s3.compression.CompressionEngine; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.services.s3.S3Client; @@ -64,11 +66,14 @@ public S3Sink(final PluginSetting pluginSetting, codec = pluginFactory.loadPlugin(OutputCodec.class, codecPluginSettings); sinkInitialized = Boolean.FALSE; + final BufferFactory innerBufferFactory; if (s3SinkConfig.getBufferType().equals(BufferTypeOptions.LOCALFILE)) { - bufferFactory = new LocalFileBufferFactory(); + innerBufferFactory = new LocalFileBufferFactory(); } else { - bufferFactory = new InMemoryBufferFactory(); + innerBufferFactory = new InMemoryBufferFactory(); } + final CompressionEngine compressionEngine = s3SinkConfig.getCompression().getCompressionEngine(); + bufferFactory = new CompressionBufferFactory(innerBufferFactory, compressionEngine); final S3Client s3Client = ClientFactory.createS3Client(s3SinkConfig, awsCredentialsSupplier); s3SinkService = new S3SinkService(s3SinkConfig, bufferFactory, codec, OutputCodecContext.fromSinkContext(sinkContext), s3Client, pluginMetrics); } diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkConfig.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkConfig.java index 1b18994f66..e39856cb12 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkConfig.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkConfig.java @@ -11,6 +11,7 @@ import jakarta.validation.constraints.NotNull; import org.opensearch.dataprepper.model.configuration.PluginModel; import org.opensearch.dataprepper.plugins.sink.s3.accumulator.BufferTypeOptions; +import org.opensearch.dataprepper.plugins.sink.s3.compression.CompressionOption; import org.opensearch.dataprepper.plugins.sink.s3.configuration.AwsAuthenticationOptions; import org.opensearch.dataprepper.plugins.sink.s3.configuration.ObjectKeyOptions; import org.opensearch.dataprepper.plugins.sink.s3.configuration.ThresholdOptions; @@ -36,6 +37,9 @@ public class S3SinkConfig { @JsonProperty("object_key") private ObjectKeyOptions objectKeyOptions; + @JsonProperty("compression") + private CompressionOption compression = CompressionOption.NONE; + @JsonProperty("threshold") @NotNull private ThresholdOptions thresholdOptions; @@ -118,4 +122,8 @@ public int getMaxConnectionRetries() { public int getMaxUploadRetries() { return maxUploadRetries; } + + public CompressionOption getCompression() { + return compression; + } } \ No newline at end of file diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/Buffer.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/Buffer.java index b90775ed47..afd695db2b 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/Buffer.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/Buffer.java @@ -6,7 +6,6 @@ package org.opensearch.dataprepper.plugins.sink.s3.accumulator; import software.amazon.awssdk.services.s3.S3Client; -import java.io.IOException; import java.io.OutputStream; /** @@ -22,11 +21,9 @@ public interface Buffer { int getEventCount(); long getDuration(); - boolean isCodecStarted(); - void setCodecStarted(boolean codecStarted); 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/s3/accumulator/CompressionBuffer.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBuffer.java new file mode 100644 index 0000000000..440c030ac0 --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBuffer.java @@ -0,0 +1,66 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3.accumulator; + +import org.opensearch.dataprepper.plugins.sink.s3.compression.CompressionEngine; +import software.amazon.awssdk.services.s3.S3Client; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.Objects; + +class CompressionBuffer implements Buffer { + private final Buffer innerBuffer; + private final CompressionEngine compressionEngine; + private volatile OutputStream outputStream; + + CompressionBuffer(final Buffer innerBuffer, final CompressionEngine compressionEngine) { + this.innerBuffer = Objects.requireNonNull(innerBuffer); + this.compressionEngine = Objects.requireNonNull(compressionEngine); + } + + @Override + public long getSize() { + return innerBuffer.getSize(); + } + + @Override + public int getEventCount() { + return innerBuffer.getEventCount(); + } + + @Override + public long getDuration() { + return innerBuffer.getDuration(); + } + + @Override + public void flushToS3(final S3Client s3Client, final String bucket, final String key) { + innerBuffer.flushToS3(s3Client, bucket, key); + } + + @Override + public OutputStream getOutputStream() { + if(outputStream == null) { + synchronized (this) { + if(outputStream == null) { + final OutputStream innerBufferOutputStream = innerBuffer.getOutputStream(); + try { + outputStream = compressionEngine.createOutputStream(innerBufferOutputStream); + } catch (final IOException e) { + throw new RuntimeException(e); + } + } + } + } + return outputStream; + } + + @Override + public void setEventCount(final int eventCount) { + innerBuffer.setEventCount(eventCount); + } +} diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBufferFactory.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBufferFactory.java new file mode 100644 index 0000000000..5dcb652f0f --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBufferFactory.java @@ -0,0 +1,25 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3.accumulator; + +import org.opensearch.dataprepper.plugins.sink.s3.compression.CompressionEngine; + +import java.util.Objects; + +public class CompressionBufferFactory implements BufferFactory { + private final BufferFactory innerBufferFactory; + private final CompressionEngine compressionEngine; + + public CompressionBufferFactory(final BufferFactory innerBufferFactory, final CompressionEngine compressionEngine) { + this.innerBufferFactory = Objects.requireNonNull(innerBufferFactory); + this.compressionEngine = Objects.requireNonNull(compressionEngine); + } + + @Override + public Buffer getBuffer() { + return new CompressionBuffer(innerBufferFactory.getBuffer(), compressionEngine); + } +} diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBuffer.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBuffer.java index ea1f3bc697..58121912d7 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBuffer.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBuffer.java @@ -10,7 +10,6 @@ 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; @@ -61,27 +60,6 @@ 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++; - } - @Override - public boolean isCodecStarted() { - return isCodecStarted; - } - - @Override - public void setCodecStarted(boolean codecStarted) { - isCodecStarted = codecStarted; - } @Override public void setEventCount(int eventCount) { this.eventCount = eventCount; diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBuffer.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBuffer.java index 733a2b86fa..52b6229d92 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBuffer.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBuffer.java @@ -77,18 +77,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. */ @@ -113,15 +101,7 @@ protected void removeTemporaryFile() { } } } - @Override - public boolean isCodecStarted() { - return isCodecStarted; - } - @Override - public void setCodecStarted(boolean codecStarted) { - isCodecStarted = codecStarted; - } @Override public void setEventCount(int eventCount) { this.eventCount = eventCount; diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/CompressionEngine.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/CompressionEngine.java new file mode 100644 index 0000000000..46ffc503ad --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/CompressionEngine.java @@ -0,0 +1,13 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3.compression; + +import java.io.IOException; +import java.io.OutputStream; + +public interface CompressionEngine { + OutputStream createOutputStream(OutputStream outputStream) throws IOException; +} diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/CompressionOption.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/CompressionOption.java new file mode 100644 index 0000000000..7e759909d5 --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/CompressionOption.java @@ -0,0 +1,41 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3.compression; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.util.Arrays; +import java.util.Map; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +public enum CompressionOption { + NONE("none", NoneCompressionEngine::new), + GZIP("gzip", GZipCompressionEngine::new); + + private static final Map OPTIONS_MAP = Arrays.stream(CompressionOption.values()) + .collect(Collectors.toMap( + value -> value.option, + value -> value + )); + + private final String option; + private final Supplier compressionEngineSupplier; + + CompressionOption(final String option, final Supplier compressionEngineSupplier) { + this.option = option.toLowerCase(); + this.compressionEngineSupplier = compressionEngineSupplier; + } + + public CompressionEngine getCompressionEngine() { + return compressionEngineSupplier.get(); + } + + @JsonCreator + public static CompressionOption fromOptionValue(final String option) { + return OPTIONS_MAP.get(option.toLowerCase()); + } +} diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/GZipCompressionEngine.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/GZipCompressionEngine.java new file mode 100644 index 0000000000..f59956a8ed --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/GZipCompressionEngine.java @@ -0,0 +1,18 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3.compression; + +import org.apache.commons.compress.compressors.gzip.GzipCompressorOutputStream; + +import java.io.IOException; +import java.io.OutputStream; + +public class GZipCompressionEngine implements CompressionEngine { + @Override + public OutputStream createOutputStream(final OutputStream outputStream) throws IOException { + return new GzipCompressorOutputStream(outputStream); + } +} diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/NoneCompressionEngine.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/NoneCompressionEngine.java new file mode 100644 index 0000000000..9c852b4f85 --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/NoneCompressionEngine.java @@ -0,0 +1,15 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3.compression; + +import java.io.OutputStream; + +public class NoneCompressionEngine implements CompressionEngine { + @Override + public OutputStream createOutputStream(final OutputStream outputStream) { + return outputStream; + } +} diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkTest.java index 61d27cecae..75ae2dde1c 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkTest.java @@ -18,6 +18,7 @@ import org.opensearch.dataprepper.model.sink.SinkContext; import org.opensearch.dataprepper.model.types.ByteCount; import org.opensearch.dataprepper.plugins.sink.s3.accumulator.BufferTypeOptions; +import org.opensearch.dataprepper.plugins.sink.s3.compression.CompressionOption; import org.opensearch.dataprepper.plugins.sink.s3.configuration.AwsAuthenticationOptions; import org.opensearch.dataprepper.plugins.sink.s3.configuration.ObjectKeyOptions; import org.opensearch.dataprepper.plugins.sink.s3.configuration.ThresholdOptions; @@ -70,6 +71,7 @@ void setUp() { when(s3SinkConfig.getThresholdOptions().getEventCount()).thenReturn(MAX_EVENTS); when(s3SinkConfig.getThresholdOptions().getMaximumSize()).thenReturn(ByteCount.parse(MAXIMUM_SIZE)); when(s3SinkConfig.getThresholdOptions().getEventCollectTimeOut()).thenReturn(Duration.ofSeconds(MAX_RETRIES)); + when(s3SinkConfig.getCompression()).thenReturn(CompressionOption.NONE); when(objectKeyOptions.getNamePattern()).thenReturn(OBJECT_KEY_NAME_PATTERN); when(s3SinkConfig.getAwsAuthenticationOptions()).thenReturn(awsAuthenticationOptions); when(awsAuthenticationOptions.getAwsRegion()).thenReturn(Region.of(S3_REGION)); diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBufferFactoryTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBufferFactoryTest.java new file mode 100644 index 0000000000..a27798f3df --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBufferFactoryTest.java @@ -0,0 +1,73 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3.accumulator; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Nested; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.plugins.sink.s3.compression.CompressionEngine; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.CoreMatchers.not; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class CompressionBufferFactoryTest { + @Mock + private BufferFactory innerBufferFactory; + + @Mock + private CompressionEngine compressionEngine; + + private CompressionBufferFactory createObjectUnderTest() { + return new CompressionBufferFactory(innerBufferFactory, compressionEngine); + } + + @Test + void constructor_throws_if_inner_BufferFactory_is_null() { + innerBufferFactory = null; + + assertThrows(NullPointerException.class, this::createObjectUnderTest); + } + + @Test + void constructor_throws_if_CompressionEngine_is_null() { + compressionEngine = null; + + assertThrows(NullPointerException.class, this::createObjectUnderTest); + } + + @Nested + class WithBuffer { + @Mock + private Buffer innerBuffer; + + @BeforeEach + void setUp() { + when(innerBufferFactory.getBuffer()).thenReturn(innerBuffer); + } + + @Test + void getBuffer_returns_CompressionBuffer() { + final Buffer buffer = createObjectUnderTest().getBuffer(); + assertThat(buffer, instanceOf(CompressionBuffer.class)); + } + + @Test + void getBuffer_returns_new_on_each_call() { + final CompressionBufferFactory objectUnderTest = createObjectUnderTest(); + final Buffer firstBuffer = objectUnderTest.getBuffer(); + + assertThat(objectUnderTest.getBuffer(), not(equalTo(firstBuffer))); + } + } +} \ No newline at end of file diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBufferTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBufferTest.java new file mode 100644 index 0000000000..3a7055414b --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CompressionBufferTest.java @@ -0,0 +1,140 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3.accumulator; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.plugins.sink.s3.compression.CompressionEngine; +import software.amazon.awssdk.services.s3.S3Client; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.Random; +import java.util.UUID; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.sameInstance; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class CompressionBufferTest { + @Mock + private Buffer innerBuffer; + + @Mock + private CompressionEngine compressionEngine; + + private Random random; + + @BeforeEach + void setUp() { + random = new Random(); + } + + private CompressionBuffer createObjectUnderTest() { + return new CompressionBuffer(innerBuffer, compressionEngine); + } + + @Test + void constructor_throws_if_innerBuffer_is_null() { + innerBuffer = null; + assertThrows(NullPointerException.class, this::createObjectUnderTest); + } + + @Test + void constructor_throws_if_compressionEngine_is_null() { + compressionEngine = null; + assertThrows(NullPointerException.class, this::createObjectUnderTest); + } + + @Test + void getSize_returns_inner_getSize() { + final long size = random.nextInt(10_000) + 1_000; + + final CompressionBuffer objectUnderTest = createObjectUnderTest(); + when(innerBuffer.getSize()).thenReturn(size); + + assertThat(objectUnderTest.getSize(), equalTo(size)); + } + + @Test + void getEventCount_returns_inner_getEventCount() { + final int eventCount = random.nextInt(10_000) + 1_000; + + final CompressionBuffer objectUnderTest = createObjectUnderTest(); + when(innerBuffer.getEventCount()).thenReturn(eventCount); + + assertThat(objectUnderTest.getEventCount(), equalTo(eventCount)); + } + + @Test + void getDuration_returns_inner_getDuration() { + final long duration = random.nextInt(10_000) + 1_000; + + final CompressionBuffer objectUnderTest = createObjectUnderTest(); + when(innerBuffer.getDuration()).thenReturn(duration); + + assertThat(objectUnderTest.getDuration(), equalTo(duration)); + } + + @Test + void flushToS3_calls_inner_flushToS3() { + final S3Client s3Client = mock(S3Client.class); + final String bucket = UUID.randomUUID().toString(); + final String key = UUID.randomUUID().toString(); + + createObjectUnderTest().flushToS3(s3Client, bucket, key); + + verify(innerBuffer).flushToS3(s3Client, bucket, key); + } + + @Test + void getOutputStream_returns_outputStream_via_CompressionEngine() throws IOException { + final OutputStream innerBufferOutputStream = mock(OutputStream.class); + when(innerBuffer.getOutputStream()).thenReturn(innerBufferOutputStream); + final OutputStream compressionEngineOutputStream = mock(OutputStream.class); + when(compressionEngine.createOutputStream(innerBufferOutputStream)).thenReturn(compressionEngineOutputStream); + + final OutputStream actualOutputStream = createObjectUnderTest().getOutputStream(); + + + assertThat(actualOutputStream, sameInstance(compressionEngineOutputStream)); + } + + @Test + void getOutputStream_wraps_OutputStream_only_once() throws IOException { + final OutputStream innerBufferOutputStream = mock(OutputStream.class); + when(innerBuffer.getOutputStream()).thenReturn(innerBufferOutputStream); + final OutputStream compressionEngineOutputStream = mock(OutputStream.class); + when(compressionEngine.createOutputStream(innerBufferOutputStream)).thenReturn(compressionEngineOutputStream); + + final CompressionBuffer objectUnderTest = createObjectUnderTest(); + final OutputStream outputStream = objectUnderTest.getOutputStream(); + assertThat(objectUnderTest.getOutputStream(), sameInstance(outputStream)); + assertThat(objectUnderTest.getOutputStream(), sameInstance(outputStream)); + assertThat(objectUnderTest.getOutputStream(), sameInstance(outputStream)); + + verify(compressionEngine, times(1)).createOutputStream(any()); + } + + @Test + void setEventCount_calls_inner_setEventCount() { + final int eventCount = random.nextInt(10_000) + 1_000; + + createObjectUnderTest().setEventCount(eventCount); + + verify(innerBuffer).setEventCount(eventCount); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/compression/GZipCompressionEngineTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/compression/GZipCompressionEngineTest.java new file mode 100644 index 0000000000..a92930e958 --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/compression/GZipCompressionEngineTest.java @@ -0,0 +1,60 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3.compression; + +import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream; +import org.apache.commons.compress.compressors.gzip.GzipCompressorOutputStream; +import org.junit.jupiter.api.Test; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.charset.StandardCharsets; +import java.util.UUID; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; + +class GZipCompressionEngineTest { + + private GZipCompressionEngine createObjectUnderTest() { + return new GZipCompressionEngine(); + } + + @Test + void createOutputStream_should_return_GzipCompressorOutputStream() throws IOException { + final OutputStream innerOutputStream = mock(OutputStream.class); + final OutputStream outputStream = createObjectUnderTest().createOutputStream(innerOutputStream); + + assertThat(outputStream, instanceOf(GzipCompressorOutputStream.class)); + } + + @Test + void createOutputStream_should_write_compressed_data() throws IOException { + final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + + final OutputStream outputStream = createObjectUnderTest().createOutputStream(byteArrayOutputStream); + + final byte[] inputBytes = UUID.randomUUID().toString().getBytes(StandardCharsets.UTF_8); + + outputStream.write(inputBytes); + outputStream.close(); + + final byte[] writtenBytes = byteArrayOutputStream.toByteArray(); + + assertTrue(GzipCompressorInputStream.matches(writtenBytes, 2)); + + final ByteArrayInputStream verificationInputStream = new ByteArrayInputStream(writtenBytes); + + final GzipCompressorInputStream uncompressingInputStream = new GzipCompressorInputStream(verificationInputStream); + final byte[] uncompressedBytes = uncompressingInputStream.readAllBytes(); + assertThat(uncompressedBytes, equalTo(inputBytes)); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/compression/NoneCompressionEngineTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/compression/NoneCompressionEngineTest.java new file mode 100644 index 0000000000..17c581b0c7 --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/compression/NoneCompressionEngineTest.java @@ -0,0 +1,38 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3.compression; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.OutputStream; + +import static org.hamcrest.CoreMatchers.sameInstance; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verifyNoInteractions; + +class NoneCompressionEngineTest { + + private OutputStream innerOutputStream; + + @BeforeEach + void setUp() { + innerOutputStream = mock(OutputStream.class); + } + + private NoneCompressionEngine createObjectUnderTest() { + return new NoneCompressionEngine(); + } + + @Test + void createOutputStream_returns_innerOutputStream() { + OutputStream outputStream = createObjectUnderTest().createOutputStream(innerOutputStream); + + assertThat(outputStream, sameInstance(innerOutputStream)); + verifyNoInteractions(innerOutputStream); + } +} \ No newline at end of file From 675c2fa73cfa1898579eb191fa2b54134d6388b4 Mon Sep 17 00:00:00 2001 From: Jonah Calvo Date: Fri, 11 Aug 2023 11:14:43 -0500 Subject: [PATCH 11/28] Add default value to cardinality keys (#3144) Signed-off-by: Jonah Calvo --- .../anomalydetector/AnomalyDetectorProcessorConfig.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorConfig.java b/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorConfig.java index c92fdb9000..7e796e660a 100644 --- a/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorConfig.java +++ b/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorConfig.java @@ -10,6 +10,7 @@ import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; +import java.util.Collections; import java.util.List; public class AnomalyDetectorProcessorConfig { @@ -22,7 +23,7 @@ public class AnomalyDetectorProcessorConfig { private List keys; @JsonProperty("identification_keys") - private List identificationKeys; + private List identificationKeys = Collections.emptyList(); @JsonProperty("verbose") private Boolean verbose = false; From a5c4fe2255eeef4939e26d3fb21f286ab6c86462 Mon Sep 17 00:00:00 2001 From: kkondaka <41027584+kkondaka@users.noreply.github.com> Date: Fri, 11 Aug 2023 10:28:08 -0700 Subject: [PATCH 12/28] Glue registry fixes. Fixed a bug in getMSKBootstrapServers (#3142) * Glue registry fixes. Fixed a bug in getMSKBootstrapServers Signed-off-by: Krishna Kondaka * Changed default auto commit reset to earliest Signed-off-by: Krishna Kondaka --------- Signed-off-by: Krishna Kondaka Co-authored-by: Krishna Kondaka --- .../kafka/configuration/TopicConfig.java | 2 +- .../consumer/KafkaSourceCustomConsumer.java | 2 +- .../plugins/kafka/source/KafkaSource.java | 21 ++++++------- .../util/KafkaSourceSecurityConfigurer.java | 30 +++++++++++++++++-- 4 files changed, 38 insertions(+), 17 deletions(-) diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfig.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfig.java index d613db3cf5..e8244098d6 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfig.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfig.java @@ -22,7 +22,7 @@ public class TopicConfig { static final Duration DEFAULT_COMMIT_INTERVAL = Duration.ofSeconds(5); static final Duration DEFAULT_SESSION_TIMEOUT = Duration.ofSeconds(45); static final int DEFAULT_MAX_RETRY_ATTEMPT = Integer.MAX_VALUE; - static final String DEFAULT_AUTO_OFFSET_RESET = "latest"; + static final String DEFAULT_AUTO_OFFSET_RESET = "earliest"; static final Duration DEFAULT_THREAD_WAITING_TIME = Duration.ofSeconds(5); static final Duration DEFAULT_MAX_RECORD_FETCH_TIME = Duration.ofSeconds(4); static final Duration DEFAULT_BUFFER_TIMEOUT = Duration.ofSeconds(5); 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 d083b4e98b..805cfb6497 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 @@ -162,7 +162,7 @@ public void consumeRecords() throws Exception { Thread.sleep(10000); } catch (RecordDeserializationException e) { LOG.warn("Deserialization error - topic {} partition {} offset {}, seeking past the error record", - e.topicPartition().topic(), e.topicPartition().partition(), e.offset()); + e.topicPartition().topic(), e.topicPartition().partition(), e.offset(), e); topicMetrics.getNumberOfDeserializationErrors().increment(); consumer.seek(e.topicPartition(), e.offset()+1); } 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 92cf2527f8..b0c752c200 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 @@ -48,8 +48,6 @@ import org.opensearch.dataprepper.plugins.kafka.util.KafkaTopicMetrics; import com.amazonaws.services.schemaregistry.deserializers.GlueSchemaRegistryKafkaDeserializer; -import com.amazonaws.services.schemaregistry.utils.AWSSchemaRegistryConstants; -import com.amazonaws.services.schemaregistry.utils.AvroRecordType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -100,6 +98,8 @@ public class KafkaSource implements Source> { private static final String SCHEMA_TYPE = "schemaType"; private final AcknowledgementSetManager acknowledgementSetManager; private static CachedSchemaRegistryClient schemaRegistryClient; + private GlueSchemaRegistryKafkaDeserializer glueDeserializer; + private StringDeserializer stringDeserializer; @DataPrepperPluginConstructor public KafkaSource(final KafkaSourceConfig sourceConfig, @@ -110,13 +110,14 @@ public KafkaSource(final KafkaSourceConfig sourceConfig, this.pluginMetrics = pluginMetrics; this.acknowledgementSetManager = acknowledgementSetManager; this.pipelineName = pipelineDescription.getPipelineName(); + this.stringDeserializer = new StringDeserializer(); shutdownInProgress = new AtomicBoolean(false); } @Override public void start(Buffer> buffer) { Properties authProperties = new Properties(); - KafkaSourceSecurityConfigurer.setAuthProperties(authProperties, sourceConfig, LOG); + glueDeserializer = KafkaSourceSecurityConfigurer.setAuthProperties(authProperties, sourceConfig, LOG); sourceConfig.getTopics().forEach(topic -> { consumerGroupID = topic.getGroupId(); KafkaTopicMetrics topicMetrics = new KafkaTopicMetrics(topic.getName(), pluginMetrics); @@ -135,7 +136,11 @@ public void start(Buffer> buffer) { break; case PLAINTEXT: default: - kafkaConsumer = new KafkaConsumer(consumerProperties); + if (sourceConfig.getSchemaConfig().getType() == SchemaRegistryType.AWS_GLUE) { + kafkaConsumer = new KafkaConsumer(consumerProperties, stringDeserializer, glueDeserializer); + } else { + kafkaConsumer = new KafkaConsumer(consumerProperties); + } break; } consumer = new KafkaSourceCustomConsumer(kafkaConsumer, shutdownInProgress, buffer, sourceConfig, topic, schemaType, acknowledgementSetManager, topicMetrics); @@ -296,7 +301,6 @@ private void setSchemaRegistryProperties(Properties properties, TopicConfig topi } if (schemaConfig.getType() == SchemaRegistryType.AWS_GLUE) { - setPropertiesForGlueSchemaRegistry(properties); return; } @@ -309,13 +313,6 @@ private void setSchemaRegistryProperties(Properties properties, TopicConfig topi } } - private void setPropertiesForGlueSchemaRegistry(Properties properties) { - properties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); - properties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, GlueSchemaRegistryKafkaDeserializer.class.getName()); - properties.put(AWSSchemaRegistryConstants.AWS_REGION, sourceConfig.getAwsConfig().getRegion()); - properties.put(AWSSchemaRegistryConstants.AVRO_RECORD_TYPE, AvroRecordType.GENERIC_RECORD.getName()); - } - private void setPropertiesForPlaintextAndJsonWithoutSchemaRegistry(Properties properties, final TopicConfig topicConfig) { MessageFormat dataFormat = topicConfig.getSerdeFormat(); schemaType = dataFormat.toString(); diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaSourceSecurityConfigurer.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaSourceSecurityConfigurer.java index 4a6aaf30da..e9e612266f 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaSourceSecurityConfigurer.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaSourceSecurityConfigurer.java @@ -11,6 +11,7 @@ import org.opensearch.dataprepper.plugins.kafka.configuration.OAuthConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.EncryptionType; import org.opensearch.dataprepper.plugins.kafka.configuration.PlainTextAuthConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.SchemaRegistryType; import org.apache.kafka.clients.consumer.ConsumerConfig; import software.amazon.awssdk.services.kafka.KafkaClient; @@ -25,9 +26,17 @@ import software.amazon.awssdk.services.sts.StsClient; import software.amazon.awssdk.regions.Region; +import com.amazonaws.services.schemaregistry.deserializers.GlueSchemaRegistryKafkaDeserializer; +import com.amazonaws.services.schemaregistry.utils.AWSSchemaRegistryConstants; +import com.amazonaws.services.schemaregistry.utils.AvroRecordType; +import com.amazonaws.services.schemaregistry.deserializers.GlueSchemaRegistryKafkaDeserializer; +import software.amazon.awssdk.services.glue.model.Compatibility; + import org.slf4j.Logger; import java.util.Base64; +import java.util.HashMap; +import java.util.Map; import java.util.Objects; import java.util.Properties; import java.util.UUID; @@ -73,6 +82,8 @@ public class KafkaSourceSecurityConfigurer { private static final int MAX_KAFKA_CLIENT_RETRIES = 360; // for one hour every 10 seconds + private static AwsCredentialsProvider credentialsProvider; + /*public static void setSaslPlainTextProperties(final KafkaSourceConfig kafkaSourConfig, final Properties properties) { @@ -173,7 +184,6 @@ public static void setAwsIamAuthProperties(Properties properties, final AwsIamAu } public static String getBootStrapServersForMsk(final AwsIamAuthConfig awsIamAuthConfig, final AwsConfig awsConfig, final Logger LOG) { - AwsCredentialsProvider credentialsProvider = DefaultCredentialsProvider.create(); if (awsIamAuthConfig == AwsIamAuthConfig.ROLE) { String sessionName = "data-prepper-kafka-session" + UUID.randomUUID(); StsClient stsClient = StsClient.builder() @@ -216,10 +226,11 @@ public static String getBootStrapServersForMsk(final AwsIamAuthConfig awsIamAuth try { Thread.sleep(10000); } catch (InterruptedException exp) {} + retryable = true; } catch (Exception e) { throw new RuntimeException("Failed to get bootstrap server information from MSK.", e); } - } while (numRetries++ < MAX_KAFKA_CLIENT_RETRIES); + } while (retryable && numRetries++ < MAX_KAFKA_CLIENT_RETRIES); if (Objects.isNull(result)) { throw new RuntimeException("Failed to get bootstrap server information from MSK after trying multiple times with retryable exceptions."); } @@ -234,11 +245,14 @@ public static String getBootStrapServersForMsk(final AwsIamAuthConfig awsIamAuth } } - public static void setAuthProperties(Properties properties, final KafkaSourceConfig sourceConfig, final Logger LOG) { + public static GlueSchemaRegistryKafkaDeserializer setAuthProperties(Properties properties, final KafkaSourceConfig sourceConfig, final Logger LOG) { final AwsConfig awsConfig = sourceConfig.getAwsConfig(); final AuthConfig authConfig = sourceConfig.getAuthConfig(); final KafkaSourceConfig.EncryptionConfig encryptionConfig = sourceConfig.getEncryptionConfig(); final EncryptionType encryptionType = encryptionConfig.getType(); + GlueSchemaRegistryKafkaDeserializer glueDeserializer = null; + + credentialsProvider = DefaultCredentialsProvider.create(); String bootstrapServers = sourceConfig.getBootStrapServers(); AwsIamAuthConfig awsIamAuthConfig = null; @@ -269,6 +283,15 @@ public static void setAuthProperties(Properties properties, final KafkaSourceCon properties.put("ssl.engine.factory.class", InsecureSslEngineFactory.class); } } + if (sourceConfig.getSchemaConfig().getType() == SchemaRegistryType.AWS_GLUE) { + Map configs = new HashMap(); + configs.put(AWSSchemaRegistryConstants.AWS_REGION, sourceConfig.getAwsConfig().getRegion()); + configs.put(AWSSchemaRegistryConstants.AVRO_RECORD_TYPE, AvroRecordType.GENERIC_RECORD.getName()); + configs.put(AWSSchemaRegistryConstants.CACHE_TIME_TO_LIVE_MILLIS, "86400000"); + configs.put(AWSSchemaRegistryConstants.CACHE_SIZE, "10"); + configs.put(AWSSchemaRegistryConstants.COMPATIBILITY_SETTING, Compatibility.FULL); + glueDeserializer = new GlueSchemaRegistryKafkaDeserializer(credentialsProvider, configs); + } if (Objects.isNull(authConfig) || Objects.isNull(authConfig.getSaslAuthConfig())) { if (encryptionType == EncryptionType.SSL) { properties.put(SECURITY_PROTOCOL, "SSL"); @@ -278,6 +301,7 @@ public static void setAuthProperties(Properties properties, final KafkaSourceCon throw new RuntimeException("Bootstrap servers are not specified"); } properties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); + return glueDeserializer; } } From 28fdf903b791ec7365e5783022b859a7910040eb Mon Sep 17 00:00:00 2001 From: kkondaka <41027584+kkondaka@users.noreply.github.com> Date: Fri, 11 Aug 2023 18:35:17 -0700 Subject: [PATCH 13/28] Fix Null pointer exception when schema registry not specified (#3147) * Fix Null pointer exception when schema registry not specified Signed-off-by: Krishna Kondaka * Fix failing test cases Signed-off-by: Krishna Kondaka --------- Signed-off-by: Krishna Kondaka Co-authored-by: Krishna Kondaka --- .../plugins/kafka/source/KafkaSource.java | 5 ++-- .../util/KafkaSourceSecurityConfigurer.java | 29 ++++++++++++------- .../plugins/kafka/source/KafkaSourceTest.java | 27 +++++++++++++++++ 3 files changed, 48 insertions(+), 13 deletions(-) diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSource.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSource.java index b0c752c200..a388d3ee6e 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 @@ -117,7 +117,7 @@ public KafkaSource(final KafkaSourceConfig sourceConfig, @Override public void start(Buffer> buffer) { Properties authProperties = new Properties(); - glueDeserializer = KafkaSourceSecurityConfigurer.setAuthProperties(authProperties, sourceConfig, LOG); + KafkaSourceSecurityConfigurer.setAuthProperties(authProperties, sourceConfig, LOG); sourceConfig.getTopics().forEach(topic -> { consumerGroupID = topic.getGroupId(); KafkaTopicMetrics topicMetrics = new KafkaTopicMetrics(topic.getName(), pluginMetrics); @@ -136,7 +136,8 @@ public void start(Buffer> buffer) { break; case PLAINTEXT: default: - if (sourceConfig.getSchemaConfig().getType() == SchemaRegistryType.AWS_GLUE) { + glueDeserializer = KafkaSourceSecurityConfigurer.getGlueSerializer(sourceConfig); + if (Objects.nonNull(glueDeserializer)) { kafkaConsumer = new KafkaConsumer(consumerProperties, stringDeserializer, glueDeserializer); } else { kafkaConsumer = new KafkaConsumer(consumerProperties); diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaSourceSecurityConfigurer.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaSourceSecurityConfigurer.java index e9e612266f..77fcd6e2fc 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaSourceSecurityConfigurer.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaSourceSecurityConfigurer.java @@ -8,6 +8,7 @@ import org.opensearch.dataprepper.plugins.kafka.configuration.AwsConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.AwsIamAuthConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.SchemaConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.OAuthConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.EncryptionType; import org.opensearch.dataprepper.plugins.kafka.configuration.PlainTextAuthConfig; @@ -83,6 +84,7 @@ public class KafkaSourceSecurityConfigurer { private static final int MAX_KAFKA_CLIENT_RETRIES = 360; // for one hour every 10 seconds private static AwsCredentialsProvider credentialsProvider; + private static GlueSchemaRegistryKafkaDeserializer glueDeserializer; /*public static void setSaslPlainTextProperties(final KafkaSourceConfig kafkaSourConfig, @@ -245,12 +247,11 @@ public static String getBootStrapServersForMsk(final AwsIamAuthConfig awsIamAuth } } - public static GlueSchemaRegistryKafkaDeserializer setAuthProperties(Properties properties, final KafkaSourceConfig sourceConfig, final Logger LOG) { + public static void setAuthProperties(Properties properties, final KafkaSourceConfig sourceConfig, final Logger LOG) { final AwsConfig awsConfig = sourceConfig.getAwsConfig(); final AuthConfig authConfig = sourceConfig.getAuthConfig(); final KafkaSourceConfig.EncryptionConfig encryptionConfig = sourceConfig.getEncryptionConfig(); final EncryptionType encryptionType = encryptionConfig.getType(); - GlueSchemaRegistryKafkaDeserializer glueDeserializer = null; credentialsProvider = DefaultCredentialsProvider.create(); @@ -283,15 +284,6 @@ public static GlueSchemaRegistryKafkaDeserializer setAuthProperties(Properties p properties.put("ssl.engine.factory.class", InsecureSslEngineFactory.class); } } - if (sourceConfig.getSchemaConfig().getType() == SchemaRegistryType.AWS_GLUE) { - Map configs = new HashMap(); - configs.put(AWSSchemaRegistryConstants.AWS_REGION, sourceConfig.getAwsConfig().getRegion()); - configs.put(AWSSchemaRegistryConstants.AVRO_RECORD_TYPE, AvroRecordType.GENERIC_RECORD.getName()); - configs.put(AWSSchemaRegistryConstants.CACHE_TIME_TO_LIVE_MILLIS, "86400000"); - configs.put(AWSSchemaRegistryConstants.CACHE_SIZE, "10"); - configs.put(AWSSchemaRegistryConstants.COMPATIBILITY_SETTING, Compatibility.FULL); - glueDeserializer = new GlueSchemaRegistryKafkaDeserializer(credentialsProvider, configs); - } if (Objects.isNull(authConfig) || Objects.isNull(authConfig.getSaslAuthConfig())) { if (encryptionType == EncryptionType.SSL) { properties.put(SECURITY_PROTOCOL, "SSL"); @@ -301,7 +293,22 @@ public static GlueSchemaRegistryKafkaDeserializer setAuthProperties(Properties p throw new RuntimeException("Bootstrap servers are not specified"); } properties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); + } + + public static GlueSchemaRegistryKafkaDeserializer getGlueSerializer(final KafkaSourceConfig sourceConfig) { + SchemaConfig schemaConfig = sourceConfig.getSchemaConfig(); + if (Objects.isNull(schemaConfig) || schemaConfig.getType() != SchemaRegistryType.AWS_GLUE) { + return null; + } + Map configs = new HashMap(); + configs.put(AWSSchemaRegistryConstants.AWS_REGION, sourceConfig.getAwsConfig().getRegion()); + configs.put(AWSSchemaRegistryConstants.AVRO_RECORD_TYPE, AvroRecordType.GENERIC_RECORD.getName()); + configs.put(AWSSchemaRegistryConstants.CACHE_TIME_TO_LIVE_MILLIS, "86400000"); + configs.put(AWSSchemaRegistryConstants.CACHE_SIZE, "10"); + configs.put(AWSSchemaRegistryConstants.COMPATIBILITY_SETTING, Compatibility.FULL); + glueDeserializer = new GlueSchemaRegistryKafkaDeserializer(credentialsProvider, configs); return glueDeserializer; } + } 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 be868b3e6f..05843ed1a9 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 @@ -13,6 +13,8 @@ import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaSourceConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.SchemaConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; +import org.opensearch.dataprepper.plugins.kafka.configuration.EncryptionType; +import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.junit.jupiter.api.Assertions; @@ -25,8 +27,10 @@ import org.mockito.quality.Strictness; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.Arrays; +import java.util.Objects; import java.time.Duration; @ExtendWith(MockitoExtension.class) @@ -37,6 +41,9 @@ class KafkaSourceTest { @Mock private KafkaSourceConfig sourceConfig; + @Mock + private KafkaSourceConfig.EncryptionConfig encryptionConfig; + @Mock private PluginMetrics pluginMetrics; @@ -64,6 +71,7 @@ public KafkaSource createObjectUnderTest() { @BeforeEach void setUp() throws Exception { sourceConfig = mock(KafkaSourceConfig.class); + encryptionConfig = mock(KafkaSourceConfig.EncryptionConfig.class); pipelineDescription = mock(PipelineDescription.class); pluginMetrics = mock(PluginMetrics.class); acknowledgementSetManager = mock(AcknowledgementSetManager.class); @@ -79,12 +87,21 @@ void setUp() throws Exception { when(topic2.getConsumerMaxPollRecords()).thenReturn(1); when(topic1.getGroupId()).thenReturn(TEST_GROUP_ID); when(topic2.getGroupId()).thenReturn(TEST_GROUP_ID); + when(topic1.getMaxPollInterval()).thenReturn(Duration.ofSeconds(5)); + when(topic2.getMaxPollInterval()).thenReturn(Duration.ofSeconds(5)); + when(topic1.getHeartBeatInterval()).thenReturn(Duration.ofSeconds(5)); + when(topic2.getHeartBeatInterval()).thenReturn(Duration.ofSeconds(5)); when(topic1.getAutoCommit()).thenReturn(false); + when(topic1.getSerdeFormat()).thenReturn(MessageFormat.PLAINTEXT); + when(topic2.getSerdeFormat()).thenReturn(MessageFormat.PLAINTEXT); when(topic2.getAutoCommit()).thenReturn(false); when(topic1.getThreadWaitingTime()).thenReturn(Duration.ofSeconds(10)); when(topic2.getThreadWaitingTime()).thenReturn(Duration.ofSeconds(10)); when(sourceConfig.getBootStrapServers()).thenReturn("http://localhost:1234"); when(sourceConfig.getTopics()).thenReturn(Arrays.asList(topic1, topic2)); + when(sourceConfig.getSchemaConfig()).thenReturn(null); + when(sourceConfig.getEncryptionConfig()).thenReturn(encryptionConfig); + when(encryptionConfig.getType()).thenReturn(EncryptionType.NONE); } /* @Test @@ -108,4 +125,14 @@ void test_kafkaSource_start_execution_exception() { kafkaSource = createObjectUnderTest(); Assertions.assertThrows(Exception.class, () -> kafkaSource.start(buffer)); } + + @Test + void test_kafkaSource_basicFunctionality() { + when(topic1.getSessionTimeOut()).thenReturn(Duration.ofSeconds(15)); + when(topic2.getSessionTimeOut()).thenReturn(Duration.ofSeconds(15)); + kafkaSource = createObjectUnderTest(); + assertTrue(Objects.nonNull(kafkaSource)); + kafkaSource.start(buffer); + assertTrue(Objects.nonNull(kafkaSource.getConsumer())); + } } From 754839062318912b5e74b46cab1767af5e4e4c4c Mon Sep 17 00:00:00 2001 From: Jonah Calvo Date: Mon, 14 Aug 2023 13:10:21 -0500 Subject: [PATCH 14/28] Fix flakey test caused by RCF variance. Update metric for RCF Instances (#3145) * Fix flakey test caused by RCF variance Signed-off-by: Jonah Calvo * Change metric name and type. Update test readability Signed-off-by: Jonah Calvo * Fix unit test to account for metric change Signed-off-by: Jonah Calvo --------- Signed-off-by: Jonah Calvo --- .../anomalydetector/AnomalyDetectorProcessor.java | 11 ++++++----- .../AnomalyDetectorProcessorTests.java | 5 +++-- .../modes/RandomCutForestModeTests.java | 6 +++++- 3 files changed, 14 insertions(+), 8 deletions(-) diff --git a/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessor.java b/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessor.java index 57872c7ecd..0eb33f979f 100644 --- a/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessor.java +++ b/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessor.java @@ -5,7 +5,7 @@ package org.opensearch.dataprepper.plugins.processor.anomalydetector; -import io.micrometer.core.instrument.Counter; + import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; @@ -23,19 +23,20 @@ import java.util.LinkedList; import java.util.List; import java.util.Objects; +import java.util.concurrent.atomic.AtomicInteger; @DataPrepperPlugin(name = "anomaly_detector", pluginType = Processor.class, pluginConfigurationType = AnomalyDetectorProcessorConfig.class) public class AnomalyDetectorProcessor extends AbstractProcessor, Record> { public static final String DEVIATION_KEY = "deviation_from_expected"; public static final String GRADE_KEY = "grade"; - static final String NUMBER_RCF_INSTANCES = "numberRCFInstances"; + static final String NUMBER_RCF_INSTANCES = "RCFInstances"; private final Boolean verbose; private final IdentificationKeysHasher identificationKeysHasher; - private final Counter numberRCFInstances; private final List keys; private final PluginFactory pluginFactory; private final HashMap forestMap; + private final AtomicInteger cardinality; private final AnomalyDetectorProcessorConfig anomalyDetectorProcessorConfig; @DataPrepperPluginConstructor @@ -44,9 +45,9 @@ public AnomalyDetectorProcessor(final AnomalyDetectorProcessorConfig anomalyDete this.identificationKeysHasher = new IdentificationKeysHasher(anomalyDetectorProcessorConfig.getIdentificationKeys()); this.anomalyDetectorProcessorConfig = anomalyDetectorProcessorConfig; this.pluginFactory = pluginFactory; - this.numberRCFInstances = pluginMetrics.counter(NUMBER_RCF_INSTANCES); this.keys = anomalyDetectorProcessorConfig.getKeys(); this.verbose = anomalyDetectorProcessorConfig.getVerbose(); + this.cardinality = pluginMetrics.gauge(NUMBER_RCF_INSTANCES, new AtomicInteger()); forestMap = new HashMap<>(); } @@ -71,10 +72,10 @@ public Collection> doExecute(Collection> records) { forest = loadAnomalyDetectorMode(pluginFactory); forest.initialize(keys, verbose); forestMap.put(identificationKeysMap.hashCode(), forest); - this.numberRCFInstances.increment(); } recordsOut.addAll(forestMap.get(identificationKeysMap.hashCode()).handleEvents(List.of(record))); } + cardinality.set(forestMap.size()); return recordsOut; } diff --git a/data-prepper-plugins/anomaly-detector-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorTests.java b/data-prepper-plugins/anomaly-detector-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorTests.java index 65c39518d2..302a692dd7 100644 --- a/data-prepper-plugins/anomaly-detector-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorTests.java +++ b/data-prepper-plugins/anomaly-detector-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorTests.java @@ -31,6 +31,7 @@ import java.util.Map; import java.util.UUID; import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicInteger; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; @@ -48,7 +49,7 @@ public class AnomalyDetectorProcessorTests { @Mock private PluginMetrics pluginMetrics; @Mock - private Counter numberRCFInstances; + private AtomicInteger numberRCFInstances; @Mock private Counter recordsIn; @@ -81,7 +82,7 @@ void setUp() { when(pluginFactory.loadPlugin(eq(AnomalyDetectorMode.class), any(PluginSetting.class))) .thenAnswer(invocation -> new RandomCutForestMode(randomCutForestModeConfig)); - when(pluginMetrics.counter(AnomalyDetectorProcessor.NUMBER_RCF_INSTANCES)).thenReturn(numberRCFInstances); + when(pluginMetrics.gauge(eq(AnomalyDetectorProcessor.NUMBER_RCF_INSTANCES), any())).thenReturn(numberRCFInstances); when(pluginMetrics.counter(MetricNames.RECORDS_IN)).thenReturn(recordsIn); when(pluginMetrics.counter(MetricNames.RECORDS_OUT)).thenReturn(recordsOut); when(pluginMetrics.timer(MetricNames.TIME_ELAPSED)).thenReturn(timeElapsed); diff --git a/data-prepper-plugins/anomaly-detector-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/modes/RandomCutForestModeTests.java b/data-prepper-plugins/anomaly-detector-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/modes/RandomCutForestModeTests.java index c52788433f..818ab14d7b 100644 --- a/data-prepper-plugins/anomaly-detector-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/modes/RandomCutForestModeTests.java +++ b/data-prepper-plugins/anomaly-detector-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/modes/RandomCutForestModeTests.java @@ -21,6 +21,7 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.both; import static org.hamcrest.Matchers.greaterThan; import org.junit.jupiter.api.Test; @@ -28,6 +29,8 @@ import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.junit.jupiter.MockitoExtension; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.mockito.Mockito.mock; import org.mockito.Mock; import static org.mockito.Mockito.when; @@ -205,6 +208,7 @@ void testRandomCutForestModeVerboseFalse() { } final List> anomalyRecords = randomCutForestMode.handleEvents(recordsWithAnomaly).stream().collect(toList());; - assertThat(anomalyRecords.size(), equalTo(1)); + // Due to inherent variance in the RCF algorithm, 1-3 anomalies will be detected after the level shift. + assertThat(anomalyRecords.size(), both(greaterThanOrEqualTo(1)).and(lessThanOrEqualTo(3))); } } From 0f70d524c90193c98c6576c01ce440afaee82488 Mon Sep 17 00:00:00 2001 From: David Venable Date: Mon, 14 Aug 2023 16:02:01 -0500 Subject: [PATCH 15/28] Fix flaky integration test by wrapping a list in a new list to avoid a ConcurrentModificationException. Resolves #3139. (#3152) Signed-off-by: David Venable --- .../opensearch/dataprepper/plugins/InMemorySinkAccessor.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/InMemorySinkAccessor.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/InMemorySinkAccessor.java index 77d659f816..21f359f361 100644 --- a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/InMemorySinkAccessor.java +++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/InMemorySinkAccessor.java @@ -34,7 +34,7 @@ public class InMemorySinkAccessor { public List> get(final String testingKey) { lock.lock(); try { - return recordsMap.getOrDefault(testingKey, Collections.emptyList()); + return new ArrayList<>(recordsMap.getOrDefault(testingKey, Collections.emptyList())); } finally { lock.unlock(); } @@ -49,7 +49,7 @@ public List> get(final String testingKey) { public List> getAndClear(final String testingKey) { lock.lock(); try { - final List> records = recordsMap.getOrDefault(testingKey, Collections.emptyList()); + final List> records = get(testingKey); recordsMap.remove(testingKey); From 60f69b3c565e7a10e6459d6bbe694d74ae6e6d96 Mon Sep 17 00:00:00 2001 From: David Venable Date: Mon, 14 Aug 2023 16:02:18 -0500 Subject: [PATCH 16/28] Fixes two flaky unit tests that have failed recently (#3150) Fixes two unit tests that have failed and are probably flaky. The ParseTreeEvaluatorListenerTest appears to be using negative values sometimes, which seems to be unsupported. The OTelLogsSourceTest test failed as well, but it appears this code may not always be executed because it was mixing Hamcrest and Mockito. Signed-off-by: David Venable --- .../ParseTreeEvaluatorListenerTest.java | 4 ++-- .../source/otellogs/OTelLogsSourceTest.java | 23 ++++++++++--------- 2 files changed, 14 insertions(+), 13 deletions(-) diff --git a/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/ParseTreeEvaluatorListenerTest.java b/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/ParseTreeEvaluatorListenerTest.java index 9b517d8ae0..f0e1fdb9f4 100644 --- a/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/ParseTreeEvaluatorListenerTest.java +++ b/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/ParseTreeEvaluatorListenerTest.java @@ -193,7 +193,7 @@ void testSimpleRelationalOperatorExpressionWithInValidLiteralType() { @Test void testSimpleRelationalOperatorExpressionWithJsonPointerTypeValidValue() { final String testKey = "testKey"; - final int testValue = random.nextInt(1000); + final int testValue = random.nextInt(1000) + 2; final Map data = Map.of(testKey, testValue); final Event testEvent = createTestEvent(data); final String greaterThanStatement = String.format(" /%s > %d", testKey, testValue - 1); @@ -207,7 +207,7 @@ void testSimpleRelationalOperatorExpressionWithJsonPointerTypeValidValue() { } @Test - void testSimpleRelationalOperatorExpressionWithJsonPointerTypeInValidValue() { + void testSimpleRelationalOperatorExpressionWithJsonPointerTypeInValidValueWithPositiveInteger() { final String testKey = "testKey"; final boolean testValue = true; final Map data = Map.of(testKey, testValue); diff --git a/data-prepper-plugins/otel-logs-source/src/test/java/org/opensearch/dataprepper/plugins/source/otellogs/OTelLogsSourceTest.java b/data-prepper-plugins/otel-logs-source/src/test/java/org/opensearch/dataprepper/plugins/source/otellogs/OTelLogsSourceTest.java index eee926a698..5543e7c21e 100644 --- a/data-prepper-plugins/otel-logs-source/src/test/java/org/opensearch/dataprepper/plugins/source/otellogs/OTelLogsSourceTest.java +++ b/data-prepper-plugins/otel-logs-source/src/test/java/org/opensearch/dataprepper/plugins/source/otellogs/OTelLogsSourceTest.java @@ -5,11 +5,6 @@ package org.opensearch.dataprepper.plugins.source.otellogs; -import com.linecorp.armeria.common.HttpHeaderNames; -import com.linecorp.armeria.common.grpc.GrpcStatusFunction; -import org.opensearch.dataprepper.plugins.codec.CompressionOption; -import org.opensearch.dataprepper.plugins.health.HealthGrpcService; -import org.opensearch.dataprepper.plugins.source.otellogs.certificate.CertificateProviderFactory; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.util.JsonFormat; @@ -18,11 +13,13 @@ import com.linecorp.armeria.client.WebClient; import com.linecorp.armeria.common.AggregatedHttpResponse; import com.linecorp.armeria.common.HttpData; +import com.linecorp.armeria.common.HttpHeaderNames; import com.linecorp.armeria.common.HttpMethod; import com.linecorp.armeria.common.HttpStatus; import com.linecorp.armeria.common.MediaType; import com.linecorp.armeria.common.RequestHeaders; import com.linecorp.armeria.common.SessionProtocol; +import com.linecorp.armeria.common.grpc.GrpcStatusFunction; import com.linecorp.armeria.server.Server; import com.linecorp.armeria.server.ServerBuilder; import com.linecorp.armeria.server.grpc.GrpcService; @@ -69,6 +66,9 @@ import org.opensearch.dataprepper.plugins.buffer.blockingbuffer.BlockingBuffer; import org.opensearch.dataprepper.plugins.certificate.CertificateProvider; import org.opensearch.dataprepper.plugins.certificate.model.Certificate; +import org.opensearch.dataprepper.plugins.codec.CompressionOption; +import org.opensearch.dataprepper.plugins.health.HealthGrpcService; +import org.opensearch.dataprepper.plugins.source.otellogs.certificate.CertificateProviderFactory; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -89,25 +89,22 @@ import java.util.stream.Stream; import java.util.zip.GZIPOutputStream; -import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.opensearch.dataprepper.plugins.source.otellogs.OTelLogsSourceConfig.DEFAULT_PORT; -import static org.opensearch.dataprepper.plugins.source.otellogs.OTelLogsSourceConfig.DEFAULT_REQUEST_TIMEOUT_MS; -import static org.opensearch.dataprepper.plugins.source.otellogs.OTelLogsSourceConfig.SSL; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.params.provider.Arguments.arguments; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.anyCollection; import static org.mockito.ArgumentMatchers.anyInt; -import static org.mockito.ArgumentMatchers.contains; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.ArgumentMatchers.isA; import static org.mockito.Mockito.doThrow; @@ -117,6 +114,9 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import static org.opensearch.dataprepper.plugins.source.otellogs.OTelLogsSourceConfig.DEFAULT_PORT; +import static org.opensearch.dataprepper.plugins.source.otellogs.OTelLogsSourceConfig.DEFAULT_REQUEST_TIMEOUT_MS; +import static org.opensearch.dataprepper.plugins.source.otellogs.OTelLogsSourceConfig.SSL; @ExtendWith(MockitoExtension.class) class OTelLogsSourceTest { @@ -800,8 +800,9 @@ private void assertSecureResponseWithStatusCode(final AggregatedHttpResponse res .stream() .map(Map.Entry::getKey) .map(AsciiString::toString) + .map(String::toLowerCase) .collect(Collectors.toList()); - assertThat("Response Header Keys", headerKeys, not(contains("server"))); + assertThat("Response Header Keys", headerKeys, not(hasItem("server"))); } private byte[] createGZipCompressedPayload(final String payload) throws IOException { From f11d8824fdf8fc9188cd78928273f0adbc681233 Mon Sep 17 00:00:00 2001 From: Asif Sohail Mohammed Date: Tue, 15 Aug 2023 10:39:01 -0500 Subject: [PATCH 17/28] Support s3:// prefix (#3156) Support s3:// prefix Signed-off-by: Asif Sohail Mohammed --- .../dataprepper/plugins/dlq/s3/S3DlqWriterConfig.java | 11 ++++++++--- .../plugins/dlq/s3/S3DlqWriterConfigTest.java | 9 +++++++++ .../dataprepper/plugins/sink/s3/S3SinkConfig.java | 7 ++++++- .../dataprepper/plugins/sink/s3/S3SinkConfigTest.java | 10 ++++++++++ .../source/configuration/S3ScanBucketOption.java | 9 +++++++++ .../dataprepper/plugins/source/ScanOptionsTest.java | 7 +++++-- .../source/configuration/S3ScanBucketOptionTest.java | 7 +++++++ 7 files changed, 54 insertions(+), 6 deletions(-) diff --git a/data-prepper-plugins/failures-common/src/main/java/org/opensearch/dataprepper/plugins/dlq/s3/S3DlqWriterConfig.java b/data-prepper-plugins/failures-common/src/main/java/org/opensearch/dataprepper/plugins/dlq/s3/S3DlqWriterConfig.java index 8eb8e401fd..f21b109ab5 100644 --- a/data-prepper-plugins/failures-common/src/main/java/org/opensearch/dataprepper/plugins/dlq/s3/S3DlqWriterConfig.java +++ b/data-prepper-plugins/failures-common/src/main/java/org/opensearch/dataprepper/plugins/dlq/s3/S3DlqWriterConfig.java @@ -6,7 +6,7 @@ package org.opensearch.dataprepper.plugins.dlq.s3; import com.fasterxml.jackson.annotation.JsonProperty; -import jakarta.validation.constraints.NotNull; +import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.Size; import software.amazon.awssdk.arns.Arn; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; @@ -33,9 +33,11 @@ public class S3DlqWriterConfig { private static final String DEFAULT_AWS_REGION = "us-east-1"; private static final String AWS_IAM_ROLE = "role"; private static final String AWS_IAM = "iam"; + private static final String S3_PREFIX = "s3://"; + @JsonProperty("bucket") - @NotNull - @Size(min = 3, max = 63, message = "bucket lengthy should be between 3 and 63 characters") + @NotEmpty + @Size(min = 3, max = 500, message = "bucket length should be at least 3 characters") private String bucket; @JsonProperty("key_path_prefix") @@ -55,6 +57,9 @@ public class S3DlqWriterConfig { private String stsExternalId; public String getBucket() { + if (bucket.startsWith(S3_PREFIX)) { + return bucket.substring(S3_PREFIX.length()); + } return bucket; } diff --git a/data-prepper-plugins/failures-common/src/test/java/org/opensearch/dataprepper/plugins/dlq/s3/S3DlqWriterConfigTest.java b/data-prepper-plugins/failures-common/src/test/java/org/opensearch/dataprepper/plugins/dlq/s3/S3DlqWriterConfigTest.java index d1f61ae14e..0629256277 100644 --- a/data-prepper-plugins/failures-common/src/test/java/org/opensearch/dataprepper/plugins/dlq/s3/S3DlqWriterConfigTest.java +++ b/data-prepper-plugins/failures-common/src/test/java/org/opensearch/dataprepper/plugins/dlq/s3/S3DlqWriterConfigTest.java @@ -7,6 +7,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.NullSource; import org.junit.jupiter.params.provider.ValueSource; import software.amazon.awssdk.regions.Region; @@ -42,6 +43,14 @@ public void getS3ClientWithInvalidStsRoleArnThrowException(final String stsRoleA assertThrows(IllegalArgumentException.class, config::getS3Client); } + @ParameterizedTest + @CsvSource({"bucket-name, bucket-name", "s3://bucket-name, bucket-name"}) + public void getS3BucketNameShouldReturnCorrectBucketName(final String bucketName, final String expectedBucketName) throws NoSuchFieldException, IllegalAccessException { + final S3DlqWriterConfig config = new S3DlqWriterConfig(); + reflectivelySetField(config, "bucket", bucketName); + assertThat(config.getBucket(), is(equalTo(expectedBucketName))); + } + @ParameterizedTest @NullSource @ValueSource(strings = {"", "arn:aws:iam::123456789012:role/some-role"}) diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkConfig.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkConfig.java index e39856cb12..6124f20538 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkConfig.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkConfig.java @@ -9,6 +9,7 @@ import jakarta.validation.Valid; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; +import jakarta.validation.constraints.Size; import org.opensearch.dataprepper.model.configuration.PluginModel; import org.opensearch.dataprepper.plugins.sink.s3.accumulator.BufferTypeOptions; import org.opensearch.dataprepper.plugins.sink.s3.compression.CompressionOption; @@ -20,6 +21,7 @@ * s3 sink configuration class contains properties, used to read yaml configuration. */ public class S3SinkConfig { + static final String S3_PREFIX = "s3://"; private static final int DEFAULT_CONNECTION_RETRIES = 5; private static final int DEFAULT_UPLOAD_RETRIES = 5; @@ -30,8 +32,8 @@ public class S3SinkConfig { private AwsAuthenticationOptions awsAuthenticationOptions; @JsonProperty("bucket") - @NotNull @NotEmpty + @Size(min = 3, max = 500, message = "bucket length should be at least 3 characters") private String bucketName; @JsonProperty("object_key") @@ -77,6 +79,9 @@ public ThresholdOptions getThresholdOptions() { * @return bucket name. */ public String getBucketName() { + if (bucketName.startsWith(S3_PREFIX)) { + return bucketName.substring(S3_PREFIX.length()); + } return bucketName; } diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkConfigTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkConfigTest.java index 297a1ef818..d1660ebc63 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkConfigTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkConfigTest.java @@ -16,6 +16,7 @@ import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.jupiter.api.Assertions.assertNull; +import static org.opensearch.dataprepper.plugins.sink.s3.S3SinkConfig.S3_PREFIX; class S3SinkConfigTest { @@ -45,6 +46,15 @@ void get_bucket_name_test() throws NoSuchFieldException, IllegalAccessException assertThat(objectUnderTest.getBucketName(), equalTo(bucketName)); } + @Test + void get_bucket_name_with_s3_prefix_test() throws NoSuchFieldException, IllegalAccessException { + final String bucketName = UUID.randomUUID().toString(); + final String bucketNameWithPrefix = S3_PREFIX + bucketName; + final S3SinkConfig objectUnderTest = new S3SinkConfig(); + ReflectivelySetField.setField(S3SinkConfig.class, objectUnderTest, "bucketName", bucketNameWithPrefix); + assertThat(objectUnderTest.getBucketName(), equalTo(bucketName)); + } + @Test void get_object_key_test() { assertThat("Object key is not an instance of ObjectKeyOptions", diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOption.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOption.java index 323b0480f4..b54dab4075 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOption.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOption.java @@ -7,6 +7,8 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.annotation.JsonDeserialize; import jakarta.validation.constraints.AssertTrue; +import jakarta.validation.constraints.NotEmpty; +import jakarta.validation.constraints.Size; import org.opensearch.dataprepper.plugins.source.CustomLocalDateTimeDeserializer; import java.time.Duration; @@ -18,7 +20,11 @@ * Class consists the bucket related configuration properties. */ public class S3ScanBucketOption { + private static final String S3_PREFIX = "s3://"; + @JsonProperty("name") + @NotEmpty + @Size(min = 3, max = 500, message = "bucket length should be at least 3 characters") private String name; @JsonDeserialize(using = CustomLocalDateTimeDeserializer.class) @@ -41,6 +47,9 @@ public boolean hasValidTimeOptions() { } public String getName() { + if (name.startsWith(S3_PREFIX)) { + return name.substring(S3_PREFIX.length()); + } return name; } diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/ScanOptionsTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/ScanOptionsTest.java index e4d50ef9ba..804d6e1e52 100644 --- a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/ScanOptionsTest.java +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/ScanOptionsTest.java @@ -42,12 +42,15 @@ public void s3scan_options_with_valid_global_time_range_build_success( @ParameterizedTest @MethodSource("invalidTimeRangeOptions") public void s3scan_options_with_invalid_global_time_range_throws_exception_when_build( - LocalDateTime startDateTime, LocalDateTime endDateTime, Duration range) { + LocalDateTime startDateTime, LocalDateTime endDateTime, Duration range) throws NoSuchFieldException, IllegalAccessException { + S3ScanBucketOption bucketOption = new S3ScanBucketOption(); + setField(S3ScanBucketOption.class, bucketOption, "name", "bucket_name"); + assertThrows(IllegalArgumentException.class, () -> ScanOptions.builder() .setStartDateTime(startDateTime) .setEndDateTime(endDateTime) .setRange(range) - .setBucketOption(new S3ScanBucketOption()) + .setBucketOption(bucketOption) .build()); } diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOptionTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOptionTest.java index 69661e6efd..629c6726e7 100644 --- a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOptionTest.java +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOptionTest.java @@ -41,4 +41,11 @@ public void s3scan_bucket_options_with_scan_buckets_yaml_configuration_test() th assertThat(s3ScanBucketOption.getS3ScanFilter().getS3ScanExcludeSuffixOptions(),instanceOf(List.class)); assertThat(s3ScanBucketOption.getS3ScanFilter().getS3ScanExcludeSuffixOptions().get(1),equalTo(".png")); } + + @Test + public void s3scan_bucket_options_with_s3_prefix_test() throws JsonProcessingException { + final String bucketOptionsYaml = "---\nname: s3://test-s3-source-test-output"; + final S3ScanBucketOption s3ScanBucketOption = objectMapper.readValue(bucketOptionsYaml, S3ScanBucketOption.class); + assertThat(s3ScanBucketOption.getName(), equalTo("test-s3-source-test-output")); + } } From 252a0dd874fcb09c7bd5bb1a4818d72c7b742d5f Mon Sep 17 00:00:00 2001 From: Taylor Gray Date: Tue, 15 Aug 2023 15:36:04 -0500 Subject: [PATCH 18/28] Add catching and logging of exceptions for s3 scan worker (#3159) Signed-off-by: Taylor Gray --- .../source/S3ScanPartitionCreationSupplier.java | 4 ++++ .../plugins/source/ScanObjectWorker.java | 14 +++++++++++++- 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplier.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplier.java index 731803f7cf..e1ebea9fa0 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplier.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplier.java @@ -116,10 +116,14 @@ private List listFilteredS3ObjectsForBucket(final List PartitionIdentifier.builder().withPartitionKey(String.format(BUCKET_OBJECT_PARTITION_KEY_FORMAT, bucket, objectKey)).build()) .collect(Collectors.toList())); + LOG.info("Found page of {} objects from bucket {}", listObjectsV2Response.keyCount(), bucket); + mostRecentLastModifiedTimestamp = getMostRecentLastModifiedTimestamp(listObjectsV2Response, mostRecentLastModifiedTimestamp); } while (listObjectsV2Response.isTruncated()); globalStateMap.put(bucket, Objects.nonNull(mostRecentLastModifiedTimestamp) ? mostRecentLastModifiedTimestamp.toString() : null); + + LOG.info("Returning partitions for {} S3 objects from bucket {}", allPartitionIdentifiers.size(), bucket); return allPartitionIdentifiers; } diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/ScanObjectWorker.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/ScanObjectWorker.java index 1257efeaf4..39a251cddc 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/ScanObjectWorker.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/ScanObjectWorker.java @@ -42,6 +42,7 @@ public class ScanObjectWorker implements Runnable{ private static final Logger LOG = LoggerFactory.getLogger(ScanObjectWorker.class); private static final int STANDARD_BACKOFF_MILLIS = 30_000; + private static final int RETRY_BACKOFF_ON_EXCEPTION_MILLIS = 5_000; static final int ACKNOWLEDGEMENT_SET_TIMEOUT_SECONDS = Integer.MAX_VALUE; static final String ACKNOWLEDGEMENT_SET_CALLBACK_METRIC_NAME = "acknowledgementSetCallbackCounter"; @@ -99,7 +100,18 @@ public ScanObjectWorker(final S3Client s3Client, @Override public void run() { while (!shouldStopProcessing) { - startProcessingObject(STANDARD_BACKOFF_MILLIS); + + try { + startProcessingObject(STANDARD_BACKOFF_MILLIS); + } catch (final Exception e) { + LOG.error("Received an exception while processing S3 objects, backing off and retrying", e); + try { + Thread.sleep(RETRY_BACKOFF_ON_EXCEPTION_MILLIS); + } catch (InterruptedException ex) { + LOG.error("S3 Scan worker thread interrupted while backing off.", ex); + } + } + } } From bf32f0748e5a3f788f3dae385a88571c588500fe Mon Sep 17 00:00:00 2001 From: David Venable Date: Wed, 16 Aug 2023 10:39:35 -0500 Subject: [PATCH 19/28] Snappy as a compression option in the S3 sink: adds new option and engine, adds missing unit test for the CompressionOption class, make other compression engine classes package private. (#3155) Signed-off-by: David Venable --- .../s3/compression/CompressionOption.java | 11 +++- .../s3/compression/GZipCompressionEngine.java | 2 +- .../s3/compression/NoneCompressionEngine.java | 2 +- .../compression/SnappyCompressionEngine.java | 18 ++++++ .../s3/compression/CompressionOptionTest.java | 51 ++++++++++++++++ .../SnappyCompressionEngineTest.java | 61 +++++++++++++++++++ 6 files changed, 140 insertions(+), 5 deletions(-) create mode 100644 data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/SnappyCompressionEngine.java create mode 100644 data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/compression/CompressionOptionTest.java create mode 100644 data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/compression/SnappyCompressionEngineTest.java diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/CompressionOption.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/CompressionOption.java index 7e759909d5..ea16e375cd 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/CompressionOption.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/CompressionOption.java @@ -14,7 +14,8 @@ public enum CompressionOption { NONE("none", NoneCompressionEngine::new), - GZIP("gzip", GZipCompressionEngine::new); + GZIP("gzip", GZipCompressionEngine::new), + SNAPPY("snappy", SnappyCompressionEngine::new); private static final Map OPTIONS_MAP = Arrays.stream(CompressionOption.values()) .collect(Collectors.toMap( @@ -23,8 +24,8 @@ public enum CompressionOption { )); private final String option; - private final Supplier compressionEngineSupplier; + private final Supplier compressionEngineSupplier; CompressionOption(final String option, final Supplier compressionEngineSupplier) { this.option = option.toLowerCase(); this.compressionEngineSupplier = compressionEngineSupplier; @@ -34,8 +35,12 @@ public CompressionEngine getCompressionEngine() { return compressionEngineSupplier.get(); } + String getOption() { + return option; + } + @JsonCreator public static CompressionOption fromOptionValue(final String option) { - return OPTIONS_MAP.get(option.toLowerCase()); + return OPTIONS_MAP.get(option); } } diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/GZipCompressionEngine.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/GZipCompressionEngine.java index f59956a8ed..3fd045b714 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/GZipCompressionEngine.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/GZipCompressionEngine.java @@ -10,7 +10,7 @@ import java.io.IOException; import java.io.OutputStream; -public class GZipCompressionEngine implements CompressionEngine { +class GZipCompressionEngine implements CompressionEngine { @Override public OutputStream createOutputStream(final OutputStream outputStream) throws IOException { return new GzipCompressorOutputStream(outputStream); diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/NoneCompressionEngine.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/NoneCompressionEngine.java index 9c852b4f85..e7eed68da2 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/NoneCompressionEngine.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/NoneCompressionEngine.java @@ -7,7 +7,7 @@ import java.io.OutputStream; -public class NoneCompressionEngine implements CompressionEngine { +class NoneCompressionEngine implements CompressionEngine { @Override public OutputStream createOutputStream(final OutputStream outputStream) { return outputStream; diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/SnappyCompressionEngine.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/SnappyCompressionEngine.java new file mode 100644 index 0000000000..03bf0eec1b --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/compression/SnappyCompressionEngine.java @@ -0,0 +1,18 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3.compression; + +import org.xerial.snappy.SnappyOutputStream; + +import java.io.IOException; +import java.io.OutputStream; + +class SnappyCompressionEngine implements CompressionEngine { + @Override + public OutputStream createOutputStream(final OutputStream outputStream) throws IOException { + return new SnappyOutputStream(outputStream); + } +} diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/compression/CompressionOptionTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/compression/CompressionOptionTest.java new file mode 100644 index 0000000000..15a13b31db --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/compression/CompressionOptionTest.java @@ -0,0 +1,51 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3.compression; + +import org.junit.jupiter.api.extension.ExtensionContext; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.ArgumentsProvider; +import org.junit.jupiter.params.provider.ArgumentsSource; +import org.junit.jupiter.params.provider.EnumSource; + +import java.util.stream.Stream; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.params.provider.Arguments.arguments; + +class CompressionOptionTest { + @ParameterizedTest + @EnumSource(CompressionOption.class) + void fromOptionValue_returns_expected_value(final CompressionOption option) { + assertThat(CompressionOption.fromOptionValue(option.getOption()), equalTo(option)); + } + + @ParameterizedTest + @EnumSource(CompressionOption.class) + void getCompressionEngine_returns_a_CompressionEngine(final CompressionOption option) { + assertThat(option.getCompressionEngine(), instanceOf(CompressionEngine.class)); + } + + @ParameterizedTest + @ArgumentsSource(OptionToExpectedEngine.class) + void getCompressionEngine_returns_expected_engine_type(final CompressionOption option, final Class expectedEngineType) { + assertThat(option.getCompressionEngine(), instanceOf(expectedEngineType)); + } + + static class OptionToExpectedEngine implements ArgumentsProvider { + @Override + public Stream provideArguments(final ExtensionContext context) { + return Stream.of( + arguments(CompressionOption.NONE, NoneCompressionEngine.class), + arguments(CompressionOption.GZIP, GZipCompressionEngine.class), + arguments(CompressionOption.SNAPPY, SnappyCompressionEngine.class) + ); + } + } +} \ No newline at end of file diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/compression/SnappyCompressionEngineTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/compression/SnappyCompressionEngineTest.java new file mode 100644 index 0000000000..709d445d05 --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/compression/SnappyCompressionEngineTest.java @@ -0,0 +1,61 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3.compression; + +import org.junit.jupiter.api.Test; +import org.xerial.snappy.SnappyCodec; +import org.xerial.snappy.SnappyInputStream; +import org.xerial.snappy.SnappyOutputStream; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.charset.StandardCharsets; +import java.util.UUID; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; + +class SnappyCompressionEngineTest { + + private SnappyCompressionEngine createObjectUnderTest() { + return new SnappyCompressionEngine(); + } + + @Test + void createOutputStream_should_return_SnappyOutputStream() throws IOException { + final OutputStream innerOutputStream = mock(OutputStream.class); + final OutputStream outputStream = createObjectUnderTest().createOutputStream(innerOutputStream); + + assertThat(outputStream, instanceOf(SnappyOutputStream.class)); + } + + @Test + void createOutputStream_should_write_compressed_data() throws IOException { + final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + + final OutputStream outputStream = createObjectUnderTest().createOutputStream(byteArrayOutputStream); + + final byte[] inputBytes = UUID.randomUUID().toString().getBytes(StandardCharsets.UTF_8); + + outputStream.write(inputBytes); + outputStream.close(); + + final byte[] writtenBytes = byteArrayOutputStream.toByteArray(); + + assertTrue(SnappyCodec.hasMagicHeaderPrefix(writtenBytes)); + + final ByteArrayInputStream verificationInputStream = new ByteArrayInputStream(writtenBytes); + + final SnappyInputStream uncompressingInputStream = new SnappyInputStream(verificationInputStream); + final byte[] uncompressedBytes = uncompressingInputStream.readAllBytes(); + assertThat(uncompressedBytes, equalTo(inputBytes)); + } +} \ No newline at end of file From 13cbf580e9332139c4b75b778ce418bc3e96566b Mon Sep 17 00:00:00 2001 From: Chase <62891993+engechas@users.noreply.github.com> Date: Wed, 16 Aug 2023 11:22:07 -0500 Subject: [PATCH 20/28] Fix S3 sink writing to closed stream exception (#3170) Signed-off-by: Chase Engelbrecht --- .../plugins/sink/s3/S3SinkService.java | 1 + .../plugins/sink/s3/S3SinkServiceTest.java | 28 +++++++++++++++++++ 2 files changed, 29 insertions(+) diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkService.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkService.java index 0b56890f8e..68d504668d 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkService.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkService.java @@ -138,6 +138,7 @@ void output(Collection> records) { releaseEventHandles(false); } currentBuffer = bufferFactory.getBuffer(); + outputStream = currentBuffer.getOutputStream(); } } } catch (IOException | InterruptedException e) { diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkServiceTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkServiceTest.java index 3334a8a6df..528ff7777b 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkServiceTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkServiceTest.java @@ -257,6 +257,34 @@ void test_output_with_uploadedToS3_success_records_byte_count() throws IOExcepti verify(s3ObjectSizeSummary, times(50)).record(objectSize); } + @Test + void test_output_with_uploadedToS3_midBatch_generatesNewOutputStream() throws IOException { + + bufferFactory = mock(BufferFactory.class); + InMemoryBuffer buffer = mock(InMemoryBuffer.class); + when(buffer.getEventCount()).thenReturn(10); + doNothing().when(buffer).flushToS3(any(S3Client.class), anyString(), any(String.class)); + when(bufferFactory.getBuffer()).thenReturn(buffer); + final OutputStream outputStream1 = mock(OutputStream.class); + final OutputStream outputStream2 = mock(OutputStream.class); + when(buffer.getOutputStream()) + .thenReturn(outputStream1) + .thenReturn(outputStream2); + + doNothing().when(codec).writeEvent(any(), eq(outputStream1)); + doNothing().when(codec).writeEvent(any(), eq(outputStream2)); + + S3SinkService s3SinkService = createObjectUnderTest(); + assertNotNull(s3SinkService); + assertThat(s3SinkService, instanceOf(S3SinkService.class)); + + s3SinkService.output(generateEventRecords(2)); + + verify(snapshotSuccessCounter, times(2)).increment(); + verify(codec).writeEvent(any(), eq(outputStream1)); + verify(codec).writeEvent(any(), eq(outputStream2)); + } + @Test void test_output_with_uploadedToS3_failed() throws IOException { when(s3SinkConfig.getBucketName()).thenReturn(UUID.randomUUID().toString()); From 1a17c08b306ec8d35e2d79f11331f517a497c5ff Mon Sep 17 00:00:00 2001 From: Chase <62891993+engechas@users.noreply.github.com> Date: Wed, 16 Aug 2023 11:52:53 -0500 Subject: [PATCH 21/28] Fix timestamp format (#3171) Signed-off-by: Chase Engelbrecht --- .../plugins/sink/s3/configuration/ObjectKeyOptions.java | 2 +- .../plugins/sink/s3/configuration/ObjectKeyOptionsTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/configuration/ObjectKeyOptions.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/configuration/ObjectKeyOptions.java index bd0b07cf1a..bc2c93eb3f 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/configuration/ObjectKeyOptions.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/configuration/ObjectKeyOptions.java @@ -11,7 +11,7 @@ * An implementation class of path prefix and file pattern configuration Options */ public class ObjectKeyOptions { - private static final String DEFAULT_OBJECT_NAME_PATTERN = "events-%{yyyy-MM-dd'T'hh-mm-ss}"; + private static final String DEFAULT_OBJECT_NAME_PATTERN = "events-%{yyyy-MM-dd'T'HH-mm-ss'Z'}"; @JsonProperty("path_prefix") private String pathPrefix; diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/configuration/ObjectKeyOptionsTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/configuration/ObjectKeyOptionsTest.java index 5f52015eaf..6dd3f9c8bf 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/configuration/ObjectKeyOptionsTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/configuration/ObjectKeyOptionsTest.java @@ -12,7 +12,7 @@ class ObjectKeyOptionsTest { - private static final String DEFAULT_FILE_PATTERN = "events-%{yyyy-MM-dd'T'hh-mm-ss}"; + private static final String DEFAULT_FILE_PATTERN = "events-%{yyyy-MM-dd'T'HH-mm-ss'Z'}"; @Test void default_file_pattern_test() { From e3b425e0fee6c0de995378c10aa6e7a5ade6866f Mon Sep 17 00:00:00 2001 From: Taylor Gray Date: Wed, 16 Aug 2023 12:18:00 -0500 Subject: [PATCH 22/28] Reword circuit breaker configuration log message (#3175) Signed-off-by: Taylor Gray --- .../org/opensearch/dataprepper/breaker/HeapCircuitBreaker.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/breaker/HeapCircuitBreaker.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/breaker/HeapCircuitBreaker.java index 827f3c9567..dc34bd5cd0 100644 --- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/breaker/HeapCircuitBreaker.java +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/breaker/HeapCircuitBreaker.java @@ -66,7 +66,7 @@ class HeapCircuitBreaker implements InnerCircuitBreaker, AutoCloseable { scheduledExecutorService .scheduleAtFixedRate(this::checkMemory, 0L, checkInterval.toMillis(), TimeUnit.MILLISECONDS); - LOG.info("Heap circuit breaker with usage of {} bytes.", usageBytes); + LOG.info("Circuit breaker heap limit is set to {} bytes.", usageBytes); } @Override From 1f0ad76de142b041a63038d958a6e622934c3cd7 Mon Sep 17 00:00:00 2001 From: kkondaka <41027584+kkondaka@users.noreply.github.com> Date: Wed, 16 Aug 2023 10:23:15 -0700 Subject: [PATCH 23/28] Minor fixes to Kafka Source (#3174) * Minor fixes to Kafka Source Signed-off-by: Krishna Kondaka * Removed unused configs Signed-off-by: Krishna Kondaka --------- Signed-off-by: Krishna Kondaka Co-authored-by: Krishna Kondaka --- .../kafka/source/KafkaSourceJsonTypeIT.java | 8 +-- .../kafka/configuration/AuthConfig.java | 26 ++++---- .../kafka/configuration/TopicConfig.java | 64 +------------------ .../consumer/KafkaSourceCustomConsumer.java | 10 ++- .../plugins/kafka/util/MessageFormat.java | 20 +++--- .../kafka/configuration/TopicConfigTest.java | 8 +-- .../test/resources/sample-pipelines-1.yaml | 1 - .../src/test/resources/sample-pipelines.yaml | 3 - 8 files changed, 39 insertions(+), 101 deletions(-) diff --git a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceJsonTypeIT.java b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceJsonTypeIT.java index dff3d2b943..2436d30b7d 100644 --- a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceJsonTypeIT.java +++ b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceJsonTypeIT.java @@ -176,7 +176,7 @@ public void TestJsonRecordsWithNullKey() throws Exception { assertThat(map.get("status"), equalTo(true)); assertThat(map.get("kafka_key"), equalTo(null)); assertThat(metadata.getAttributes().get("kafka_topic"), equalTo(topicName)); - assertThat(metadata.getAttributes().get("kafka_partition"), equalTo(0)); + assertThat(metadata.getAttributes().get("kafka_partition"), equalTo("0")); } try (AdminClient adminClient = AdminClient.create(props)) { try { @@ -235,7 +235,7 @@ public void TestJsonRecordsWithKafkaKeyModeDiscard() throws Exception { assertThat(map.get("id"), equalTo(TEST_ID+i)); assertThat(map.get("status"), equalTo(true)); assertThat(metadata.getAttributes().get("kafka_topic"), equalTo(topicName)); - assertThat(metadata.getAttributes().get("kafka_partition"), equalTo(0)); + assertThat(metadata.getAttributes().get("kafka_partition"), equalTo("0")); } try (AdminClient adminClient = AdminClient.create(props)) { try { @@ -295,7 +295,7 @@ public void TestJsonRecordsWithKafkaKeyModeAsField() throws Exception { assertThat(map.get("status"), equalTo(true)); assertThat(map.get("kafka_key"), equalTo(testKey)); assertThat(metadata.getAttributes().get("kafka_topic"), equalTo(topicName)); - assertThat(metadata.getAttributes().get("kafka_partition"), equalTo(0)); + assertThat(metadata.getAttributes().get("kafka_partition"), equalTo("0")); } try (AdminClient adminClient = AdminClient.create(props)) { try { @@ -356,7 +356,7 @@ public void TestJsonRecordsWithKafkaKeyModeAsMetadata() throws Exception { assertThat(map.get("status"), equalTo(true)); assertThat(metadata.getAttributes().get("kafka_key"), equalTo(testKey)); assertThat(metadata.getAttributes().get("kafka_topic"), equalTo(topicName)); - assertThat(metadata.getAttributes().get("kafka_partition"), equalTo(0)); + assertThat(metadata.getAttributes().get("kafka_partition"), equalTo("0")); } try (AdminClient adminClient = AdminClient.create(props)) { try { diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/AuthConfig.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/AuthConfig.java index 4447f81a97..fc469950de 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/AuthConfig.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/AuthConfig.java @@ -54,25 +54,29 @@ public boolean hasOnlyOneConfig() { } - public static class SslAuthConfig { - // TODO Add Support for SSL authentication types like - // one-way or two-way authentication + /* + * TODO + public static class SslAuthConfig { + // TODO Add Support for SSL authentication types like + // one-way or two-way authentication - public SslAuthConfig() { + public SslAuthConfig() { + } + } + + @JsonProperty("ssl") + private SslAuthConfig sslAuthConfig; + + public SslAuthConfig getSslAuthConfig() { + return sslAuthConfig; } - } - @JsonProperty("ssl") - private SslAuthConfig sslAuthConfig; + */ @Valid @JsonProperty("sasl") private SaslAuthConfig saslAuthConfig; - public SslAuthConfig getSslAuthConfig() { - return sslAuthConfig; - } - public SaslAuthConfig getSaslAuthConfig() { return saslAuthConfig; } diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfig.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfig.java index e8244098d6..efab2ad36c 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfig.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfig.java @@ -21,19 +21,16 @@ public class TopicConfig { static final boolean DEFAULT_AUTO_COMMIT = false; static final Duration DEFAULT_COMMIT_INTERVAL = Duration.ofSeconds(5); static final Duration DEFAULT_SESSION_TIMEOUT = Duration.ofSeconds(45); - static final int DEFAULT_MAX_RETRY_ATTEMPT = Integer.MAX_VALUE; static final String DEFAULT_AUTO_OFFSET_RESET = "earliest"; static final Duration DEFAULT_THREAD_WAITING_TIME = Duration.ofSeconds(5); static final Duration DEFAULT_MAX_RECORD_FETCH_TIME = Duration.ofSeconds(4); - static final Duration DEFAULT_BUFFER_TIMEOUT = Duration.ofSeconds(5); - static final Duration DEFAULT_MAX_RETRY_DELAY = Duration.ofSeconds(1); static final Integer DEFAULT_FETCH_MAX_BYTES = 52428800; static final Integer DEFAULT_FETCH_MAX_WAIT = 500; static final Integer DEFAULT_FETCH_MIN_BYTES = 1; static final Duration DEFAULT_RETRY_BACKOFF = Duration.ofSeconds(10); static final Duration DEFAULT_RECONNECT_BACKOFF = Duration.ofSeconds(10); static final Integer DEFAULT_MAX_PARTITION_FETCH_BYTES = 1048576; - static final Duration DEFAULT_MAX_POLL_INTERVAL = Duration.ofSeconds(300000); + static final Duration DEFAULT_MAX_POLL_INTERVAL = Duration.ofSeconds(300); static final Integer DEFAULT_CONSUMER_MAX_POLL_RECORDS = 500; static final Integer DEFAULT_NUM_OF_WORKERS = 2; static final Duration DEFAULT_HEART_BEAT_INTERVAL_DURATION = Duration.ofSeconds(5); @@ -53,16 +50,6 @@ public class TopicConfig { @Size(min = 1, max = 200, message = "Number of worker threads should lies between 1 and 200") private Integer workers = DEFAULT_NUM_OF_WORKERS; - @JsonProperty("max_retry_attempts") - @Valid - @Size(min = 1, max = Integer.MAX_VALUE, message = " Max retry attempts should lies between 1 and Integer.MAX_VALUE") - private Integer maxRetryAttempts = DEFAULT_MAX_RETRY_ATTEMPT; - - @JsonProperty("max_retry_delay") - @Valid - @Size(min = 1) - private Duration maxRetryDelay = DEFAULT_MAX_RETRY_DELAY; - @JsonProperty("serde_format") private MessageFormat serdeFormat= MessageFormat.PLAINTEXT; @@ -82,25 +69,12 @@ public class TopicConfig { @JsonProperty("auto_offset_reset") private String autoOffsetReset = DEFAULT_AUTO_OFFSET_RESET; - @JsonProperty("group_name") - @Valid - @Size(min = 1, max = 255, message = "size of group name should be between 1 and 255") - private String groupName; - @JsonProperty("thread_waiting_time") private Duration threadWaitingTime = DEFAULT_THREAD_WAITING_TIME; - @JsonProperty("max_record_fetch_time") - private Duration maxRecordFetchTime = DEFAULT_MAX_RECORD_FETCH_TIME; - @JsonProperty("max_partition_fetch_bytes") private Integer maxPartitionFetchBytes = DEFAULT_MAX_PARTITION_FETCH_BYTES; - @JsonProperty("buffer_default_timeout") - @Valid - @Size(min = 1) - private Duration bufferDefaultTimeout = DEFAULT_BUFFER_TIMEOUT; - @JsonProperty("fetch_max_bytes") @Valid @Size(min = 1, max = 52428800) @@ -144,10 +118,6 @@ public void setGroupId(String groupId) { this.groupId = groupId; } - public void setMaxRetryAttempts(Integer maxRetryAttempts) { - this.maxRetryAttempts = maxRetryAttempts; - } - public MessageFormat getSerdeFormat() { return serdeFormat; } @@ -176,14 +146,6 @@ public void setAutoOffsetReset(String autoOffsetReset) { this.autoOffsetReset = autoOffsetReset; } - public String getGroupName() { - return groupName; - } - - public void setGroupName(String groupName) { - this.groupName = groupName; - } - public Duration getThreadWaitingTime() { return threadWaitingTime; } @@ -192,26 +154,10 @@ public void setThreadWaitingTime(Duration threadWaitingTime) { this.threadWaitingTime = threadWaitingTime; } - public Duration getMaxRecordFetchTime() { - return maxRecordFetchTime; - } - public Integer getMaxPartitionFetchBytes() { return maxPartitionFetchBytes; } - public void setMaxRecordFetchTime(Duration maxRecordFetchTime) { - this.maxRecordFetchTime = maxRecordFetchTime; - } - - public Duration getBufferDefaultTimeout() { - return bufferDefaultTimeout; - } - - public void setBufferDefaultTimeout(Duration bufferDefaultTimeout) { - this.bufferDefaultTimeout = bufferDefaultTimeout; - } - public Integer getFetchMaxBytes() { return fetchMaxBytes; } @@ -264,14 +210,6 @@ public void setWorkers(Integer workers) { this.workers = workers; } - public Duration getMaxRetryDelay() { - return maxRetryDelay; - } - - public void setMaxRetryDelay(Duration maxRetryDelay) { - this.maxRetryDelay = maxRetryDelay; - } - public Duration getHeartBeatInterval() { return heartBeatInterval; } 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 805cfb6497..04153f7543 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 @@ -234,14 +234,20 @@ Map getOffsetsToCommit() { @Override public void run() { consumer.subscribe(Arrays.asList(topicName)); + boolean retryingAfterException = false; while (!shutdownInProgress.get()) { try { + if (retryingAfterException) { + Thread.sleep(10000); + } resetOffsets(); commitOffsets(); consumeRecords(); topicMetrics.update(consumer); + retryingAfterException = false; } catch (Exception exp) { - LOG.error("Error while reading the records from the topic {}", topicName, exp); + LOG.error("Error while reading the records from the topic {}. Retry after 10 seconds", topicName, exp); + retryingAfterException = true; } } } @@ -292,7 +298,7 @@ private Record getRecord(ConsumerRecord consumerRecord, in eventMetadata.setAttribute("kafka_key", key); } eventMetadata.setAttribute("kafka_topic", topicName); - eventMetadata.setAttribute("kafka_partition", partition); + eventMetadata.setAttribute("kafka_partition", String.valueOf(partition)); return new Record(event); } diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/MessageFormat.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/MessageFormat.java index 8cf19a32bf..f2074f1ab6 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/MessageFormat.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/MessageFormat.java @@ -5,9 +5,10 @@ package org.opensearch.dataprepper.plugins.kafka.util; +import com.fasterxml.jackson.annotation.JsonCreator; + import java.util.Arrays; import java.util.Map; -import java.util.function.Function; import java.util.stream.Collectors; /** @@ -19,19 +20,18 @@ public enum MessageFormat { PLAINTEXT("plaintext"), JSON("json"), AVRO("avro"); private static final Map MESSAGE_FORMAT_MAP = Arrays.stream(MessageFormat.values()) - .collect(Collectors.toMap(MessageFormat::toString, Function.identity())); - - private final String messageFormatName; + .collect(Collectors.toMap( + value -> value.type, + value -> value + )); - MessageFormat(final String name) { - this.messageFormatName = name; - } + private final String type; - @Override - public String toString() { - return this.messageFormatName; + MessageFormat(final String type) { + this.type = type; } + @JsonCreator public static MessageFormat getByMessageFormatByName(final String name) { return MESSAGE_FORMAT_MAP.get(name.toLowerCase()); } diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfigTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfigTest.java index e244ed15b1..26e6b4415e 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfigTest.java +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfigTest.java @@ -73,8 +73,6 @@ void testConfigValues_default() { assertEquals(TopicConfig.DEFAULT_SESSION_TIMEOUT, topicConfig.getSessionTimeOut()); assertEquals(TopicConfig.DEFAULT_AUTO_OFFSET_RESET, topicConfig.getAutoOffsetReset()); assertEquals(TopicConfig.DEFAULT_THREAD_WAITING_TIME, topicConfig.getThreadWaitingTime()); - assertEquals(TopicConfig.DEFAULT_MAX_RECORD_FETCH_TIME, topicConfig.getMaxRecordFetchTime()); - assertEquals(TopicConfig.DEFAULT_BUFFER_TIMEOUT, topicConfig.getBufferDefaultTimeout()); assertEquals(TopicConfig.DEFAULT_FETCH_MAX_BYTES, topicConfig.getFetchMaxBytes()); assertEquals(TopicConfig.DEFAULT_FETCH_MAX_WAIT, topicConfig.getFetchMaxWait()); assertEquals(TopicConfig.DEFAULT_FETCH_MIN_BYTES, topicConfig.getFetchMinBytes()); @@ -96,13 +94,11 @@ void testConfigValues_from_yaml() { assertEquals(45000, topicConfig.getSessionTimeOut().toMillis()); assertEquals("earliest", topicConfig.getAutoOffsetReset()); assertEquals(Duration.ofSeconds(1), topicConfig.getThreadWaitingTime()); - assertEquals(Duration.ofSeconds(4), topicConfig.getMaxRecordFetchTime()); - assertEquals(Duration.ofSeconds(5), topicConfig.getBufferDefaultTimeout()); assertEquals(52428800, topicConfig.getFetchMaxBytes().longValue()); assertEquals(500L, topicConfig.getFetchMaxWait().longValue()); assertEquals(1L, topicConfig.getFetchMinBytes().longValue()); assertEquals(Duration.ofSeconds(100), topicConfig.getRetryBackoff()); - assertEquals(Duration.ofSeconds(300000), topicConfig.getMaxPollInterval()); + assertEquals(Duration.ofSeconds(300), topicConfig.getMaxPollInterval()); assertEquals(500L, topicConfig.getConsumerMaxPollRecords().longValue()); assertEquals(5, topicConfig.getWorkers().intValue()); assertEquals(Duration.ofSeconds(3), topicConfig.getHeartBeatInterval()); @@ -118,8 +114,6 @@ void testConfigValues_from_yaml_not_null() { assertNotNull(topicConfig.getSessionTimeOut()); assertNotNull(topicConfig.getAutoOffsetReset()); assertNotNull(topicConfig.getThreadWaitingTime()); - assertNotNull(topicConfig.getMaxRecordFetchTime()); - assertNotNull(topicConfig.getBufferDefaultTimeout()); assertNotNull(topicConfig.getFetchMaxBytes()); assertNotNull(topicConfig.getFetchMaxWait()); assertNotNull(topicConfig.getFetchMinBytes()); diff --git a/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-1.yaml b/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-1.yaml index 67f655e167..6f3502a175 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-1.yaml +++ b/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-1.yaml @@ -5,7 +5,6 @@ log-pipeline: - 127.0.0.1:9093 topics: - name: my-topic-2 - group_name: kafka-consumer-group-2 group_id: my-test-group - name: my-topic-1 group_id: my-test-group diff --git a/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines.yaml b/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines.yaml index e6ecc1a243..4a5adbace2 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines.yaml +++ b/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines.yaml @@ -12,12 +12,9 @@ log-pipeline: auto_commit: false commit_interval: PT5S session_timeout: PT45S - max_retry_attempts: 1000 auto_offset_reset: earliest thread_waiting_time: PT1S - max_record_fetch_time: PT4S heart_beat_interval: PT3S - buffer_default_timeout: PT5S fetch_max_bytes: 52428800 fetch_max_wait: 500 fetch_min_bytes: 1 From 1ede4b611db28f027a2808d7db4498bedefa4318 Mon Sep 17 00:00:00 2001 From: Taylor Gray Date: Wed, 16 Aug 2023 12:27:29 -0500 Subject: [PATCH 24/28] Deprecate document_id_field and add support for document_id with formatting (#3153) Deprecate document_id_field and add support for document_id with formatting Signed-off-by: Taylor Gray --- .../sink/opensearch/OpenSearchSink.java | 16 +++---- .../opensearch/index/IndexConfiguration.java | 42 ++++++++++++++++--- .../index/IndexConfigurationTests.java | 14 +++---- 3 files changed, 51 insertions(+), 21 deletions(-) diff --git a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchSink.java b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchSink.java index ccd430c982..04147619b7 100644 --- a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchSink.java +++ b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchSink.java @@ -101,6 +101,7 @@ public class OpenSearchSink extends AbstractSink> { private final long flushTimeout; private final IndexType indexType; private final String documentIdField; + private final String documentId; private final String routingField; private final String action; private final String documentRootKey; @@ -117,7 +118,6 @@ public class OpenSearchSink extends AbstractSink> { private PluginSetting pluginSetting; private final SinkContext sinkContext; private final ExpressionEvaluator expressionEvaluator; - private final boolean isDocumentIdAnExpression; private FailedBulkOperationConverter failedBulkOperationConverter; @@ -145,7 +145,7 @@ public OpenSearchSink(final PluginSetting pluginSetting, this.flushTimeout = openSearchSinkConfig.getIndexConfiguration().getFlushTimeout(); this.indexType = openSearchSinkConfig.getIndexConfiguration().getIndexType(); this.documentIdField = openSearchSinkConfig.getIndexConfiguration().getDocumentIdField(); - this.isDocumentIdAnExpression = expressionEvaluator.isValidExpressionStatement(documentIdField); + this.documentId = openSearchSinkConfig.getIndexConfiguration().getDocumentId(); this.routingField = openSearchSinkConfig.getIndexConfiguration().getRoutingField(); this.action = openSearchSinkConfig.getIndexConfiguration().getAction(); this.documentRootKey = openSearchSinkConfig.getIndexConfiguration().getDocumentRootKey(); @@ -337,14 +337,14 @@ private SerializedJson getDocument(final Event event) { String docId = null; - if (isDocumentIdAnExpression) { + if (Objects.nonNull(documentIdField)) { + docId = event.get(documentIdField, String.class); + } else if (Objects.nonNull(documentId)) { try { - docId = (String) expressionEvaluator.evaluate(documentIdField, event); - } catch (final ExpressionEvaluationException e) { - LOG.error("Unable to construct document_id_field from expression {}, the document_id will be generated by OpenSearch", documentIdField); + docId = event.formatString(documentId, expressionEvaluator); + } catch (final ExpressionEvaluationException | EventKeyNotFoundException e) { + LOG.error("Unable to construct document_id with format {}, the document_id will be generated by OpenSearch", documentId, e); } - } else if (Objects.nonNull(documentIdField)) { - docId = event.get(documentIdField, String.class); } String routing = (routingField != null) ? event.get(routingField, String.class) : null; diff --git a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/IndexConfiguration.java b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/IndexConfiguration.java index ccc2e1c951..e23fdd4e26 100644 --- a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/IndexConfiguration.java +++ b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/IndexConfiguration.java @@ -9,11 +9,14 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.commons.lang3.EnumUtils; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.plugins.sink.opensearch.DistributionVersion; import org.opensearch.dataprepper.plugins.sink.opensearch.bulk.BulkAction; import org.opensearch.dataprepper.plugins.sink.opensearch.s3.FileReader; import org.opensearch.dataprepper.plugins.sink.opensearch.s3.S3ClientProvider; import org.opensearch.dataprepper.plugins.sink.opensearch.s3.S3FileReader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import software.amazon.awssdk.arns.Arn; import software.amazon.awssdk.services.s3.S3Client; @@ -23,12 +26,15 @@ import java.net.URL; import java.util.HashMap; import java.util.Map; +import java.util.Objects; import java.util.Optional; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; public class IndexConfiguration { + private static final Logger LOG = LoggerFactory.getLogger(IndexConfiguration.class); + public static final String SETTINGS = "settings"; public static final String INDEX_ALIAS = "index"; public static final String INDEX_TYPE = "index_type"; @@ -41,6 +47,7 @@ public class IndexConfiguration { public static final String MAX_LOCAL_COMPRESSIONS_FOR_ESTIMATION = "max_local_compressions_for_estimation"; public static final String FLUSH_TIMEOUT = "flush_timeout"; public static final String DOCUMENT_ID_FIELD = "document_id_field"; + public static final String DOCUMENT_ID = "document_id"; public static final String ROUTING_FIELD = "routing_field"; public static final String ISM_POLICY_FILE = "ism_policy_file"; public static final long DEFAULT_BULK_SIZE = 5L; @@ -61,6 +68,7 @@ public class IndexConfiguration { private final String indexAlias; private final Map indexTemplate; private final String documentIdField; + private final String documentId; private final String routingField; private final long bulkSize; private final boolean estimateBulkSizeUsingCompression; @@ -119,12 +127,14 @@ private IndexConfiguration(final Builder builder) { this.routingField = builder.routingField; String documentIdField = builder.documentIdField; + String documentId = builder.documentId; if (indexType.equals(IndexType.TRACE_ANALYTICS_RAW)) { - documentIdField = "spanId"; + documentId = "${spanId}"; } else if (indexType.equals(IndexType.TRACE_ANALYTICS_SERVICE_MAP)) { - documentIdField = "hashId"; + documentId = "${hashId}"; } this.documentIdField = documentIdField; + this.documentId = documentId; this.ismPolicyFile = builder.ismPolicyFile; this.action = builder.action; this.documentRootKey = builder.documentRootKey; @@ -180,10 +190,21 @@ public static IndexConfiguration readIndexConfig(final PluginSetting pluginSetti final long flushTimeout = pluginSetting.getLongOrDefault(FLUSH_TIMEOUT, DEFAULT_FLUSH_TIMEOUT); builder = builder.withFlushTimeout(flushTimeout); - final String documentId = pluginSetting.getStringOrDefault(DOCUMENT_ID_FIELD, null); - if (documentId != null) { - builder = builder.withDocumentIdField(documentId); + final String documentIdField = pluginSetting.getStringOrDefault(DOCUMENT_ID_FIELD, null); + final String documentId = pluginSetting.getStringOrDefault(DOCUMENT_ID, null); + + + if (Objects.nonNull(documentIdField) && Objects.nonNull(documentId)) { + throw new InvalidPluginConfigurationException("Both document_id_field and document_id cannot be used at the same time. It is preferred to only use document_id as document_id_field is deprecated."); + } + + if (documentIdField != null) { + LOG.warn("document_id_field is deprecated in favor of document_id, and support for document_id_field will be removed in a future major version release."); + builder = builder.withDocumentIdField(documentIdField); + } else if (documentId != null) { + builder = builder.withDocumentId(documentId); } + final String routingField = pluginSetting.getStringOrDefault(ROUTING_FIELD, null); if (routingField != null) { builder = builder.withRoutingField(routingField); @@ -242,6 +263,8 @@ public String getDocumentIdField() { return documentIdField; } + public String getDocumentId() { return documentId; } + public String getRoutingField() { return routingField; } @@ -349,6 +372,7 @@ public static class Builder { private int numReplicas; private String routingField; private String documentIdField; + private String documentId; private long bulkSize = DEFAULT_BULK_SIZE; private boolean estimateBulkSizeUsingCompression = DEFAULT_ESTIMATE_BULK_SIZE_USING_COMPRESSION; private int maxLocalCompressionsForEstimation = DEFAULT_MAX_LOCAL_COMPRESSIONS_FOR_ESTIMATION; @@ -391,11 +415,17 @@ public Builder withTemplateFile(final String templateFile) { } public Builder withDocumentIdField(final String documentIdField) { - checkNotNull(documentIdField, "documentId field cannot be null"); + checkNotNull(documentIdField, "document_id_field cannot be null"); this.documentIdField = documentIdField; return this; } + public Builder withDocumentId(final String documentId) { + checkNotNull(documentId, "document_id cannot be null"); + this.documentId = documentId; + return this; + } + public Builder withRoutingField(final String routingField) { this.routingField = routingField; return this; diff --git a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/IndexConfigurationTests.java b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/IndexConfigurationTests.java index 64810bf1f2..cda9476743 100644 --- a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/IndexConfigurationTests.java +++ b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/IndexConfigurationTests.java @@ -286,7 +286,7 @@ public void testReadIndexConfig_RawIndexType() { assertEquals(60_000L, indexConfiguration.getFlushTimeout()); assertEquals(false, indexConfiguration.isEstimateBulkSizeUsingCompression()); assertEquals(2, indexConfiguration.getMaxLocalCompressionsForEstimation()); - assertEquals("spanId", indexConfiguration.getDocumentIdField()); + assertEquals("${spanId}", indexConfiguration.getDocumentId()); } @Test @@ -312,7 +312,7 @@ public void testReadIndexConfig_ServiceMapIndexType() { assertEquals(60_000L, indexConfiguration.getFlushTimeout()); assertEquals(false, indexConfiguration.isEstimateBulkSizeUsingCompression()); assertEquals(2, indexConfiguration.getMaxLocalCompressionsForEstimation()); - assertEquals("hashId", indexConfiguration.getDocumentIdField()); + assertEquals("${hashId}", indexConfiguration.getDocumentId()); } @Test @@ -335,7 +335,7 @@ public void testReadIndexConfigCustom() { assertEquals(testFlushTimeout, indexConfiguration.getFlushTimeout()); assertEquals(true, indexConfiguration.isEstimateBulkSizeUsingCompression()); assertEquals(5, indexConfiguration.getMaxLocalCompressionsForEstimation()); - assertEquals(testIdField, indexConfiguration.getDocumentIdField()); + assertEquals(testIdField, indexConfiguration.getDocumentId()); } @Test @@ -356,7 +356,7 @@ public void testReadIndexConfig_ExplicitCustomIndexType() { assertFalse(indexConfiguration.getIndexTemplate().isEmpty()); assertEquals(testBulkSize, indexConfiguration.getBulkSize()); assertEquals(testFlushTimeout, indexConfiguration.getFlushTimeout()); - assertEquals(testIdField, indexConfiguration.getDocumentIdField()); + assertEquals(testIdField, indexConfiguration.getDocumentId()); } @Test @@ -458,7 +458,7 @@ private PluginSetting getPluginSetting(Map metadata) { } private Map initializeConfigMetaData( - String indexType, String indexAlias, String templateFilePath, Long bulkSize, Long flushTimeout, String documentIdField) { + String indexType, String indexAlias, String templateFilePath, Long bulkSize, Long flushTimeout, String documentId) { final Map metadata = new HashMap<>(); if (indexType != null) { metadata.put(IndexConfiguration.INDEX_TYPE, indexType); @@ -475,8 +475,8 @@ private Map initializeConfigMetaData( if (flushTimeout != null) { metadata.put(IndexConfiguration.FLUSH_TIMEOUT, flushTimeout); } - if (documentIdField != null) { - metadata.put(IndexConfiguration.DOCUMENT_ID_FIELD, documentIdField); + if (documentId != null) { + metadata.put(IndexConfiguration.DOCUMENT_ID, documentId); } return metadata; } From 91b9eb63f9462ed9a1b604cbd50952c3917f22b1 Mon Sep 17 00:00:00 2001 From: Asif Sohail Mohammed Date: Wed, 16 Aug 2023 12:53:02 -0500 Subject: [PATCH 25/28] Support other scan time configs= combinations (#3151) * Added support for additional time comibinations in s3 scan Signed-off-by: Asif Sohail Mohammed --------- Signed-off-by: Asif Sohail Mohammed --- .../S3ScanPartitionCreationSupplier.java | 10 +++- .../plugins/source/ScanOptions.java | 42 +++++++++----- .../configuration/S3ScanBucketOptions.java | 3 + .../plugins/source/ScanOptionsTest.java | 58 +++++++------------ 4 files changed, 61 insertions(+), 52 deletions(-) diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplier.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplier.java index e1ebea9fa0..89b85c6b83 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplier.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplier.java @@ -138,9 +138,15 @@ private LocalDateTime instantToLocalDateTime(final Instant instant) { */ private boolean isKeyMatchedBetweenTimeRange(final LocalDateTime lastModifiedTime, final LocalDateTime startDateTime, - final LocalDateTime endDateTime){ - if (Objects.isNull(startDateTime) || Objects.isNull(endDateTime) || Objects.nonNull(schedulingOptions)) { + final LocalDateTime endDateTime) { + if (Objects.nonNull(schedulingOptions)) { return true; + } else if (Objects.isNull(startDateTime) && Objects.isNull(endDateTime)) { + return true; + } else if (Objects.isNull(startDateTime)) { + return lastModifiedTime.isBefore(endDateTime); + } else if (Objects.isNull(endDateTime)) { + return lastModifiedTime.isAfter(startDateTime); } return lastModifiedTime.isAfter(startDateTime) && lastModifiedTime.isBefore(endDateTime); } diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/ScanOptions.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/ScanOptions.java index b1a33db719..389c2150f8 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/ScanOptions.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/ScanOptions.java @@ -5,6 +5,8 @@ package org.opensearch.dataprepper.plugins.source; import org.opensearch.dataprepper.plugins.source.configuration.S3ScanBucketOption; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.time.Duration; import java.time.LocalDateTime; @@ -15,7 +17,7 @@ * Class consists the scan related properties. */ public class ScanOptions { - + private static final Logger LOG = LoggerFactory.getLogger(ScanOptions.class); private LocalDateTime startDateTime; private Duration range; @@ -102,21 +104,28 @@ public ScanOptions build() { .filter(Objects::nonNull) .count(); - if (nonNullCount == 0 || nonNullCount == 2) { - setDateTimeToUse(bucketStartDateTime, bucketEndDateTime, bucketRange); - } else if (nonNullCount == 3) { - long originalBucketLevelNonNullCount = Stream.of( - bucketOption.getStartTime(), bucketOption.getEndTime(), bucketOption.getRange()) - .filter(Objects::nonNull) - .count(); + long originalBucketLevelNonNullCount = Stream.of( + bucketOption.getStartTime(), bucketOption.getEndTime(), bucketOption.getRange()) + .filter(Objects::nonNull) + .count(); - if (originalBucketLevelNonNullCount == 2) { - setDateTimeToUse(bucketOption.getStartTime(), bucketOption.getEndTime(), bucketOption.getRange()); - } else { + if (nonNullCount == 3) { + if (originalBucketLevelNonNullCount == 3) { scanRangeDateValidationError(); + } else if (originalBucketLevelNonNullCount == 2) { + setDateTimeToUse(bucketOption.getStartTime(), bucketOption.getEndTime(), bucketOption.getRange()); + } else if (originalBucketLevelNonNullCount == 1) { + if (Objects.nonNull(bucketOption.getStartTime()) || Objects.nonNull(bucketOption.getEndTime())) { + setDateTimeToUse(bucketOption.getStartTime(), bucketOption.getEndTime(), bucketOption.getRange()); + } else { + LOG.warn("Scan is configured with start_time and end_time at global level and range at bucket level for the bucket with name {}. " + + "Unable to establish a time period with range alone at bucket level. " + + "Using start_time and end_time configured at global level and ignoring range.", bucketOption.getName()); + setDateTimeToUse(startDateTime, endDateTime, range); + } } } else { - scanRangeDateValidationError(); + setDateTimeToUse(bucketStartDateTime, bucketEndDateTime, bucketRange); } return new ScanOptions(this); } @@ -132,10 +141,17 @@ private void setDateTimeToUse(LocalDateTime bucketStartDateTime, LocalDateTime b } else if (Objects.nonNull(bucketEndDateTime) && Objects.nonNull(bucketRange)) { this.useStartDateTime = bucketEndDateTime.minus(bucketRange); this.useEndDateTime = bucketEndDateTime; + } else if (Objects.nonNull(bucketStartDateTime)) { + this.useStartDateTime = bucketStartDateTime; + } else if (Objects.nonNull(bucketEndDateTime)) { + this.useEndDateTime = bucketEndDateTime; + } else if (Objects.nonNull(bucketRange)) { + LOG.warn("Scan is configured with just range for the bucket with name {}. Unable to establish a time period with range alone. " + + "Configure start_time or end_time, else all the objects in the bucket will be included", bucketOption.getName()); } } - private void scanRangeDateValidationError(){ + private void scanRangeDateValidationError() { String message = "To set a time range for the bucket with name " + bucketOption.getName() + ", specify any two configurations from start_time, end_time and range"; throw new IllegalArgumentException(message); diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOptions.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOptions.java index 86292feffe..1bf077813d 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOptions.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/configuration/S3ScanBucketOptions.java @@ -5,11 +5,14 @@ package org.opensearch.dataprepper.plugins.source.configuration; import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.Valid; + /** * Class consists the bucket properties. */ public class S3ScanBucketOptions { @JsonProperty("bucket") + @Valid private S3ScanBucketOption scanBucketOption; public S3ScanBucketOption getS3ScanBucketOption() { diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/ScanOptionsTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/ScanOptionsTest.java index 804d6e1e52..e508307b2d 100644 --- a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/ScanOptionsTest.java +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/ScanOptionsTest.java @@ -39,21 +39,6 @@ public void s3scan_options_with_valid_global_time_range_build_success( assertThat(scanOptions.getBucketOption(), instanceOf(S3ScanBucketOption.class)); } - @ParameterizedTest - @MethodSource("invalidTimeRangeOptions") - public void s3scan_options_with_invalid_global_time_range_throws_exception_when_build( - LocalDateTime startDateTime, LocalDateTime endDateTime, Duration range) throws NoSuchFieldException, IllegalAccessException { - S3ScanBucketOption bucketOption = new S3ScanBucketOption(); - setField(S3ScanBucketOption.class, bucketOption, "name", "bucket_name"); - - assertThrows(IllegalArgumentException.class, () -> ScanOptions.builder() - .setStartDateTime(startDateTime) - .setEndDateTime(endDateTime) - .setRange(range) - .setBucketOption(bucketOption) - .build()); - } - @ParameterizedTest @MethodSource("validBucketTimeRangeOptions") public void s3scan_options_with_valid_bucket_time_range_build_success( @@ -142,6 +127,8 @@ private static Stream validGlobalTimeRangeOptions() { LocalDateTime.parse("2023-01-21T18:00:00"), LocalDateTime.parse("2023-01-24T18:00:00")), Arguments.of(null, LocalDateTime.parse("2023-01-24T18:00:00"), Duration.parse("P3D"), LocalDateTime.parse("2023-01-21T18:00:00"), LocalDateTime.parse("2023-01-24T18:00:00")), + Arguments.of(LocalDateTime.parse("2023-01-21T18:00:00"), null, null, LocalDateTime.parse("2023-01-21T18:00:00"), null), + Arguments.of(null, LocalDateTime.parse("2023-01-21T18:00:00"), null, null, LocalDateTime.parse("2023-01-21T18:00:00")), Arguments.of(null, null, null, null, null) ); } @@ -149,10 +136,7 @@ private static Stream validGlobalTimeRangeOptions() { private static Stream invalidTimeRangeOptions() { return Stream.of( Arguments.of(LocalDateTime.parse("2023-01-21T18:00:00"), LocalDateTime.parse("2023-04-21T18:00:00"), - Duration.parse("P90DT3H4M")), - Arguments.of(LocalDateTime.parse("2023-01-21T18:00:00"), null, null), - Arguments.of(null, LocalDateTime.parse("2023-04-21T18:00:00"), null), - Arguments.of(null, null, Duration.parse("P90DT3H4M")) + Duration.parse("P90DT3H4M")) ); } @@ -164,6 +148,8 @@ private static Stream validBucketTimeRangeOptions() { LocalDateTime.parse("2023-01-21T18:00:00"), LocalDateTime.parse("2023-01-24T18:00:00")), Arguments.of(null, LocalDateTime.parse("2023-01-24T18:00:00"), Duration.ofDays(3L), LocalDateTime.parse("2023-01-21T18:00:00"), LocalDateTime.parse("2023-01-24T18:00:00")), + Arguments.of(LocalDateTime.parse("2023-01-21T18:00:00"), null, null, LocalDateTime.parse("2023-01-21T18:00:00"), null), + Arguments.of(null, LocalDateTime.parse("2023-01-21T18:00:00"), null, null, LocalDateTime.parse("2023-01-21T18:00:00")), Arguments.of(null, null, null, null, null) ); } @@ -189,30 +175,28 @@ private static Stream validCombinedTimeRangeOptions() { Arguments.of( LocalDateTime.parse("2023-01-21T18:00:00"), LocalDateTime.parse("2023-01-24T18:00:00"), null, LocalDateTime.parse("2023-05-21T18:00:00"), null, Duration.ofDays(3L), - LocalDateTime.parse("2023-05-21T18:00:00"), LocalDateTime.parse("2023-05-24T18:00:00")) - ); - } - - private static Stream invalidCombinedTimeRangeOptions() { - return Stream.of( - Arguments.of( - LocalDateTime.parse("2023-05-21T18:00:00"), null, null, - LocalDateTime.parse("2023-05-24T18:00:00"), null, null), - Arguments.of( - null, LocalDateTime.parse("2023-05-24T18:00:00"), null, - null, LocalDateTime.parse("2023-05-21T18:00:00"), null), + LocalDateTime.parse("2023-05-21T18:00:00"), LocalDateTime.parse("2023-05-24T18:00:00")), Arguments.of( + LocalDateTime.parse("2023-01-21T18:00:00"), LocalDateTime.parse("2023-01-24T18:00:00"), null, null, null, Duration.ofDays(3L), - null, null, Duration.ofDays(3L)), + LocalDateTime.parse("2023-01-21T18:00:00"), LocalDateTime.parse("2023-01-24T18:00:00")), Arguments.of( - LocalDateTime.parse("2023-01-21T18:00:00"), LocalDateTime.parse("2023-01-24T18:00:00"), null, - null, null, Duration.ofDays(3L)), + LocalDateTime.parse("2023-01-21T18:00:00"), LocalDateTime.parse("2023-12-24T18:00:00"), null, + LocalDateTime.parse("2023-05-21T18:00:00"), null, null, + LocalDateTime.parse("2023-05-21T18:00:00"), LocalDateTime.parse("2023-12-24T18:00:00")), Arguments.of( LocalDateTime.parse("2023-01-21T18:00:00"), null, Duration.ofDays(3L), - null, LocalDateTime.parse("2023-05-24T18:00:00"), null), + null, LocalDateTime.parse("2023-05-21T18:00:00"), null, + null, LocalDateTime.parse("2023-05-21T18:00:00")), Arguments.of( - null, LocalDateTime.parse("2023-01-24T18:00:00"), Duration.ofDays(3L), - LocalDateTime.parse("2023-05-21T18:00:00"), null, null), + null, LocalDateTime.parse("2023-01-21T18:00:00"), Duration.ofDays(3L), + LocalDateTime.parse("2023-05-21T18:00:00"), null, null, + LocalDateTime.parse("2023-05-21T18:00:00"), null) + ); + } + + private static Stream invalidCombinedTimeRangeOptions() { + return Stream.of( Arguments.of( LocalDateTime.parse("2023-01-21T18:00:00"), LocalDateTime.parse("2023-01-24T18:00:00"), Duration.ofDays(3L), LocalDateTime.parse("2023-05-21T18:00:00"), LocalDateTime.parse("2023-05-24T18:00:00"), Duration.ofDays(3L)) From 6949780d0e648f4974edc9b94ced4410b004e623 Mon Sep 17 00:00:00 2001 From: Taylor Gray Date: Wed, 16 Aug 2023 13:09:41 -0500 Subject: [PATCH 26/28] Fix NPE on s3 source stopping without sqs, stop s3 scan worker thread on stopping of the s3 source (#3178) Signed-off-by: Taylor Gray --- .../org/opensearch/dataprepper/pipeline/Pipeline.java | 2 +- .../dataprepper/plugins/source/S3ScanService.java | 4 ++++ .../opensearch/dataprepper/plugins/source/S3Source.java | 9 +++++++-- .../dataprepper/plugins/source/ScanObjectWorker.java | 5 +++-- 4 files changed, 15 insertions(+), 5 deletions(-) diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/Pipeline.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/Pipeline.java index f47a4da2ca..f41ab33d7c 100644 --- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/Pipeline.java +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/Pipeline.java @@ -274,7 +274,7 @@ public synchronized void shutdown() { stopRequested.set(true); } catch (Exception ex) { LOG.error("Pipeline [{}] - Encountered exception while stopping the source, " + - "proceeding with termination of process workers", name); + "proceeding with termination of process workers", name, ex); } shutdownExecutorService(processorExecutorService, processorShutdownTimeout.toMillis(), "processor"); diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanService.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanService.java index 19d8a9d679..9b2ba8999b 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanService.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanService.java @@ -65,6 +65,10 @@ public void start() { scanObjectWorkerThread.start(); } + public void stop() { + scanObjectWorkerThread.interrupt(); + } + /** * This Method Used to fetch the scan options details from {@link S3SourceConfig} amd build the * all the s3 scan buckets information in list. diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3Source.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3Source.java index 7a0717d707..2e1b861032 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3Source.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3Source.java @@ -131,8 +131,13 @@ public void start(Buffer> buffer) { @Override public void stop() { - sqsService.stop(); - if (Objects.nonNull(sourceCoordinator)) { + + if (Objects.nonNull(sqsService)) { + sqsService.stop(); + } + + if (Objects.nonNull(s3ScanService) && Objects.nonNull(sourceCoordinator)) { + s3ScanService.stop(); sourceCoordinator.giveUpPartitions(); } } diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/ScanObjectWorker.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/ScanObjectWorker.java index 39a251cddc..8e962a16d0 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/ScanObjectWorker.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/ScanObjectWorker.java @@ -65,7 +65,7 @@ public class ScanObjectWorker implements Runnable{ private final AcknowledgementSetManager acknowledgementSetManager; // Should there be a duration or time that is configured in the source to stop processing? Otherwise will only stop when data prepper is stopped - private final boolean shouldStopProcessing = false; + private boolean shouldStopProcessing = false; private final boolean deleteS3ObjectsOnRead; private final S3ObjectDeleteWorker s3ObjectDeleteWorker; private final PluginMetrics pluginMetrics; @@ -109,6 +109,7 @@ public void run() { Thread.sleep(RETRY_BACKOFF_ON_EXCEPTION_MILLIS); } catch (InterruptedException ex) { LOG.error("S3 Scan worker thread interrupted while backing off.", ex); + return; } } @@ -129,7 +130,7 @@ private void startProcessingObject(final int waitTimeMillis) { try { Thread.sleep(waitTimeMillis); } catch (InterruptedException e) { - e.printStackTrace(); + shouldStopProcessing = true; } return; } From b0e5006ed79b3ce8e6cff869c55c93e9a5e5d027 Mon Sep 17 00:00:00 2001 From: Taylor Gray Date: Wed, 16 Aug 2023 13:33:10 -0500 Subject: [PATCH 27/28] Only log that the scan is complete one time for s3 scan (#3168) Signed-off-by: Taylor Gray --- .../source/S3ScanPartitionCreationSupplier.java | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplier.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplier.java index 89b85c6b83..5ec4848354 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplier.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/S3ScanPartitionCreationSupplier.java @@ -36,6 +36,7 @@ public class S3ScanPartitionCreationSupplier implements Function globalStateMap) { globalStateMap.put(SCAN_COUNT, 0); + globalStateMap.put(SINGLE_SCAN_COMPLETE, false); } private boolean isLastModifiedTimeAfterMostRecentScanForBucket(final String bucketName, @@ -184,14 +186,22 @@ private Instant getMostRecentLastModifiedTimestamp(final ListObjectsV2Response l } private boolean shouldScanBeSkipped(final Map globalStateMap) { + if (Objects.isNull(schedulingOptions) && hasAlreadyBeenScanned(globalStateMap)) { - LOG.info("Skipping scan because the buckets have already been scanned once"); + + if (!(Boolean) globalStateMap.get(SINGLE_SCAN_COMPLETE)) { + LOG.info("Single S3 scan has already been completed"); + globalStateMap.put(SINGLE_SCAN_COMPLETE, true); + } + return true; } if (Objects.nonNull(schedulingOptions) && (hasReachedMaxScanCount(globalStateMap) || !hasReachedScheduledScanTime(globalStateMap))) { + + if (hasReachedMaxScanCount(globalStateMap)) { LOG.info("Skipping scan as the max scan count {} has been reached", schedulingOptions.getCount()); } else { From b7661e6413e8e0c0df9e751ccda6ba7c169a0193 Mon Sep 17 00:00:00 2001 From: David Venable Date: Wed, 16 Aug 2023 14:50:36 -0500 Subject: [PATCH 28/28] Adds a new integration test to the S3 sink which can test different scenarios. This currently is testing against ndjson since this codec generally works. (#3179) Signed-off-by: David Venable --- build.gradle | 2 +- data-prepper-plugins/s3-sink/build.gradle | 6 +- .../plugins/sink/s3/CompressionScenario.java | 19 ++ .../sink/s3/GZipCompressionScenario.java | 24 ++ .../plugins/sink/s3/NdjsonOutputScenario.java | 52 ++++ .../sink/s3/NoneCompressionScenario.java | 23 ++ .../plugins/sink/s3/OutputScenario.java | 34 +++ .../dataprepper/plugins/sink/s3/S3SinkIT.java | 261 ++++++++++++++++++ .../src/test/resources/log4j2.properties | 14 + 9 files changed, 433 insertions(+), 2 deletions(-) create mode 100644 data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/CompressionScenario.java create mode 100644 data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/GZipCompressionScenario.java create mode 100644 data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/NdjsonOutputScenario.java create mode 100644 data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/NoneCompressionScenario.java create mode 100644 data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/OutputScenario.java create mode 100644 data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkIT.java create mode 100644 data-prepper-plugins/s3-sink/src/test/resources/log4j2.properties diff --git a/build.gradle b/build.gradle index 006e641354..efc059810d 100644 --- a/build.gradle +++ b/build.gradle @@ -196,7 +196,7 @@ subprojects { configure(subprojects.findAll {it.name != 'data-prepper-api'}) { dependencies { - implementation platform('software.amazon.awssdk:bom:2.17.264') + implementation platform('software.amazon.awssdk:bom:2.20.67') implementation 'jakarta.validation:jakarta.validation-api:3.0.2' } } diff --git a/data-prepper-plugins/s3-sink/build.gradle b/data-prepper-plugins/s3-sink/build.gradle index 5d74fd169d..7f26ad6ed9 100644 --- a/data-prepper-plugins/s3-sink/build.gradle +++ b/data-prepper-plugins/s3-sink/build.gradle @@ -27,6 +27,8 @@ dependencies { testImplementation project(':data-prepper-plugins:parse-json-processor') testImplementation project(':data-prepper-plugins:csv-processor') testImplementation project(':data-prepper-plugins:avro-codecs') + testImplementation testLibs.slf4j.simple + testImplementation 'software.amazon.awssdk:s3-transfer-manager' } test { @@ -56,10 +58,12 @@ task integrationTest(type: Test) { useJUnitPlatform() classpath = sourceSets.integrationTest.runtimeClasspath + + systemProperty 'log4j.configurationFile', 'src/test/resources/log4j2.properties' systemProperty 'tests.s3sink.bucket', System.getProperty('tests.s3sink.bucket') systemProperty 'tests.s3sink.region', System.getProperty('tests.s3sink.region') filter { includeTestsMatching '*IT' } -} \ No newline at end of file +} diff --git a/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/CompressionScenario.java b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/CompressionScenario.java new file mode 100644 index 0000000000..72af20226d --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/CompressionScenario.java @@ -0,0 +1,19 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3; + +import org.opensearch.dataprepper.plugins.sink.s3.compression.CompressionOption; + +import java.io.IOException; +import java.io.InputStream; + +/** + * A scenario for whole-file compression. + */ +public interface CompressionScenario { + CompressionOption getCompressionOption(); + InputStream decompressingInputStream(final InputStream inputStream) throws IOException; +} diff --git a/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/GZipCompressionScenario.java b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/GZipCompressionScenario.java new file mode 100644 index 0000000000..b45e2f0d63 --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/GZipCompressionScenario.java @@ -0,0 +1,24 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3; + +import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream; +import org.opensearch.dataprepper.plugins.sink.s3.compression.CompressionOption; + +import java.io.IOException; +import java.io.InputStream; + +public class GZipCompressionScenario implements CompressionScenario { + @Override + public CompressionOption getCompressionOption() { + return CompressionOption.GZIP; + } + + @Override + public InputStream decompressingInputStream(final InputStream inputStream) throws IOException { + return new GzipCompressorInputStream(inputStream); + } +} diff --git a/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/NdjsonOutputScenario.java b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/NdjsonOutputScenario.java new file mode 100644 index 0000000000..dcc9079f8e --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/NdjsonOutputScenario.java @@ -0,0 +1,52 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.opensearch.dataprepper.model.codec.OutputCodec; +import org.opensearch.dataprepper.plugins.codec.json.NdjsonOutputCodec; +import org.opensearch.dataprepper.plugins.codec.json.NdjsonOutputConfig; + +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Scanner; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; + +public class NdjsonOutputScenario implements OutputScenario { + + public static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + @Override + public OutputCodec getCodec() { + return new NdjsonOutputCodec(new NdjsonOutputConfig()); + } + + @Override + public void validate(final List> allEventData, final File actualContentFile) throws IOException { + final FileInputStream fileInputStream = new FileInputStream(actualContentFile); + + final Scanner scanner = new Scanner(fileInputStream); + + int i = 0; + while (scanner.hasNext()) { + final Map expectedData = allEventData.get(i); + + final String actualJsonString = scanner.next(); + + final Map actualData = OBJECT_MAPPER.readValue(actualJsonString, Map.class); + + assertThat(actualData, equalTo(expectedData)); + i++; + } + + assertThat(i, equalTo(allEventData.size())); + } +} diff --git a/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/NoneCompressionScenario.java b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/NoneCompressionScenario.java new file mode 100644 index 0000000000..44a5030949 --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/NoneCompressionScenario.java @@ -0,0 +1,23 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3; + +import org.opensearch.dataprepper.plugins.sink.s3.compression.CompressionOption; + +import java.io.IOException; +import java.io.InputStream; + +public class NoneCompressionScenario implements CompressionScenario { + @Override + public CompressionOption getCompressionOption() { + return CompressionOption.NONE; + } + + @Override + public InputStream decompressingInputStream(final InputStream inputStream) throws IOException { + return inputStream; + } +} diff --git a/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/OutputScenario.java b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/OutputScenario.java new file mode 100644 index 0000000000..35bc446f30 --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/OutputScenario.java @@ -0,0 +1,34 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3; + +import org.opensearch.dataprepper.model.codec.OutputCodec; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Map; + +/** + * Represents a scenario for the output format. + */ +public interface OutputScenario { + /** + * Gets the codec this scenario uses. + * + * @return The {@link OutputCodec} + */ + OutputCodec getCodec(); + + /** + * Validates the data against all the events provided. + * + * @param allEventData The collection of all the expected event maps. + * @param actualContentFile The actual file which has been downloaded and decompressed as part of the test + * @throws IOException Some IOException + */ + void validate(List> allEventData, File actualContentFile) throws IOException; +} diff --git a/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkIT.java b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkIT.java new file mode 100644 index 0000000000..2175fdc79b --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkIT.java @@ -0,0 +1,261 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3; + +import org.apache.commons.io.IOUtils; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.ExtensionContext; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.ArgumentsProvider; +import org.junit.jupiter.params.provider.ArgumentsSource; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +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.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.plugin.PluginFactory; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.sink.SinkContext; +import org.opensearch.dataprepper.model.types.ByteCount; +import org.opensearch.dataprepper.plugins.sink.s3.accumulator.BufferTypeOptions; +import org.opensearch.dataprepper.plugins.sink.s3.configuration.AwsAuthenticationOptions; +import org.opensearch.dataprepper.plugins.sink.s3.configuration.ObjectKeyOptions; +import org.opensearch.dataprepper.plugins.sink.s3.configuration.ThresholdOptions; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.s3.S3AsyncClient; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Response; +import software.amazon.awssdk.services.s3.model.S3Object; +import software.amazon.awssdk.transfer.s3.S3TransferManager; +import software.amazon.awssdk.transfer.s3.model.DownloadFileRequest; +import software.amazon.awssdk.transfer.s3.model.FileDownload; + +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.time.Duration; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.params.provider.Arguments.arguments; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +public class S3SinkIT { + @Mock + private PluginSetting pluginSetting; + @Mock + private S3SinkConfig s3SinkConfig; + @Mock + private PluginFactory pluginFactory; + @Mock + private SinkContext sinkContext; + @Mock + private AwsCredentialsSupplier awsCredentialsSupplier; + + @Mock + private ThresholdOptions thresholdOptions; + @Mock + private ObjectKeyOptions objectKeyOptions; + private String s3region; + private String bucketName; + private S3Client s3Client; + + @TempDir + private File s3FileLocation; + private S3TransferManager transferManager; + + @BeforeEach + void setUp() { + when(pluginSetting.getPipelineName()).thenReturn(UUID.randomUUID().toString()); + when(pluginSetting.getName()).thenReturn("s3"); + + s3region = System.getProperty("tests.s3sink.region"); + + final AwsCredentialsProvider awsCredentialsProvider = DefaultCredentialsProvider.create(); + + final Region region = Region.of(s3region); + s3Client = S3Client.builder().region(region).build(); + bucketName = System.getProperty("tests.s3sink.bucket"); + + when(s3SinkConfig.getBucketName()).thenReturn(bucketName); + when(objectKeyOptions.getNamePattern()).thenReturn("events-%{yyyy-MM-dd'T'hh-mm-ss}"); + when(s3SinkConfig.getObjectKeyOptions()).thenReturn(objectKeyOptions); + + when(s3SinkConfig.getThresholdOptions()).thenReturn(thresholdOptions); + when(thresholdOptions.getEventCollectTimeOut()).thenReturn(Duration.ofDays(1)); + when(thresholdOptions.getMaximumSize()).thenReturn(ByteCount.parse("1gb")); + + final PluginModel pluginModel = mock(PluginModel.class); + when(s3SinkConfig.getCodec()).thenReturn(pluginModel); + when(pluginModel.getPluginName()).thenReturn(UUID.randomUUID().toString()); + when(pluginModel.getPluginSettings()).thenReturn(Collections.emptyMap()); + + final AwsAuthenticationOptions awsAuthenticationOptions = mock(AwsAuthenticationOptions.class); + when(s3SinkConfig.getAwsAuthenticationOptions()).thenReturn(awsAuthenticationOptions); + when(awsAuthenticationOptions.getAwsRegion()).thenReturn(region); + + when(awsCredentialsSupplier.getProvider(any())).thenReturn(awsCredentialsProvider); + + final S3AsyncClient s3AsyncClient = S3AsyncClient + .builder() + .credentialsProvider(awsCredentialsProvider) + .region(region) + .build(); + + transferManager = S3TransferManager.builder() + .s3Client(s3AsyncClient) + .build(); + } + + private S3Sink createObjectUnderTest() { + return new S3Sink(pluginSetting, s3SinkConfig, pluginFactory, sinkContext, awsCredentialsSupplier); + } + + @ParameterizedTest + @ArgumentsSource(IntegrationTestArguments.class) + void test(final OutputScenario outputScenario, final BufferTypeOptions bufferTypeOptions, final CompressionScenario compressionScenario, final int batchSize, final int numberOfBatches) throws IOException { + + final String pathPrefix = Instant.now().toString() + "-" + UUID.randomUUID(); + when(objectKeyOptions.getPathPrefix()).thenReturn(pathPrefix + "/"); + + when(pluginFactory.loadPlugin(eq(OutputCodec.class), any())).thenReturn(outputScenario.getCodec()); + when(s3SinkConfig.getBufferType()).thenReturn(bufferTypeOptions); + when(s3SinkConfig.getCompression()).thenReturn(compressionScenario.getCompressionOption()); + when(thresholdOptions.getEventCount()).thenReturn(batchSize * numberOfBatches); + + final S3Sink objectUnderTest = createObjectUnderTest(); + + final List> allEventData = new ArrayList<>(batchSize * numberOfBatches); + for (int batchNumber = 0; batchNumber < numberOfBatches; batchNumber++) { + final int currentBatchNumber = batchNumber; + final List> events = IntStream.range(0, batchSize) + .mapToObj(sequence -> generateEventData(currentBatchNumber * sequence)) + .peek(allEventData::add) + .map(this::generateTestEvent) + .map(Record::new) + .collect(Collectors.toList()); + + objectUnderTest.doOutput(events); + } + + assertThat(allEventData.size(), equalTo(batchSize * numberOfBatches)); + + final ListObjectsV2Response listObjectsResponse = s3Client.listObjectsV2(ListObjectsV2Request.builder() + .bucket(bucketName) + .prefix(pathPrefix) + .build()); + + assertThat(listObjectsResponse.contents(), notNullValue()); + assertThat(listObjectsResponse.contents().size(), equalTo(1)); + + final S3Object s3Object = listObjectsResponse.contents().get(0); + + final File target = new File(s3FileLocation, pathPrefix + ".original"); + + final FileDownload fileDownload = transferManager.downloadFile(DownloadFileRequest.builder() + .destination(target) + .getObjectRequest(GetObjectRequest.builder() + .bucket(bucketName) + .key(s3Object.key()) + .build()) + .build()); + + fileDownload.completionFuture().join(); + + final File actualContentFile = new File(s3FileLocation, pathPrefix + ".content"); + IOUtils.copy( + compressionScenario.decompressingInputStream(new FileInputStream(target)), + new FileOutputStream(actualContentFile)); + + outputScenario.validate(allEventData, actualContentFile); + } + + private Event generateTestEvent(final Map eventData) { + final EventMetadata defaultEventMetadata = DefaultEventMetadata.builder() + .withEventType(EventType.LOG.toString()) + .build(); + final JacksonEvent event = JacksonLog.builder().withData(eventData).withEventMetadata(defaultEventMetadata).build(); + event.setEventHandle(mock(EventHandle.class)); + return JacksonEvent.builder() + .withData(eventData) + .withEventMetadata(defaultEventMetadata) + .build(); + } + + private static Map generateEventData(final int sequence) { + final Map eventDataMap = new LinkedHashMap<>(); + eventDataMap.put("sequence", sequence); + for (int i = 0; i < 2; i++) { + eventDataMap.put("field" + i, UUID.randomUUID().toString()); + eventDataMap.put("float" + i, (float) i * 1.1); + } + for (int i = 0; i < 2; i++) { + eventDataMap.put("list" + i, + List.of(UUID.randomUUID().toString(), UUID.randomUUID().toString(), UUID.randomUUID().toString())); + } + return eventDataMap; + } + + static class IntegrationTestArguments implements ArgumentsProvider { + @Override + public Stream provideArguments(final ExtensionContext context) { + final List bufferTypeOptions = Arrays.asList(BufferTypeOptions.values()); + final List outputScenarios = List.of( + new NdjsonOutputScenario()); + final List compressionScenarios = List.of( + new NoneCompressionScenario(), + new GZipCompressionScenario() + ); + final List numberOfRecordsPerBatchList = List.of(1, 25, 500); + final List numberOfBatchesList = List.of(1, 25); + + return outputScenarios + .stream() + .flatMap(outputScenario -> bufferTypeOptions + .stream() + .flatMap(bufferTypeOption -> compressionScenarios + .stream() + .flatMap(compressionScenario -> numberOfRecordsPerBatchList + .stream() + .flatMap(batchRecordCount -> numberOfBatchesList + .stream() + .map(batchCount -> arguments(outputScenario, bufferTypeOption, compressionScenario, batchRecordCount, batchCount)) + )))); + } + } +} diff --git a/data-prepper-plugins/s3-sink/src/test/resources/log4j2.properties b/data-prepper-plugins/s3-sink/src/test/resources/log4j2.properties new file mode 100644 index 0000000000..2c8c1cd296 --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/test/resources/log4j2.properties @@ -0,0 +1,14 @@ +# +# Copyright OpenSearch Contributors +# SPDX-License-Identifier: Apache-2.0 +# + +appender.console.type = Console +appender.console.name = STDOUT +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{ISO8601} [%t] %-5p %40C - %m%n + + +rootLogger.level = debug +rootLogger.appenderRef.stdout.ref = STDOUT +